You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2022/04/26 05:16:58 UTC

[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2569: Single node prop store refactor

ctubbsii commented on code in PR #2569:
URL: https://github.com/apache/accumulo/pull/2569#discussion_r857982934


##########
test/src/main/resources/accumulo.properties:
##########
@@ -0,0 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+# File needs to exist for unit tests

Review Comment:
   What integration test requires this to exist? Most of the ITs in this class use MiniAccumuloCluster which provides its own properties file.



##########
test/src/test/resources/accumulo.properties:
##########
@@ -0,0 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+# File needs to exist for unit tests

Review Comment:
   What unit test requires this new file?



##########
test/src/main/java/org/apache/accumulo/test/conf/util/ConfigPropertyUpgraderTest.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test.conf.util;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER;
+import static org.easymock.EasyMock.createNiceMock;
+import static org.easymock.EasyMock.expect;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.io.File;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
+import org.apache.accumulo.server.conf.util.ConfigPropertyUpgrader;
+import org.apache.accumulo.test.conf.store.PropStoreZooKeeperIT;
+import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZKUtil;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Tag(ZOOKEEPER_TESTING_SERVER)
+class ConfigPropertyUpgraderTest {
+
+  private static final Logger log = LoggerFactory.getLogger(PropStoreZooKeeperIT.class);
+  private static ZooKeeperTestingServer testZk = null;
+  private static ZooKeeper zooKeeper;
+  private static ZooReaderWriter zrw;
+
+  private InstanceId instanceId = null;
+
+  @TempDir
+  private static File tempDir;
+
+  @BeforeAll
+  public static void setupZk() {
+
+    // using default zookeeper port - we don't have a full configuration
+    testZk = new ZooKeeperTestingServer(tempDir);
+    zooKeeper = testZk.getZooKeeper();
+    zooKeeper.addAuthInfo("digest", "accumulo:test".getBytes(UTF_8));
+
+    zrw = testZk.getZooReaderWriter();
+
+  }
+
+  @AfterAll
+  public static void shutdownZK() throws Exception {
+    testZk.close();
+  }
+
+  @BeforeEach
+  public void setupZnodes() throws Exception {
+
+    instanceId = InstanceId.of(UUID.randomUUID());
+
+    testZk.initPaths(ZooUtil.getRoot(instanceId));
+
+    ServerContext context = createNiceMock(ServerContext.class);
+    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
+    expect(context.getZooKeepersSessionTimeOut()).andReturn(zooKeeper.getSessionTimeout())
+        .anyTimes();
+    expect(context.getInstanceID()).andReturn(instanceId).anyTimes();
+
+    List<LegacyPropData.PropNode> nodes = LegacyPropData.getData(instanceId);
+    for (LegacyPropData.PropNode node : nodes) {
+      zrw.putPersistentData(node.getPath(), node.getData(), ZooUtil.NodeExistsPolicy.SKIP);
+    }
+
+    try {
+      zrw.putPersistentData(ZooUtil.getRoot(instanceId) + Constants.ZCONFIG, new byte[0],
+          ZooUtil.NodeExistsPolicy.SKIP);
+    } catch (KeeperException ex) {
+      log.trace("Issue during zk initialization, skipping", ex);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("Interrupted during zookeeper path initialization", ex);
+    }
+  }
+
+  @AfterEach
+  public void cleanupZnodes() {
+    try {
+      ZKUtil.deleteRecursive(zooKeeper, Constants.ZROOT);
+    } catch (KeeperException | InterruptedException ex) {
+      throw new IllegalStateException("Failed to clean-up test zooKeeper nodes.", ex);
+    }
+  }
+
+  @Test
+  void doUpgrade() {
+    ConfigPropertyUpgrader upgrader = new ConfigPropertyUpgrader();
+    upgrader.doUpgrade(instanceId, zrw);
+
+    PropStore propStore =
+        new ZooPropStore.Builder(instanceId, zrw, zooKeeper.getSessionTimeout()).build();
+
+    var sysKey = PropCacheKey.forSystem(instanceId);
+    log.info("PropStore: {}", propStore.get(sysKey));
+
+    var vProps = propStore.get(sysKey);
+    if (vProps == null) {
+      fail("unexpected null returned from prop store get for " + sysKey);
+      return; // keep spotbugs happy
+    }

Review Comment:
   Easier way to make spotbugs happy:
   
   ```suggestion
       assertNotNull(vProps, "unexpected null returned from prop store get for " + sysKey);
   ```



##########
test/src/main/java/org/apache/accumulo/test/conf/util/TransformTokenTest.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test.conf.util;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.File;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStoreException;
+import org.apache.accumulo.server.conf.store.impl.PropStoreWatcher;
+import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
+import org.apache.accumulo.server.conf.util.TransformToken;
+import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZKUtil;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+@Tag(ZOOKEEPER_TESTING_SERVER)
+public class TransformTokenTest {
+
+  @TempDir
+  private static File tempDir;
+
+  private static ZooKeeperTestingServer testZk = null;
+  private static ZooKeeper zooKeeper;
+  private static ZooReaderWriter zrw;
+  private InstanceId instanceId = null;
+
+  @BeforeAll
+  public static void setupZk() {
+
+    // using default zookeeper port - we don't have a full configuration
+    testZk = new ZooKeeperTestingServer(tempDir);
+    zooKeeper = testZk.getZooKeeper();
+    zrw = testZk.getZooReaderWriter();
+  }
+
+  @AfterAll
+  public static void shutdownZK() throws Exception {
+    testZk.close();
+  }
+
+  @BeforeEach
+  public void testSetup() {
+    instanceId = InstanceId.of(UUID.randomUUID());
+  }
+
+  @AfterEach
+  public void cleanupZnodes() {
+    try {
+      ZKUtil.deleteRecursive(zooKeeper, Constants.ZROOT);
+    } catch (KeeperException | InterruptedException ex) {
+      throw new IllegalStateException("Failed to clean-up test zooKeeper nodes.", ex);
+    }
+  }
+
+  @Test
+  public void tokenGoPathTest() throws Exception {
+
+    List<LegacyPropData.PropNode> nodes = LegacyPropData.getData(instanceId);
+    for (LegacyPropData.PropNode node : nodes) {
+      zrw.putPersistentData(node.getPath(), node.getData(), ZooUtil.NodeExistsPolicy.SKIP);
+    }
+
+    ZooPropStore propStore = new ZooPropStore.Builder(instanceId, zrw, 30_000).build();
+
+    ServerContext context = createMock(ServerContext.class);
+    expect(context.getInstanceID()).andReturn(instanceId).anyTimes();
+    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
+    expect(context.getPropStore()).andReturn(propStore).anyTimes();
+
+    PropStoreWatcher watcher = createMock(PropStoreWatcher.class);
+
+    replay(context, watcher);
+
+    var sysPropKey = PropCacheKey.forSystem(instanceId);
+
+    TransformToken token = TransformToken.createToken(sysPropKey, zrw);
+
+    assertTrue(token.haveToken());
+    token.releaseToken();
+    assertFalse(token.haveToken());
+
+    // relock by getting a new lock
+    TransformToken lock2 = TransformToken.createToken(sysPropKey, zrw);
+    assertTrue(lock2.haveToken());
+
+    // fail with a current lock node present
+    TransformToken lock3 = TransformToken.createToken(sysPropKey, zrw);
+    assertFalse(lock3.haveToken());
+

Review Comment:
   ```suggestion
       assertFalse(lock3.haveToken());
   
       // confirm current lock is still present
       assertTrue(lock2.haveToken());
   
       verify(context, watcher);
   ```



##########
test/src/main/java/org/apache/accumulo/test/conf/PropStoreConfigTest.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test.conf;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.Map;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Tag(MINI_CLUSTER_ONLY)
+@Tag(SUNNY_DAY)
+public class PropStoreConfigTest extends AccumuloClusterHarness {
+
+  private static final Logger log = LoggerFactory.getLogger(PropStoreConfigTest.class);
+
+  private AccumuloClient accumuloClient;
+
+  @BeforeEach
+  public void setup() {
+    accumuloClient = Accumulo.newClient().from(getClientProps()).build();
+  }
+
+  @AfterAll
+  public static void teardown() {
+    SharedMiniClusterBase.stopMiniCluster();
+  }

Review Comment:
   This class extends AccumuloClusterHarness. A separate mini instance is created for each test method and shut down after each one. So, it isn't necessary to stop it this way after all the tests. This is only applicable if you were using SharedMiniClusterBase as the parent class instead of AccumuloClusterHarness, in which case, you would also have a `@BeforeAll` method to call `SharedMiniClusterBase.startMiniCluster()`.



##########
server/manager/src/main/java/org/apache/accumulo/manager/Manager.java:
##########
@@ -260,6 +261,18 @@ synchronized void setManagerState(ManagerState newState) {
       ThreadPools.watchNonCriticalScheduledTask(future);
     }
 
+    if (oldState != newState && (newState == ManagerState.HAVE_LOCK)) {
+      try {
+        log.info("Starting property conversion");
+        var context = getContext();
+        ConfigPropertyUpgrader configUpgrader = new ConfigPropertyUpgrader();
+        configUpgrader.doUpgrade(context.getInstanceID(), context.getZooReaderWriter());
+        log.info("Completed property conversion");
+      } catch (Exception ex) {

Review Comment:
   Can we catch a more narrow exception here?



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader9to10.java:
##########
@@ -484,11 +452,10 @@ static Map<String,DataFileValue> cleanupRootTabletFiles(VolumeManager fs, String
   public void upgradeFileDeletes(ServerContext context, Ample.DataLevel level) {
 
     String tableName = level.metaTable();
-    AccumuloClient c = context;

Review Comment:
   I agree with @dlmarion that this doesn't look fixed. I think this and the changes below here should be reverted. The code was much cleaner before the changes below this point in the file. Also a context is already a client... there isn't a need to cast it at all. It was being assigned to an AccumuloClient type because that made it obvious that the places where it is used below only need the more narrow type, and to make the code more readable. The casting makes it much worse to read.



##########
core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java:
##########
@@ -134,6 +134,34 @@ public boolean putPersistentData(String zPath, byte[] data, NodeExistsPolicy pol
         e -> e.code() == Code.NONODE && policy == NodeExistsPolicy.OVERWRITE);
   }
 
+  /**
+   * Overwrite a persistent node if the data version matches.
+   *
+   * @param zPath
+   *          the zookeeper path
+   * @param data
+   *          the byte array data
+   * @param expectedVersion
+   *          the expected data version of the zookeeper node.
+   * @return true if the data was set, false if the version does not match expected.
+   * @throws KeeperException
+   *           if a KeeperException occurs (no node most likely)
+   * @throws InterruptedException
+   *           if the zookeeper write is interrupted.
+   */
+  public boolean overwritePersistentData(String zPath, byte[] data, final int expectedVersion)
+      throws KeeperException, InterruptedException {
+    // zk allows null ACLs, but it's probably a bug in Accumulo if we see it used in our code

Review Comment:
   This comment doesn't really apply to this code.
   
   ```suggestion
   ```



##########
test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java:
##########
@@ -113,6 +115,8 @@ public void testExpectedClasses() {
     expectSet.put("check-server-config", CheckServerConfig.class);
     expectSet.put("compaction-coordinator", CoordinatorExecutable.class);
     expectSet.put("compactor", CompactorExecutable.class);
+    expectSet.put("config-print", ConfigPropertyPrinter.class);
+    expectSet.put("config-upgrade", ConfigPropertyUpgrader.class);

Review Comment:
   Upgrading the configuration is a somewhat one-off upgrade task. But, as a utility, It may be useful in future upgrades. Maybe this should have a name that is more generic, like `pre-start-upgrade` (or a better name), so it can be re-used in future for similar upgrade tasks.



##########
test/src/main/java/org/apache/accumulo/test/zookeeper/ZooKeeperTestingServer.java:
##########
@@ -119,7 +119,7 @@ public void initPaths(String s) {
       for (String p : paths) {
         if (!p.isEmpty()) {
           path = path + slash + p;
-          log.debug("building default paths, creating node {}", path);
+          log.warn("building default paths, creating node {}", path);

Review Comment:
   I'm guessing this was changed to make it more visible while testing. It can be changed back.



##########
test/src/main/java/org/apache/accumulo/test/conf/util/ConfigPropertyUpgraderTest.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test.conf.util;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER;
+import static org.easymock.EasyMock.createNiceMock;
+import static org.easymock.EasyMock.expect;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.io.File;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
+import org.apache.accumulo.server.conf.util.ConfigPropertyUpgrader;
+import org.apache.accumulo.test.conf.store.PropStoreZooKeeperIT;
+import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZKUtil;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Tag(ZOOKEEPER_TESTING_SERVER)
+class ConfigPropertyUpgraderTest {
+
+  private static final Logger log = LoggerFactory.getLogger(PropStoreZooKeeperIT.class);
+  private static ZooKeeperTestingServer testZk = null;
+  private static ZooKeeper zooKeeper;
+  private static ZooReaderWriter zrw;
+
+  private InstanceId instanceId = null;
+
+  @TempDir
+  private static File tempDir;
+
+  @BeforeAll
+  public static void setupZk() {
+
+    // using default zookeeper port - we don't have a full configuration
+    testZk = new ZooKeeperTestingServer(tempDir);
+    zooKeeper = testZk.getZooKeeper();
+    zooKeeper.addAuthInfo("digest", "accumulo:test".getBytes(UTF_8));
+
+    zrw = testZk.getZooReaderWriter();
+
+  }
+
+  @AfterAll
+  public static void shutdownZK() throws Exception {
+    testZk.close();
+  }
+
+  @BeforeEach
+  public void setupZnodes() throws Exception {
+
+    instanceId = InstanceId.of(UUID.randomUUID());
+
+    testZk.initPaths(ZooUtil.getRoot(instanceId));
+
+    ServerContext context = createNiceMock(ServerContext.class);
+    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
+    expect(context.getZooKeepersSessionTimeOut()).andReturn(zooKeeper.getSessionTimeout())
+        .anyTimes();
+    expect(context.getInstanceID()).andReturn(instanceId).anyTimes();
+
+    List<LegacyPropData.PropNode> nodes = LegacyPropData.getData(instanceId);
+    for (LegacyPropData.PropNode node : nodes) {
+      zrw.putPersistentData(node.getPath(), node.getData(), ZooUtil.NodeExistsPolicy.SKIP);
+    }
+
+    try {
+      zrw.putPersistentData(ZooUtil.getRoot(instanceId) + Constants.ZCONFIG, new byte[0],
+          ZooUtil.NodeExistsPolicy.SKIP);
+    } catch (KeeperException ex) {
+      log.trace("Issue during zk initialization, skipping", ex);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("Interrupted during zookeeper path initialization", ex);
+    }
+  }
+
+  @AfterEach
+  public void cleanupZnodes() {
+    try {
+      ZKUtil.deleteRecursive(zooKeeper, Constants.ZROOT);
+    } catch (KeeperException | InterruptedException ex) {
+      throw new IllegalStateException("Failed to clean-up test zooKeeper nodes.", ex);
+    }

Review Comment:
   No need to bother with the catching and throwing RTE:
   
   ```suggestion
     public void cleanupZnodes() throws KeeperException, InterruptedException {
       ZKUtil.deleteRecursive(zooKeeper, Constants.ZROOT);
   ```



##########
test/src/main/java/org/apache/accumulo/test/conf/util/ConfigTransformerTest.java:
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test.conf.util;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.MINUTES;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.io.File;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.fate.util.Retry;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+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.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStoreException;
+import org.apache.accumulo.server.conf.store.impl.PropStoreWatcher;
+import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
+import org.apache.accumulo.server.conf.util.ConfigTransformer;
+import org.apache.accumulo.server.conf.util.TransformToken;
+import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZKUtil;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Tag(ZOOKEEPER_TESTING_SERVER)
+public class ConfigTransformerTest {
+
+  private static final Logger log = LoggerFactory.getLogger(ConfigTransformerTest.class);
+  private final static VersionedPropCodec codec = VersionedPropCodec.getDefault();
+  @TempDir
+  private static File tempDir;
+  private static ZooKeeperTestingServer testZk = null;
+  private static ZooKeeper zooKeeper;
+  private static ZooReaderWriter zrw;
+
+  private InstanceId instanceId = null;
+  private ZooPropStore propStore = null;
+  private PropStoreWatcher watcher = null;
+
+  @BeforeAll
+  public static void setupZk() {
+
+    // using default zookeeper port - we don't have a full configuration
+    testZk = new ZooKeeperTestingServer(tempDir);
+    zooKeeper = testZk.getZooKeeper();
+    zrw = testZk.getZooReaderWriter();
+  }
+
+  @AfterAll
+  public static void shutdownZK() throws Exception {
+    testZk.close();
+  }
+
+  @BeforeEach
+  public void testSetup() throws Exception {
+    instanceId = InstanceId.of(UUID.randomUUID());
+
+    List<LegacyPropData.PropNode> nodes = LegacyPropData.getData(instanceId);
+    for (LegacyPropData.PropNode node : nodes) {
+      zrw.putPersistentData(node.getPath(), node.getData(), ZooUtil.NodeExistsPolicy.SKIP);
+    }
+
+    propStore = new ZooPropStore.Builder(instanceId, zrw, 30_000).build();
+
+    ServerContext context = createMock(ServerContext.class);
+    expect(context.getInstanceID()).andReturn(instanceId).anyTimes();
+    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
+    expect(context.getPropStore()).andReturn(propStore).anyTimes();
+
+    watcher = createMock(PropStoreWatcher.class);
+
+    replay(context, watcher);
+
+  }
+
+  @AfterEach
+  public void cleanupZnodes() {
+    try {

Review Comment:
   Don't forget to make sure that you call verify on any mock objects you create before each test ends. The EasyMock lifecycle is: `create -> expect -> reply -> verify`
   
   ```suggestion
       verify(context, watcher);
       try {
   ```



##########
test/src/main/java/org/apache/accumulo/test/conf/util/ConfigTransformerTest.java:
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test.conf.util;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.MINUTES;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.io.File;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.fate.util.Retry;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+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.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStoreException;
+import org.apache.accumulo.server.conf.store.impl.PropStoreWatcher;
+import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
+import org.apache.accumulo.server.conf.util.ConfigTransformer;
+import org.apache.accumulo.server.conf.util.TransformToken;
+import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZKUtil;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Tag(ZOOKEEPER_TESTING_SERVER)
+public class ConfigTransformerTest {
+
+  private static final Logger log = LoggerFactory.getLogger(ConfigTransformerTest.class);
+  private final static VersionedPropCodec codec = VersionedPropCodec.getDefault();
+  @TempDir
+  private static File tempDir;
+  private static ZooKeeperTestingServer testZk = null;
+  private static ZooKeeper zooKeeper;
+  private static ZooReaderWriter zrw;
+
+  private InstanceId instanceId = null;
+  private ZooPropStore propStore = null;
+  private PropStoreWatcher watcher = null;
+
+  @BeforeAll
+  public static void setupZk() {
+
+    // using default zookeeper port - we don't have a full configuration
+    testZk = new ZooKeeperTestingServer(tempDir);
+    zooKeeper = testZk.getZooKeeper();
+    zrw = testZk.getZooReaderWriter();
+  }
+
+  @AfterAll
+  public static void shutdownZK() throws Exception {
+    testZk.close();
+  }
+
+  @BeforeEach
+  public void testSetup() throws Exception {
+    instanceId = InstanceId.of(UUID.randomUUID());
+
+    List<LegacyPropData.PropNode> nodes = LegacyPropData.getData(instanceId);
+    for (LegacyPropData.PropNode node : nodes) {
+      zrw.putPersistentData(node.getPath(), node.getData(), ZooUtil.NodeExistsPolicy.SKIP);
+    }
+
+    propStore = new ZooPropStore.Builder(instanceId, zrw, 30_000).build();
+
+    ServerContext context = createMock(ServerContext.class);
+    expect(context.getInstanceID()).andReturn(instanceId).anyTimes();
+    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
+    expect(context.getPropStore()).andReturn(propStore).anyTimes();
+
+    watcher = createMock(PropStoreWatcher.class);
+
+    replay(context, watcher);
+
+  }
+
+  @AfterEach
+  public void cleanupZnodes() {
+    try {
+      ZKUtil.deleteRecursive(zooKeeper, Constants.ZROOT);
+    } catch (KeeperException | InterruptedException ex) {
+      throw new IllegalStateException("Failed to clean-up test zooKeeper nodes.", ex);
+    }
+  }
+
+  @Test
+  public void propStoreConversionTest() throws Exception {
+
+    var sysPropKey = PropCacheKey.forSystem(instanceId);
+
+    List<String> sysLegacy = zrw.getChildren(sysPropKey.getBasePath());
+    log.info("Before: {}", sysLegacy);
+
+    var vProps = propStore.get(sysPropKey);
+    assertNotNull(vProps);
+    log.info("Converted: {}", vProps);
+
+    sysLegacy = zrw.getChildren(sysPropKey.getBasePath());
+    log.info("After: {}", sysLegacy);
+
+  }
+
+  @Test
+  public void transformTest() throws Exception {
+
+    var sysPropKey = PropCacheKey.forSystem(instanceId);
+
+    ConfigTransformer transformer = new ConfigTransformer(zrw, codec, watcher);
+    List<String> sysLegacy = zrw.getChildren(sysPropKey.getBasePath());
+    log.info("Before: {}", sysLegacy);
+
+    var converted = transformer.transform(sysPropKey);
+
+    assertEquals(sysLegacy.size(), converted.getProperties().size());
+  }
+
+  @Test
+  public void failToGetLock() throws Exception {
+    var sysPropKey = PropCacheKey.forSystem(instanceId);
+
+    Retry retry =
+        Retry.builder().maxRetries(3).retryAfter(250, MILLISECONDS).incrementBy(500, MILLISECONDS)
+            .maxWait(5, SECONDS).backOffFactor(1.75).logInterval(3, MINUTES).createRetry();
+
+    ConfigTransformer transformer = new ConfigTransformer(zrw, codec, watcher, retry);
+    // manually create a lock so transformer fails
+    zrw.putEphemeralData(sysPropKey.getBasePath() + TransformToken.TRANSFORM_TOKEN, new byte[0]);
+
+    assertThrows(PropStoreException.class, () -> transformer.transform(sysPropKey));
+
+  }
+
+  @Test
+  public void continueOnLockRelease() {
+
+  }
+
+  @Test
+  public void createdByAnother() {
+
+  }
+

Review Comment:
   ```suggestion
   ```



##########
test/src/main/java/org/apache/accumulo/test/conf/store/PropCacheCaffeineImplZkTest.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test.conf.store;
+
+import static org.apache.accumulo.core.conf.Property.GC_PORT;
+import static org.apache.accumulo.core.conf.Property.MANAGER_CLIENTPORT;
+import static org.apache.accumulo.core.conf.Property.TSERV_CLIENTPORT;
+import static org.apache.accumulo.core.conf.Property.TSERV_NATIVEMAP_ENABLED;
+import static org.apache.accumulo.core.conf.Property.TSERV_SCAN_MAX_OPENFILES;
+import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.metrics.MetricsUtil;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+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.impl.PropCacheCaffeineImpl;
+import org.apache.accumulo.server.conf.store.impl.PropStoreMetrics;
+import org.apache.accumulo.server.conf.store.impl.PropStoreWatcher;
+import org.apache.accumulo.server.conf.store.impl.ReadyMonitor;
+import org.apache.accumulo.server.conf.store.impl.ZooPropLoader;
+import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZKUtil;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.easymock.EasyMock;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Tag(ZOOKEEPER_TESTING_SERVER)
+public class PropCacheCaffeineImplZkTest {
+
+  private static final Logger log = LoggerFactory.getLogger(PropCacheCaffeineImplZkTest.class);
+  private static final InstanceId INSTANCE_ID = InstanceId.of(UUID.randomUUID());
+
+  private static ZooKeeperTestingServer testZk = null;
+  private static ZooReaderWriter zrw;
+  private static ZooKeeper zooKeeper;
+
+  private final TableId tIdA = TableId.of("A");
+  private final TableId tIdB = TableId.of("B");
+  private final PropStoreMetrics cacheMetrics = new PropStoreMetrics();
+
+  @TempDir
+  private static File tempDir;
+
+  @BeforeAll
+  public static void setupZk() {
+    // using default zookeeper port - we don't have a full configuration
+    testZk = new ZooKeeperTestingServer(tempDir);
+    zooKeeper = testZk.getZooKeeper();
+
+    zrw = testZk.getZooReaderWriter();
+    ServerContext context = EasyMock.createNiceMock(ServerContext.class);
+    EasyMock.expect(context.getInstanceID()).andReturn(INSTANCE_ID).anyTimes();
+    EasyMock.expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
+
+    EasyMock.replay(context);
+  }
+
+  @AfterAll
+  public static void shutdownZK() throws Exception {
+    testZk.close();
+  }
+
+  @BeforeEach
+  public void setupZnodes() {
+    testZk.initPaths(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZCONFIG);
+    try {
+      zooKeeper.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES, new byte[0],
+          ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+      zooKeeper.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tIdA.canonical(),
+          new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+      zooKeeper.create(
+          ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tIdA.canonical() + "/conf",
+          new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+      zooKeeper.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tIdB.canonical(),
+          new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+      zooKeeper.create(
+          ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tIdB.canonical() + "/conf",
+          new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+    } catch (KeeperException ex) {
+      log.trace("Issue during zk initialization, skipping", ex);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("Interrupted during zookeeper path initialization", ex);
+    }
+  }
+
+  @AfterEach
+  public void cleanupZnodes() {
+    try {
+      ZKUtil.deleteRecursive(zooKeeper, "/accumulo");
+    } catch (KeeperException | InterruptedException ex) {
+      throw new IllegalStateException("Failed to clean-up test zooKeeper nodes.", ex);
+    }
+  }
+
+  @Test
+  public void init() throws Exception {
+    Map<String,String> props = new HashMap<>();
+    props.put(TSERV_CLIENTPORT.getKey(), "1234");
+    props.put(TSERV_NATIVEMAP_ENABLED.getKey(), "false");
+    props.put(TSERV_SCAN_MAX_OPENFILES.getKey(), "2345");
+    props.put(MANAGER_CLIENTPORT.getKey(), "3456");
+    props.put(GC_PORT.getKey(), "4567");
+    VersionedProperties vProps = new VersionedProperties(props);
+
+    // directly create prop node - simulate existing properties.
+    PropCacheKey propCacheKey = PropCacheKey.forTable(INSTANCE_ID, tIdA);
+    var created = zrw.putPersistentData(propCacheKey.getPath(),
+        VersionedPropCodec.getDefault().toBytes(vProps), ZooUtil.NodeExistsPolicy.FAIL);
+
+    assertTrue(created, "expected properties to be created");
+
+    ReadyMonitor readyMonitor = new ReadyMonitor("test", zooKeeper.getSessionTimeout());
+
+    PropStoreWatcher propStoreWatcher = new PropStoreWatcher(readyMonitor);
+
+    MetricsUtil.initializeProducers(cacheMetrics);
+
+    ZooPropLoader propLoader =
+        new ZooPropLoader(zrw, VersionedPropCodec.getDefault(), propStoreWatcher, cacheMetrics);
+    PropCacheCaffeineImpl cache =
+        new PropCacheCaffeineImpl.Builder(propLoader, cacheMetrics).build();
+
+    VersionedProperties readProps = cache.get(propCacheKey);
+
+    if (readProps == null) {
+      fail("Received null for versioned properties");
+    } else {
+      log.info("Props read from cache: {}", readProps.print(true));
+    }
+
+  }
+
+  // TODO - remove - this is not testing but was used for development?

Review Comment:
   Looks like this was left in and should be removed?



##########
test/src/main/java/org/apache/accumulo/test/conf/PropStoreConfigTest.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test.conf;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.Map;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Tag(MINI_CLUSTER_ONLY)
+@Tag(SUNNY_DAY)
+public class PropStoreConfigTest extends AccumuloClusterHarness {

Review Comment:
   Mini tests should be integration tests, ITs, and have a test name corresponding to `*IT`. Names with the pattern `*Test` are for unit tests, which should be in `src/test/java` and be in the module where the class they are testing is located. Unit tests in the test module are for unit testing the integration testing code itself.
   
   maven-surefire-plugin won't even look in src/main/java for test classes, so this test shouldn't even get executed automatically during a build because it won't be found.
   
   It looks like there are several tests with this naming/discovery issue in the test/ module in this PR.



##########
server/base/src/test/java/org/apache/accumulo/server/MockServerContext.java:
##########
@@ -54,4 +57,17 @@ public static ServerContext getWithZK(InstanceId instanceID, String zk, int zkTi
     return sc;
   }
 
+  public static ServerContext getMockContextWithPropStore(final InstanceId instanceID,
+      ZooReaderWriter zrw, PropStore propStore) {
+    try {
+      ServerContext sc = createMock(ServerContext.class);
+      expect(sc.getInstanceID()).andReturn(instanceID).anyTimes();
+      expect(sc.getZooReaderWriter()).andReturn(zrw).anyTimes();
+      expect(sc.getZooKeeperRoot()).andReturn("/accumulo/" + instanceID).anyTimes();
+      expect(sc.getPropStore()).andReturn(propStore).anyTimes();
+      return sc;
+    } catch (NullPointerException ex) {
+      throw new IllegalStateException("Failed to create mock test context", ex);
+    }

Review Comment:
   It doesn't really seem necessary to capture one RTE type and throw a different RTE, for a test utility. Any RTE is going to be the same problem, and the stack trace will identify the cause. You can just let the original RTE fall through.



##########
test/src/main/java/org/apache/accumulo/test/conf/util/TransformTokenTest.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test.conf.util;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.File;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStoreException;
+import org.apache.accumulo.server.conf.store.impl.PropStoreWatcher;
+import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
+import org.apache.accumulo.server.conf.util.TransformToken;
+import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZKUtil;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+@Tag(ZOOKEEPER_TESTING_SERVER)
+public class TransformTokenTest {
+
+  @TempDir
+  private static File tempDir;
+
+  private static ZooKeeperTestingServer testZk = null;
+  private static ZooKeeper zooKeeper;
+  private static ZooReaderWriter zrw;
+  private InstanceId instanceId = null;
+
+  @BeforeAll
+  public static void setupZk() {
+
+    // using default zookeeper port - we don't have a full configuration
+    testZk = new ZooKeeperTestingServer(tempDir);
+    zooKeeper = testZk.getZooKeeper();
+    zrw = testZk.getZooReaderWriter();
+  }
+
+  @AfterAll
+  public static void shutdownZK() throws Exception {
+    testZk.close();
+  }
+
+  @BeforeEach
+  public void testSetup() {
+    instanceId = InstanceId.of(UUID.randomUUID());
+  }
+
+  @AfterEach
+  public void cleanupZnodes() {
+    try {
+      ZKUtil.deleteRecursive(zooKeeper, Constants.ZROOT);
+    } catch (KeeperException | InterruptedException ex) {
+      throw new IllegalStateException("Failed to clean-up test zooKeeper nodes.", ex);
+    }
+  }
+
+  @Test
+  public void tokenGoPathTest() throws Exception {
+
+    List<LegacyPropData.PropNode> nodes = LegacyPropData.getData(instanceId);
+    for (LegacyPropData.PropNode node : nodes) {
+      zrw.putPersistentData(node.getPath(), node.getData(), ZooUtil.NodeExistsPolicy.SKIP);
+    }
+
+    ZooPropStore propStore = new ZooPropStore.Builder(instanceId, zrw, 30_000).build();
+
+    ServerContext context = createMock(ServerContext.class);
+    expect(context.getInstanceID()).andReturn(instanceId).anyTimes();
+    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
+    expect(context.getPropStore()).andReturn(propStore).anyTimes();
+
+    PropStoreWatcher watcher = createMock(PropStoreWatcher.class);
+
+    replay(context, watcher);
+
+    var sysPropKey = PropCacheKey.forSystem(instanceId);
+
+    TransformToken token = TransformToken.createToken(sysPropKey, zrw);
+
+    assertTrue(token.haveToken());
+    token.releaseToken();
+    assertFalse(token.haveToken());
+
+    // relock by getting a new lock
+    TransformToken lock2 = TransformToken.createToken(sysPropKey, zrw);
+    assertTrue(lock2.haveToken());
+
+    // fail with a current lock node present
+    TransformToken lock3 = TransformToken.createToken(sysPropKey, zrw);
+    assertFalse(lock3.haveToken());
+
+  }
+
+  @Test
+  public void failOnInvalidLockTest() throws Exception {
+
+    List<LegacyPropData.PropNode> nodes = LegacyPropData.getData(instanceId);
+    for (LegacyPropData.PropNode node : nodes) {
+      zrw.putPersistentData(node.getPath(), node.getData(), ZooUtil.NodeExistsPolicy.SKIP);
+    }
+
+    ZooPropStore propStore = new ZooPropStore.Builder(instanceId, zrw, 30_000).build();
+
+    ServerContext context = createMock(ServerContext.class);
+    expect(context.getInstanceID()).andReturn(instanceId).anyTimes();
+    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
+    expect(context.getPropStore()).andReturn(propStore).anyTimes();
+
+    PropStoreWatcher watcher = createMock(PropStoreWatcher.class);
+
+    replay(context, watcher);
+
+    var sysPropKey = PropCacheKey.forSystem(instanceId);
+    var tokenPath = sysPropKey.getBasePath() + TransformToken.TRANSFORM_TOKEN;
+
+    TransformToken lock = TransformToken.createToken(sysPropKey, zrw);
+
+    // force change in lock
+    assertTrue(lock.haveToken());
+    zrw.mutateExisting(tokenPath, v -> UUID.randomUUID().toString().getBytes(UTF_8));
+    assertThrows(PropStoreException.class, lock::releaseToken,
+        "Expected unlock to fail on different UUID");
+
+    // clean-up and get new lock
+    zrw.delete(tokenPath);
+    TransformToken lock3 = TransformToken.createToken(sysPropKey, zrw);
+    assertTrue(lock3.haveToken());
+    zrw.delete(tokenPath);
+    assertThrows(PropStoreException.class, lock::releaseToken,
+        "Expected unlock to fail when no lock present");
+
+  }

Review Comment:
   ```suggestion
       verify(context, watcher);
     }
   ```



##########
server/manager/src/main/java/org/apache/accumulo/manager/Manager.java:
##########
@@ -260,6 +261,18 @@ synchronized void setManagerState(ManagerState newState) {
       ThreadPools.watchNonCriticalScheduledTask(future);
     }
 
+    if (oldState != newState && (newState == ManagerState.HAVE_LOCK)) {

Review Comment:
   This block of code and the conditional here probably warrants a comment. I have no idea what this conditional is checking.



##########
server/manager/src/main/java/org/apache/accumulo/manager/Manager.java:
##########
@@ -260,6 +261,18 @@ synchronized void setManagerState(ManagerState newState) {
       ThreadPools.watchNonCriticalScheduledTask(future);
     }
 
+    if (oldState != newState && (newState == ManagerState.HAVE_LOCK)) {
+      try {
+        log.info("Starting property conversion");
+        var context = getContext();
+        ConfigPropertyUpgrader configUpgrader = new ConfigPropertyUpgrader();
+        configUpgrader.doUpgrade(context.getInstanceID(), context.getZooReaderWriter());

Review Comment:
   Is it only the manager that does this upgrade? What about other servers?



##########
server/base/src/test/java/org/apache/accumulo/server/conf/codec/VersionedPropertiesTest.java:
##########
@@ -157,7 +157,6 @@ public void getInitialDataVersion() {
     assertEquals(0, vProps.getDataVersion());
 
     // the initial version for write should be 0

Review Comment:
   Comment no longer applies
   ```suggestion
   ```



##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImport.java:
##########
@@ -183,8 +183,14 @@ private void checkForMerge(final long tid, final Manager manager) throws Excepti
     VolumeManager fs = manager.getVolumeManager();
     final Path bulkDir = new Path(bulkInfo.sourceDir);
 
-    int maxTablets = Integer.parseInt(manager.getContext().getTableConfiguration(bulkInfo.tableId)
-        .get(Property.TABLE_BULK_MAX_TABLETS));
+    String value = manager.getContext().getTableConfiguration(bulkInfo.tableId)
+        .get(Property.TABLE_BULK_MAX_TABLETS);
+    if (value == null) {
+      throw new IllegalStateException("The property: " + Property.TABLE_BULK_MAX_TABLETS.getKey()

Review Comment:
   This should be calling `.getCount()`, which returns an `int`. We shouldn't be parsing as an int here ourselves. This property is a `PropertyType.COUNT` type, and can be directly retrieved as such. That would address any spotbugs issue you saw.



##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportPopulateZookeeper.java:
##########
@@ -88,13 +88,15 @@ public Repo<Manager> call(long tid, Manager env) throws Exception {
     }
 
     VolumeManager volMan = env.getVolumeManager();
-    for (Entry<String,String> entry : getExportedProps(volMan).entrySet())
-      if (!TablePropUtil.setTableProperty(env.getContext(), tableInfo.tableId, entry.getKey(),
-          entry.getValue())) {
-        throw new AcceptableThriftTableOperationException(tableInfo.tableId.canonical(),
-            tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
-            "Invalid table property " + entry.getKey());
-      }
+
+    try {
+      TablePropUtil.factory().setProperties(env.getContext(), tableInfo.tableId,

Review Comment:
   Calling `TablePropUtil.factory()` and passing in a `ServerContext` is a hassle. The factory is just a static stateless singleton instance. So, now instead of having the methods be static utility methods, they are non-static methods in a static singleton. I don't see these as substantively different, but this made the entry point a bit longer and more unwieldy.
   
   I get that this was done to make them both satisfy the same interface. But, I think that might be a bit overkill, as these are very primitive helper utility methods and we didn't really need to have a common parent interface. They only existed in separate classes to make the naming easier, but they could have been four methods in the same util class.
   
   Since I don't think this change was strictly necessary, I'd be happy reverting to the old static utility methods to make the calling code that changed, go back to the way it was and make this diff a little smaller.
   
   If you don't want to revert these because you have plans to grow these utilities later, then I think removing the factory singleton and constructing them in the ServerContext, with an entry point there to have "effective" singletons inside the context, would be better. I can make this change if you wish, along with narrowing the scope of the generics so you can avoid the casting from `AbstractId<?>` to the more specific type. I've already made those changes locally while inspecting this.



##########
server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoaderTest.java:
##########
@@ -0,0 +1,675 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.conf.store.impl;
+
+import static org.apache.accumulo.core.conf.Property.GC_PORT;
+import static org.apache.accumulo.core.conf.Property.MANAGER_CLIENTPORT;
+import static org.apache.accumulo.core.conf.Property.TSERV_CLIENTPORT;
+import static org.apache.accumulo.core.conf.Property.TSERV_NATIVEMAP_ENABLED;
+import static org.apache.accumulo.core.conf.Property.TSERV_SCAN_MAX_OPENFILES;
+import static org.apache.accumulo.server.conf.store.impl.PropCacheCaffeineImpl.REFRESH_MIN;
+import static org.easymock.EasyMock.anyLong;
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.capture;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.eq;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.expectLastCall;
+import static org.easymock.EasyMock.newCapture;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+import java.time.Instant;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.TableId;
+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.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.easymock.Capture;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ZooPropLoaderTest {
+
+  private static final Logger log = LoggerFactory.getLogger(ZooPropLoaderTest.class);
+
+  private PropCacheCaffeineImplTest.TestTicker ticker;
+  private InstanceId instanceId;
+  private ServerContext context;
+  private PropCacheKey propCacheKey;
+  private VersionedPropCodec propCodec;
+
+  // mocks
+  private PropStoreMetrics cacheMetrics;
+  private PropStoreWatcher propStoreWatcher;
+  private ZooReaderWriter zrw;
+
+  private ZooPropLoader loader;
+
+  @BeforeEach
+  public void initCommonMocks() {
+    ticker = new PropCacheCaffeineImplTest.TestTicker();
+    instanceId = InstanceId.of(UUID.randomUUID());
+
+    propCacheKey = PropCacheKey.forSystem(instanceId);
+    propCodec = VersionedPropCodec.getDefault();
+
+    // mocks
+    context = createMock(ServerContext.class);
+    expect(context.getInstanceID()).andReturn(instanceId).anyTimes();
+
+    zrw = createMock(ZooReaderWriter.class);
+
+    cacheMetrics = createMock(PropStoreMetrics.class);
+
+    propStoreWatcher = createMock(PropStoreWatcher.class);
+
+    // loader used in tests
+    loader = new ZooPropLoader(zrw, propCodec, propStoreWatcher, cacheMetrics);
+
+  }
+
+  @AfterEach
+  public void verifyCommonMocks() {
+    verify(context, zrw, propStoreWatcher, cacheMetrics);
+  }
+
+  @Test
+  public void loadTest() throws Exception {
+
+    VersionedProperties defaultProps = new VersionedProperties();
+
+    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+        .andReturn(propCodec.toBytes(defaultProps)).anyTimes();
+
+    cacheMetrics.addLoadTime(anyLong());
+    expectLastCall().times(1);
+
+    replay(context, zrw, propStoreWatcher, cacheMetrics);
+
+    assertNotNull(loader.load(propCacheKey));
+  }
+
+  // from cache loader
+
+  /**
+   * Verify that first call loads from ZooKeeper, then second call returns from the cache.
+   *
+   * @throws Exception
+   *           any exception is a test failure.
+   */
+  @Test
+  public void loadAndCacheTest() throws Exception {
+
+    VersionedProperties defaultProps = new VersionedProperties();
+
+    expect(zrw.getStatus(propCacheKey.getPath())).andThrow(new KeeperException.NoNodeException() {})
+        .anyTimes();
+    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+        .andReturn(propCodec.toBytes(defaultProps)).once();
+
+    cacheMetrics.addLoadTime(anyLong());
+    expectLastCall().times(1);
+
+    replay(context, zrw, propStoreWatcher, cacheMetrics);
+
+    PropCacheCaffeineImpl cache =
+        new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).withTicker(ticker).build();
+
+    // load into cache
+    assertNotNull(cache.get(propCacheKey));
+
+    // read cached entry - load count should not change.
+    ticker.advance(1, TimeUnit.MINUTES);
+    assertNotNull(cache.get(propCacheKey));
+  }
+
+  // TODO - may be just an exception on Zk read.
+  @Test
+  public void getExpireTimeoutTest() {
+    replay(context, zrw, propStoreWatcher, cacheMetrics);
+    // TODO implement test
+    // fail("Implement test");
+  }
+
+  /**
+   * Verify that an exception on load result in null value and that the exception does not escape
+   * the load call.
+   * <p>
+   * throws Exception any exception is a test failure.
+   */
+  @Test
+  public void loadFailTest() throws Exception {
+
+    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+        .andThrow(new KeeperException.NoNodeException("force no node exception")).once();
+
+    propStoreWatcher.signalZkChangeEvent(eq(propCacheKey));
+    expectLastCall();
+
+    cacheMetrics.incrZkError();
+    expectLastCall().once();
+
+    replay(context, zrw, propStoreWatcher, cacheMetrics);
+
+    PropCacheCaffeineImpl cache =
+        new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).withTicker(ticker).build();
+
+    assertNull(cache.get(propCacheKey));
+
+    log.info("Metrics: {}", cacheMetrics);
+  }
+
+  /**
+   * Validate that cache expiration functions as expected.
+   * <p>
+   * throws Exception any exception is a test failure.
+   */
+  @Test
+  public void expireTest() throws Exception {
+
+    VersionedProperties defaultProps = new VersionedProperties();
+
+    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+        .andReturn(propCodec.toBytes(defaultProps)).times(2);
+
+    cacheMetrics.addLoadTime(anyLong());
+    expectLastCall().times(2);
+
+    cacheMetrics.incrEviction();
+    expectLastCall().once();
+
+    replay(context, zrw, propStoreWatcher, cacheMetrics);
+
+    PropCacheCaffeineImpl cache =
+        new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).withTicker(ticker).build();
+
+    // load cache
+    assertNotNull(cache.get(propCacheKey));
+
+    ticker.advance(70, TimeUnit.MINUTES);
+    cache.cleanUp();
+
+    assertNotNull(cache.get(propCacheKey));
+
+  }
+
+  /**
+   * Test that a ZooKeeper exception on an async reload task is correctly handed and that the value
+   * is removed from the cache when the refresh fails and the next get.
+   *
+   * @throws Exception
+   *           if a test error occurs.
+   */
+  @Test
+  public void reloadExceptionTest() throws Exception {
+
+    final VersionedProperties defaultProps = new VersionedProperties();
+
+    expect(zrw.getData(eq(propCacheKey.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.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+        .andThrow(new KeeperException.NoNodeException("forced no node")).anyTimes();
+
+    propStoreWatcher.signalZkChangeEvent(eq(propCacheKey));
+    expectLastCall().anyTimes();
+
+    propStoreWatcher.signalZkChangeEvent(eq(PropCacheKey.forSystem(instanceId)));
+    expectLastCall().anyTimes();
+
+    propStoreWatcher.signalCacheChangeEvent(eq(PropCacheKey.forSystem(instanceId)));
+    expectLastCall().anyTimes();
+
+    cacheMetrics.addLoadTime(anyLong());
+    expectLastCall().times(1);
+    cacheMetrics.incrRefresh();
+    expectLastCall().times(1);
+    cacheMetrics.incrRefreshLoad();
+    expectLastCall().times(1);
+    cacheMetrics.incrZkError();
+    expectLastCall().times(2);
+
+    replay(context, zrw, propStoreWatcher, cacheMetrics);
+
+    PropCacheCaffeineImpl cache =
+        new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).withTicker(ticker).build();
+
+    // prime cache
+    assertNotNull(cache.get(propCacheKey));
+
+    ticker.advance(5, TimeUnit.MINUTES);
+    cache.cleanUp();
+
+    // read cached value
+    assertNotNull(cache.get(propCacheKey));
+
+    // advance so refresh called.
+    ticker.advance(20, TimeUnit.MINUTES);
+    cache.cleanUp();
+
+    assertNotNull(cache.get(propCacheKey));
+
+    try {
+      // yield so async thread completes.
+      Thread.sleep(250);
+    } catch (InterruptedException ex) {
+      // empty
+    }
+
+    assertNull(cache.get(propCacheKey));
+  }
+
+  @Test
+  public void getWithoutCachingTest() {
+
+    replay(context, zrw, propStoreWatcher, cacheMetrics);
+
+    PropCacheCaffeineImpl cache =
+        new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).withTicker(ticker).build();
+
+    assertNull(cache.getWithoutCaching(propCacheKey));
+
+  }
+
+  @Test
+  public void removeTest() throws Exception {
+    final PropCacheKey sysPropKey = PropCacheKey.forSystem(instanceId);
+    final PropCacheKey tablePropKey = PropCacheKey.forTable(instanceId, TableId.of("t1"));
+
+    VersionedProperties defaultProps = new VersionedProperties();
+
+    expect(zrw.getData(eq(sysPropKey.getPath()), anyObject(), anyObject()))
+        .andReturn(propCodec.toBytes(defaultProps)).once();
+    expect(zrw.getData(eq(tablePropKey.getPath()), anyObject(), anyObject()))
+        .andReturn(propCodec.toBytes(defaultProps)).once();
+
+    cacheMetrics.addLoadTime(anyLong());
+    expectLastCall().times(2);
+
+    replay(context, zrw, propStoreWatcher, cacheMetrics);
+
+    PropCacheCaffeineImpl cache =
+        new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).withTicker(ticker).build();
+
+    // load into cache
+    assertNotNull(cache.get(sysPropKey));
+    assertNotNull(cache.get(tablePropKey));
+
+    cache.remove(tablePropKey);
+    cache.cleanUp();
+
+    // verify retrieved from cache without loading.
+    assertNotNull(cache.getWithoutCaching(sysPropKey));
+    assertNull(cache.getWithoutCaching(tablePropKey));
+  }
+
+  @Test
+  public void removeAllTest() throws Exception {
+    final PropCacheKey sysPropKey = PropCacheKey.forSystem(instanceId);
+    final PropCacheKey tablePropKey = PropCacheKey.forTable(instanceId, TableId.of("t1"));
+
+    VersionedProperties defaultProps = new VersionedProperties();
+
+    expect(zrw.getData(eq(sysPropKey.getPath()), anyObject(), anyObject()))
+        .andReturn(propCodec.toBytes(defaultProps)).once();
+    expect(zrw.getData(eq(tablePropKey.getPath()), anyObject(), anyObject()))
+        .andReturn(propCodec.toBytes(defaultProps)).once();
+
+    cacheMetrics.addLoadTime(anyLong());
+    expectLastCall().times(2);
+
+    replay(context, zrw, propStoreWatcher, cacheMetrics);
+
+    PropCacheCaffeineImpl cache =
+        new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).withTicker(ticker).build();
+
+    // load into cache
+    assertNotNull(cache.get(sysPropKey));
+    assertNotNull(cache.get(tablePropKey));
+
+    cache.removeAll();
+    cache.cleanUp();
+
+    // verify retrieved from cache without loading.
+    assertNull(cache.getWithoutCaching(sysPropKey));
+    assertNull(cache.getWithoutCaching(tablePropKey));
+  }
+
+  @Test
+  public void getWithoutCachingNotPresentTest() {
+    replay(context, zrw, propStoreWatcher, cacheMetrics);
+
+    PropCacheCaffeineImpl cache =
+        new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).withTicker(ticker).build();
+
+    // load into cache
+    assertNull(cache.getWithoutCaching(propCacheKey));
+  }
+
+  @Test
+  public void refreshTest() throws Exception {
+
+    VersionedProperties defaultProps = new VersionedProperties();
+
+    // first call loads cache
+    Capture<Stat> stat = newCapture();
+    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
+      Stat s = stat.getValue();
+      s.setCtime(System.currentTimeMillis());
+      s.setMtime(System.currentTimeMillis());
+      s.setCzxid(1234);
+      s.setVersion(0);
+      stat.setValue(s);
+      return propCodec.toBytes(defaultProps);
+    }).times(1);
+
+    Stat expectedStat = new Stat();
+    expectedStat.setVersion(0);
+    expect(zrw.getStatus(propCacheKey.getPath())).andReturn(expectedStat).times(2);
+
+    cacheMetrics.addLoadTime(anyLong());
+    expectLastCall().times(1);
+    cacheMetrics.incrRefresh();
+    expectLastCall().times(2);
+
+    replay(context, zrw, propStoreWatcher, cacheMetrics);
+
+    PropCacheCaffeineImpl cache =
+        new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).withTicker(ticker).build();
+
+    // load cache
+    log.info("received: {}", cache.get(propCacheKey));
+
+    ticker.advance(REFRESH_MIN + 1, TimeUnit.MINUTES);
+
+    assertNotNull(cache.get(propCacheKey));
+
+    ticker.advance(REFRESH_MIN / 2, TimeUnit.MINUTES);
+
+    assertNotNull(cache.get(propCacheKey));
+
+    Thread.sleep(100);
+
+    ticker.advance(REFRESH_MIN + 1, TimeUnit.MINUTES);
+
+    assertNotNull(cache.get(propCacheKey));
+
+    Thread.sleep(100);
+
+    ticker.advance(1, TimeUnit.MINUTES);
+
+    assertNotNull(cache.get(propCacheKey));
+
+  }
+
+  /**
+   * Test that when the refreshAfterWrite period expires that the data version is checked against
+   * stored value - and on mismatch, rereads the values from ZooKeeper.
+   */
+  @Test
+  public void refreshDifferentVersionTest() throws Exception {
+
+    final int initialVersion = 123;
+    Capture<PropStoreWatcher> propStoreWatcherCapture = newCapture();
+
+    Capture<Stat> stat = newCapture();
+
+    expect(zrw.getData(eq(propCacheKey.getPath()), capture(propStoreWatcherCapture), capture(stat)))
+        .andAnswer(() -> {
+          Stat s = stat.getValue();
+          s.setCtime(System.currentTimeMillis());
+          s.setMtime(System.currentTimeMillis());
+          s.setVersion(initialVersion + 1);
+          stat.setValue(s);
+          return propCodec.toBytes(new VersionedProperties(initialVersion + 1, Instant.now(),
+              Map.of(Property.TABLE_SPLIT_THRESHOLD.getKey(), "7G")));
+        }).once();
+
+    // 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();
+
+    Capture<Stat> stat3 = newCapture();
+
+    expect(
+        zrw.getData(eq(propCacheKey.getPath()), capture(propStoreWatcherCapture), capture(stat3)))
+            .andAnswer(() -> {
+              Stat s = stat3.getValue();
+              s.setCtime(System.currentTimeMillis());
+              s.setMtime(System.currentTimeMillis());
+              s.setVersion(initialVersion + 4);
+              stat3.setValue(s);
+              return propCodec.toBytes(new VersionedProperties(initialVersion + 3, Instant.now(),
+                  Map.of(Property.TABLE_SPLIT_THRESHOLD.getKey(), "12G")));
+            }).once();
+
+    propStoreWatcher.signalCacheChangeEvent(eq(propCacheKey));
+    expectLastCall();
+
+    cacheMetrics.addLoadTime(anyLong());
+    expectLastCall().times(2);
+
+    cacheMetrics.incrRefresh();
+    expectLastCall().times(1);
+
+    cacheMetrics.incrRefreshLoad();
+    expectLastCall().times(1);
+
+    replay(context, zrw, propStoreWatcher, cacheMetrics);
+
+    PropCacheCaffeineImpl cache =
+        new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).withTicker(ticker).build();
+
+    // prime cache
+    assertNotNull(cache.get(propCacheKey));
+
+    ticker.advance(REFRESH_MIN + 1, TimeUnit.MINUTES);
+    // first call after refresh return original and schedules update
+    var originalProps = cache.get(propCacheKey);
+    assertNotNull(originalProps);
+    assertEquals("7G", originalProps.getProperties().get(Property.TABLE_SPLIT_THRESHOLD.getKey()));
+
+    // allow refresh thread to run
+    Thread.sleep(50);
+
+    // refresh should have loaded updated value;
+    var updatedProps = cache.get(propCacheKey);
+    log.info("Updated props: {}", updatedProps == null ? "null" : updatedProps.print(true));
+
+    assertNotNull(updatedProps);
+    Thread.sleep(250);
+
+    assertEquals("12G", updatedProps.getProperties().get(Property.TABLE_SPLIT_THRESHOLD.getKey()));
+  }
+
+  /**
+   * Test that when the refreshAfterWrite period expires that the data version is checked against
+   * stored value - and on match, returns the current value without rereading the values from
+   * ZooKeeper.
+   *
+   * @throws Exception
+   *           any exception is a test failure
+   */
+  @Test
+  public void refreshSameVersionTest() throws Exception {
+
+    final int expectedVersion = 123;
+
+    VersionedProperties mockProps = createMock(VersionedProperties.class);
+    expect(mockProps.getTimestamp()).andReturn(Instant.now()).once();
+    expect(mockProps.getProperties()).andReturn(Map.of());
+
+    Capture<Stat> stat = newCapture();
+
+    // first call loads cache
+    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
+      Stat s = stat.getValue();
+      s.setCtime(System.currentTimeMillis());
+      s.setMtime(System.currentTimeMillis());
+      s.setVersion(expectedVersion);
+      stat.setValue(s);
+      return propCodec.toBytes(mockProps);
+    }).times(1);
+
+    Stat stat2 = new Stat();
+    stat2.setCtime(System.currentTimeMillis());
+    stat2.setMtime(System.currentTimeMillis());
+    stat2.setVersion(expectedVersion);
+
+    expect(zrw.getStatus(propCacheKey.getPath())).andReturn(stat2).once();
+
+    cacheMetrics.addLoadTime(anyLong());
+    expectLastCall().times(1);
+    cacheMetrics.incrRefresh();
+    expectLastCall().times(1);
+
+    replay(context, zrw, propStoreWatcher, cacheMetrics, mockProps);
+
+    PropCacheCaffeineImpl cache =
+        new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).withTicker(ticker).build();
+
+    // prime cache
+    cache.get(propCacheKey);
+
+    ticker.advance(30, TimeUnit.MINUTES);
+    cache.cleanUp();
+
+    VersionedProperties vPropsRead = cache.get(propCacheKey);
+
+    assertNotNull(vPropsRead);
+
+    try {
+      Thread.sleep(250);
+      cache.get(propCacheKey);
+    } catch (InterruptedException ex) {
+      // empty
+    }

Review Comment:
   Just `throw Exception` in the test method. There's no reason to capture InterruptedExceptions in test code. If the test is interrupted, we want it to fall through and quit. Otherwise, the user experience is that tests are hanging even after we've killed them.



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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org