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/02/24 14:32:35 UTC

[accumulo] branch main updated: Some cleanup of Zoo-related code (#2522)

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 a9775e7  Some cleanup of Zoo-related code (#2522)
a9775e7 is described below

commit a9775e70d6161ecd082f49715ed3f443c48a44f7
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Thu Feb 24 09:32:25 2022 -0500

    Some cleanup of Zoo-related code (#2522)
    
    Remove three unneeded ZooCache constructors - these were simple
    variations of the main one, with very little added value, and removing
    them made some method calls slightly less convenient, but enabled a lot
    of code complexity reduction.
    
    Remove unneeded ZooConfigurationFactory - it was unnecessarily complex,
    written to handle caching of ZooConfiguration objects for multiple
    InstanceIds; however, it was only ever used by
    ServerConfigurationFactory, which only ever called it once, with the
    first instanceId it found. This is sufficient. If further work is needed
    to ensure ServerConfigurationFactory only uses a single InstanceId, that
    work should be done in that class. This change preserves the existing
    behavior, but much more simplified. An additional benefit to removing
    this is that it can no longer be confused with the similarly named
    ZooCacheFactory.
    
    Remove unneeded "hack" to use a watcher in ZooCacheFactory that supplied
    an unused Watcher to force the ZooCacheFactory to return a new instance.
    Now, it explicitly returns a new instance, with a method that says as
    much in its name and javadoc. The old method wasn't used for anything
    else.
    
    Remove unnecessary call in ReplicationIT to ZooCacheFactory, instead
    using the ZooCache that is already available in the ClientContext in the
    test.
---
 .../apache/accumulo/fate/zookeeper/ZooCache.java   | 30 ---------
 .../accumulo/fate/zookeeper/ZooCacheFactory.java   | 25 ++------
 .../fate/zookeeper/ZooCacheFactoryTest.java        | 20 ++----
 .../org/apache/accumulo/server/ServerContext.java  |  6 +-
 .../server/conf/NamespaceConfiguration.java        |  8 +--
 .../server/conf/ServerConfigurationFactory.java    |  9 ++-
 .../server/conf/ZooConfigurationFactory.java       | 74 ----------------------
 .../apache/accumulo/server/util/ListInstances.java |  2 +-
 .../conf/ServerConfigurationFactoryTest.java       | 11 +---
 .../server/conf/ZooConfigurationFactoryTest.java   | 71 ---------------------
 .../DistributedWorkQueueWorkAssigner.java          |  2 +-
 .../accumulo/test/BadDeleteMarkersCreatedIT.java   |  8 +--
 .../accumulo/test/functional/CacheTestReader.java  |  3 +-
 .../functional/TabletStateChangeIteratorIT.java    | 15 ++---
 .../accumulo/test/replication/ReplicationIT.java   |  7 +-
 15 files changed, 45 insertions(+), 246 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java b/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
index b5df0c0..ace2b02 100644
--- a/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
+++ b/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
@@ -201,36 +201,6 @@ public class ZooCache {
   }
 
   /**
-   * Creates a new cache.
-   *
-   * @param zooKeepers
-   *          comma-separated list of ZooKeeper host[:port]s
-   * @param sessionTimeout
-   *          ZooKeeper session timeout
-   */
-  public ZooCache(String zooKeepers, int sessionTimeout) {
-    this(zooKeepers, sessionTimeout, null);
-  }
-
-  /**
-   * Creates a new cache. The given watcher is called whenever a watched node changes.
-   *
-   * @param zooKeepers
-   *          comma-separated list of ZooKeeper host[:port]s
-   * @param sessionTimeout
-   *          ZooKeeper session timeout
-   * @param watcher
-   *          watcher object
-   */
-  public ZooCache(String zooKeepers, int sessionTimeout, Watcher watcher) {
-    this(new ZooReader(zooKeepers, sessionTimeout), watcher);
-  }
-
-  public ZooCache(ZooReaderWriter reader) {
-    this(reader, null);
-  }
-
-  /**
    * Creates a new cache. The given watcher is called whenever a watched node changes.
    *
    * @param reader
diff --git a/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java b/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java
index c4d92a4..76bb3f0 100644
--- a/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java
+++ b/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java
@@ -23,7 +23,6 @@ import java.util.Map;
 
 import org.apache.accumulo.core.singletons.SingletonManager;
 import org.apache.accumulo.core.singletons.SingletonService;
-import org.apache.zookeeper.Watcher;
 
 /**
  * A factory for {@link ZooCache} instances.
@@ -33,6 +32,8 @@ public class ZooCacheFactory {
   private static Map<String,ZooCache> instances = new HashMap<>();
   private static boolean enabled = true;
 
+  public ZooCacheFactory() {}
+
   private static boolean isEnabled() {
     synchronized (instances) {
       return enabled;
@@ -96,33 +97,21 @@ public class ZooCacheFactory {
         throw new IllegalStateException("The Accumulo singleton for zookeeper caching is "
             + "disabled. This is likely caused by all AccumuloClients being closed");
       }
-      ZooCache zc = instances.get(key);
-      if (zc == null) {
-        zc = new ZooCache(zooKeepers, sessionTimeout);
-        instances.put(key, zc);
-      }
-      return zc;
+      return instances.computeIfAbsent(key, k -> getNewZooCache(zooKeepers, sessionTimeout));
     }
   }
 
   /**
-   * Gets a watched {@link ZooCache}. If the watcher is null, then the same (unwatched) object may
-   * be returned for multiple calls with the same remaining arguments.
+   * Always return a new {@link ZooCache}.
    *
    * @param zooKeepers
    *          comma-separated list of ZooKeeper host[:port]s
    * @param sessionTimeout
    *          session timeout
-   * @param watcher
-   *          watcher (optional)
-   * @return cache object
+   * @return a new instance
    */
-  public ZooCache getZooCache(String zooKeepers, int sessionTimeout, Watcher watcher) {
-    if (watcher == null) {
-      // reuse
-      return getZooCache(zooKeepers, sessionTimeout);
-    }
-    return new ZooCache(zooKeepers, sessionTimeout, watcher);
+  public ZooCache getNewZooCache(String zooKeepers, int sessionTimeout) {
+    return new ZooCache(new ZooReader(zooKeepers, sessionTimeout), null);
   }
 
   /**
diff --git a/core/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java b/core/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java
index f9a0d62..fd44579 100644
--- a/core/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java
+++ b/core/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java
@@ -18,12 +18,10 @@
  */
 package org.apache.accumulo.fate.zookeeper;
 
-import static org.easymock.EasyMock.createMock;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNotSame;
 import static org.junit.jupiter.api.Assertions.assertSame;
 
-import org.apache.zookeeper.Watcher;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
@@ -61,22 +59,16 @@ public class ZooCacheFactoryTest {
   }
 
   @Test
-  public void testGetZooCacheWatcher() {
+  public void testGetNewZooCache() {
     String zks1 = "zk1";
     int timeout1 = 1000;
-    Watcher watcher = createMock(Watcher.class);
-    ZooCache zc1 = zcf.getZooCache(zks1, timeout1, watcher);
-    assertNotNull(zc1);
-  }
-
-  @Test
-  public void testGetZooCacheWatcher_Null() {
-    String zks1 = "zk1";
-    int timeout1 = 1000;
-    ZooCache zc1 = zcf.getZooCache(zks1, timeout1, null);
+    ZooCache zc1 = zcf.getNewZooCache(zks1, timeout1);
     assertNotNull(zc1);
     ZooCache zc1a = zcf.getZooCache(zks1, timeout1);
-    assertSame(zc1, zc1a);
+    assertNotSame(zc1, zc1a);
+    ZooCache zc1b = zcf.getNewZooCache(zks1, timeout1);
+    assertNotSame(zc1, zc1b);
+    assertNotSame(zc1a, zc1b);
   }
 
   @Test
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
index 406310f..9ab212e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
@@ -56,6 +56,7 @@ import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooReader;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.server.conf.NamespaceConfiguration;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
@@ -155,7 +156,10 @@ public class ServerContext extends ClientContext {
   @Override
   public AccumuloConfiguration getConfiguration() {
     if (systemConfig == null) {
-      ZooCache propCache = new ZooCache(getZooKeepers(), getZooKeepersSessionTimeOut());
+      // system configuration uses its own instance of ZooCache
+      // this could be useful to keep its update counter independent
+      ZooCache propCache =
+          new ZooCache(new ZooReader(getZooKeepers(), getZooKeepersSessionTimeOut()), null);
       systemConfig = new ZooConfiguration(this, propCache, getSiteConfiguration());
     }
     return systemConfig;
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 9e6b404..b81cf91 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
@@ -69,12 +69,8 @@ public class NamespaceConfiguration extends AccumuloConfiguration {
   private ZooCache getZooCache() {
     synchronized (propCaches) {
       PropCacheKey key = new PropCacheKey(context.getInstanceID(), namespaceId.canonical());
-      ZooCache propCache = propCaches.get(key);
-      if (propCache == null) {
-        propCache = zcf.getZooCache(context.getZooKeepers(), context.getZooKeepersSessionTimeOut());
-        propCaches.put(key, propCache);
-      }
-      return propCache;
+      return propCaches.computeIfAbsent(key,
+          k -> zcf.getZooCache(context.getZooKeepers(), context.getZooKeepersSessionTimeOut()));
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java
index f562476..7b32e74 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.ServerContext;
 
@@ -105,8 +106,12 @@ public class ServerConfigurationFactory extends ServerConfiguration {
   @Override
   public synchronized AccumuloConfiguration getSystemConfiguration() {
     if (systemConfig == null) {
-      systemConfig =
-          new ZooConfigurationFactory().getInstance(context, zcf, getSiteConfiguration());
+      // Force the creation of a new ZooCache instead of using a shared one.
+      // This is done so that the ZooCache will update less often, causing the
+      // configuration update count to increment more slowly.
+      ZooCache propCache =
+          zcf.getNewZooCache(context.getZooKeepers(), context.getZooKeepersSessionTimeOut());
+      systemConfig = new ZooConfiguration(context, propCache, getSiteConfiguration());
     }
     return systemConfig;
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfigurationFactory.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfigurationFactory.java
deleted file mode 100644
index e3c5e40..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfigurationFactory.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
-import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-
-/**
- * A factory for {@link ZooConfiguration} objects.
- */
-class ZooConfigurationFactory {
-  private static final Map<InstanceId,ZooConfiguration> instances = new HashMap<>();
-
-  /**
-   * Gets a configuration object for the given instance with the given parent. Repeated calls will
-   * return the same object.
-   *
-   * @param context
-   *          ServerContext; if null, instance is determined from HDFS
-   * @param zcf
-   *          {@link ZooCacheFactory} for building {@link ZooCache} to contact ZooKeeper (required)
-   * @param parent
-   *          parent configuration (required)
-   * @return configuration
-   */
-  ZooConfiguration getInstance(ServerContext context, ZooCacheFactory zcf,
-      AccumuloConfiguration parent) {
-    ZooConfiguration config;
-    synchronized (instances) {
-      config = instances.get(context.getInstanceID());
-      if (config == null) {
-        ZooCache propCache;
-
-        // The purpose of this watcher is a hack. It forces the creation on a new zoocache instead
-        // of using a shared one. This was done so that the zoocache
-        // would update less, causing the configuration update count to changes less.
-        Watcher watcher = new Watcher() {
-          @Override
-          public void process(WatchedEvent arg0) {}
-        };
-        propCache = zcf.getZooCache(context.getZooKeepers(), context.getZooKeepersSessionTimeOut(),
-            watcher);
-        config = new ZooConfiguration(context, propCache, parent);
-        instances.put(context.getInstanceID(), config);
-      }
-    }
-    return config;
-  }
-
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java b/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
index 014eb8f..5a2c847 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
@@ -84,7 +84,7 @@ public class ListInstances {
 
     System.out.println("INFO : Using ZooKeepers " + keepers);
     ZooReader rdr = new ZooReader(keepers, ZOOKEEPER_TIMER_MILLIS);
-    ZooCache cache = new ZooCache(keepers, ZOOKEEPER_TIMER_MILLIS);
+    ZooCache cache = new ZooCache(rdr, null);
 
     TreeMap<String,InstanceId> instanceNames = getInstanceNames(rdr, printErrors);
 
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/ServerConfigurationFactoryTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/ServerConfigurationFactoryTest.java
index bad4d07..6d35ccc 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/ServerConfigurationFactoryTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/ServerConfigurationFactoryTest.java
@@ -24,7 +24,6 @@ import static org.easymock.EasyMock.createMock;
 import static org.easymock.EasyMock.endsWith;
 import static org.easymock.EasyMock.eq;
 import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
 import static org.easymock.EasyMock.replay;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -39,7 +38,6 @@ import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.MockServerContext;
 import org.apache.accumulo.server.ServerContext;
-import org.easymock.EasyMock;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -59,14 +57,11 @@ public class ServerConfigurationFactoryTest {
   public static void setUpClass() {
     zcf = createMock(ZooCacheFactory.class);
     zc = createMock(ZooCache.class);
-    expect(zcf.getZooCache(eq(ZK_HOST), eq(ZK_TIMEOUT), EasyMock.anyObject())).andReturn(zc);
-    expectLastCall().anyTimes();
-    expect(zcf.getZooCache(ZK_HOST, ZK_TIMEOUT)).andReturn(zc);
-    expectLastCall().anyTimes();
+    expect(zcf.getNewZooCache(eq(ZK_HOST), eq(ZK_TIMEOUT))).andReturn(zc).anyTimes();
+    expect(zcf.getZooCache(ZK_HOST, ZK_TIMEOUT)).andReturn(zc).anyTimes();
     replay(zcf);
 
-    expect(zc.getChildren(anyObject(String.class))).andReturn(null);
-    expectLastCall().anyTimes();
+    expect(zc.getChildren(anyObject(String.class))).andReturn(null).anyTimes();
     // CheckServerConfig looks at timeout
     expect(zc.get(endsWith("timeout"))).andReturn(("" + ZK_TIMEOUT + "ms").getBytes(UTF_8));
     replay(zc);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/ZooConfigurationFactoryTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/ZooConfigurationFactoryTest.java
deleted file mode 100644
index db7eb9c..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/ZooConfigurationFactoryTest.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.eq;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.isA;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertSame;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
-import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
-import org.apache.accumulo.server.MockServerContext;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.zookeeper.Watcher;
-import org.junit.Before;
-import org.junit.Test;
-
-public class ZooConfigurationFactoryTest {
-
-  private ServerContext context;
-  private ZooCacheFactory zcf;
-  private ZooCache zc;
-  private ZooConfigurationFactory zconff;
-  private AccumuloConfiguration parent;
-
-  @Before
-  public void setUp() {
-    context = MockServerContext.getWithZK(InstanceId.of("iid"), "localhost", 120000);
-    zcf = createMock(ZooCacheFactory.class);
-    zc = createMock(ZooCache.class);
-    zconff = new ZooConfigurationFactory();
-    parent = createMock(AccumuloConfiguration.class);
-  }
-
-  @Test
-  public void testGetInstance() {
-    replay(context);
-    expect(zcf.getZooCache(eq("localhost"), eq(120000), isA(Watcher.class))).andReturn(zc)
-        .anyTimes();
-    replay(zcf);
-
-    ZooConfiguration c = zconff.getInstance(context, zcf, parent);
-    assertNotNull(c);
-    assertSame(c, zconff.getInstance(context, zcf, parent));
-
-    verify(context);
-    verify(zcf);
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/DistributedWorkQueueWorkAssigner.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/DistributedWorkQueueWorkAssigner.java
index 4bad480..9ccf59f 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/DistributedWorkQueueWorkAssigner.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/replication/DistributedWorkQueueWorkAssigner.java
@@ -110,7 +110,7 @@ public abstract class DistributedWorkQueueWorkAssigner implements WorkAssigner {
     initializeQueuedWork();
 
     if (zooCache == null) {
-      zooCache = new ZooCache(workQueue.getZooReaderWriter());
+      zooCache = new ZooCache(workQueue.getZooReaderWriter(), null);
     }
 
     // Get the maximum number of entries we want to queue work for (or the default)
diff --git a/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java b/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
index 5cc93ca..7e07cb6 100644
--- a/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
@@ -34,7 +34,7 @@ import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.admin.InstanceOperations;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
+import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
@@ -99,9 +99,9 @@ public class BadDeleteMarkersCreatedIT extends AccumuloClusterHarness {
     }
     getCluster().getClusterControl().stopAllServers(ServerType.GARBAGE_COLLECTOR);
 
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      ClientInfo info = ClientInfo.from(client.properties());
-      ZooCache zcache = new ZooCache(info.getZooKeepers(), info.getZooKeepersSessionTimeOut());
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build();
+        ClientContext context = (ClientContext) client) {
+      ZooCache zcache = context.getZooCache();
       zcache.clear();
       var path = ServiceLock
           .path(ZooUtil.getRoot(client.instanceOperations().getInstanceId()) + Constants.ZGC_LOCK);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java
index 22f5d26..cc17d13 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java
@@ -31,6 +31,7 @@ import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooReader;
 import org.slf4j.LoggerFactory;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
@@ -47,7 +48,7 @@ public class CacheTestReader {
     File myfile = new File(reportDir + "/" + UUID.randomUUID());
     myfile.deleteOnExit();
 
-    ZooCache zc = new ZooCache(keepers, 30000);
+    ZooCache zc = new ZooCache(new ZooReader(keepers, 30000), null);
 
     while (true) {
       if (myfile.exists() && !myfile.delete()) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
index 21fe72b..bbb296c 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
@@ -46,7 +46,6 @@ import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.TableId;
@@ -60,7 +59,6 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Cu
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.fate.util.UtilWaitThread;
 import org.apache.accumulo.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.server.manager.state.CurrentState;
@@ -291,10 +289,10 @@ public class TabletStateChangeIteratorIT extends AccumuloClusterHarness {
 
   private static class State implements CurrentState {
 
-    final AccumuloClient client;
+    final ClientContext context;
 
     State(AccumuloClient client) {
-      this.client = client;
+      this.context = (ClientContext) client;
     }
 
     private Set<TServerInstance> tservers;
@@ -303,13 +301,11 @@ public class TabletStateChangeIteratorIT extends AccumuloClusterHarness {
     @Override
     public Set<TServerInstance> onlineTabletServers() {
       HashSet<TServerInstance> tservers = new HashSet<>();
-      for (String tserver : client.instanceOperations().getTabletServers()) {
+      for (String tserver : context.instanceOperations().getTabletServers()) {
         try {
-          var zPath = ServiceLock.path(ZooUtil.getRoot(client.instanceOperations().getInstanceId())
+          var zPath = ServiceLock.path(ZooUtil.getRoot(context.instanceOperations().getInstanceId())
               + Constants.ZTSERVERS + "/" + tserver);
-          ClientInfo info = getClientInfo();
-          long sessionId = ServiceLock.getSessionId(
-              new ZooCache(info.getZooKeepers(), info.getZooKeepersSessionTimeOut()), zPath);
+          long sessionId = ServiceLock.getSessionId(context.getZooCache(), zPath);
           tservers.add(new TServerInstance(tserver, sessionId));
         } catch (Exception e) {
           throw new RuntimeException(e);
@@ -321,7 +317,6 @@ public class TabletStateChangeIteratorIT extends AccumuloClusterHarness {
 
     @Override
     public Set<TableId> onlineTables() {
-      ClientContext context = (ClientContext) client;
       Set<TableId> onlineTables = context.getTableIdToNameMap().keySet();
       this.onlineTables =
           Sets.filter(onlineTables, tableId -> context.getTableState(tableId) == TableState.ONLINE);
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java
index 9c46d06..3ba5134 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java
@@ -56,7 +56,7 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
+import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -85,7 +85,6 @@ import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
-import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.gc.SimpleGarbageCollector;
 import org.apache.accumulo.minicluster.ServerType;
@@ -205,9 +204,7 @@ public class ReplicationIT extends ConfigurableMacBase {
 
   private void waitForGCLock(AccumuloClient client) throws InterruptedException {
     // Check if the GC process has the lock before wasting our retry attempts
-    ZooCacheFactory zcf = new ZooCacheFactory();
-    ClientInfo info = ClientInfo.from(client.properties());
-    ZooCache zcache = zcf.getZooCache(info.getZooKeepers(), info.getZooKeepersSessionTimeOut());
+    ZooCache zcache = ((ClientContext) client).getZooCache();
     var zkPath = ServiceLock
         .path(ZooUtil.getRoot(client.instanceOperations().getInstanceId()) + Constants.ZGC_LOCK);
     log.info("Looking for GC lock at {}", zkPath);