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 aa7b4952c2 Use ZooCache detached from Context for ServerPaths (#5253)
aa7b4952c2 is described below

commit aa7b4952c27eb1309dca286d62c21595cb53c641
Author: Christopher Tubbs <ctubb...@apache.org>
AuthorDate: Tue Jan 14 13:17:26 2025 -0500

    Use ZooCache detached from Context for ServerPaths (#5253)
    
    * Use ZooCache detached from Context for ServerPaths
    
    ServiceLockPaths cannot use ClientContext because that can be closed,
    and ServiceLockPaths needs to be reused inside the tablet locator thread
    pool, which is a static singleton that can survive longer than the
    client context.
    
    This removes context from ServiceLockPaths and causes
    ZookeeperLockChecker to provide it with a separate root path and
    ZooCache.
    
    This fixes the ITs broken after the merge of #5192 in
    e745a5dcd4646059f71a026caa277651295f5df0
    
    * Fix ManagerAssignmentIT
    
    * This fixes ManagerAssignmentIT flakiness that occurs because of the
      reuse of a ZooSession from a closed client connection in subsequent
      tests (flakiness depends on test order)
    * This should be fixed in subsequent fixes by removing the static
      singleton location cache and coupling it to the Context lifecycle
      (ServerContext or AccumuloClient/ClientContext) instead
    * This fixes the flakiness by maintaining a single AccumuloClient for
      the entirety of the SharedMiniClusterBase test cases; since only one
      cluster instance is used for the test, it only needs one client
      instance
---
 .../accumulo/core/clientImpl/ClientContext.java    |   6 +-
 .../core/clientImpl/ZookeeperLockChecker.java      |   4 +-
 .../accumulo/core/lock/ServiceLockPaths.java       |  73 +--
 .../core/clientImpl/ZookeeperLockCheckerTest.java  |   4 +-
 .../accumulo/core/lock/ServiceLockPathsTest.java   | 631 ++++++-------------
 .../org/apache/accumulo/server/util/AdminTest.java |   8 +-
 .../accumulo/server/util/ServiceStatusCmdTest.java |   2 +-
 .../accumulo/test/fate/FateOpsCommandsIT.java      |   5 +-
 .../test/functional/ManagerAssignmentIT.java       | 687 ++++++++++-----------
 9 files changed, 590 insertions(+), 830 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index 966ca3b62e..e898a1424c 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -247,7 +247,8 @@ public class ClientContext implements AccumuloClient {
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
-    this.serverPaths = Suppliers.memoize(() -> new ServiceLockPaths(this));
+    this.serverPaths =
+        Suppliers.memoize(() -> new ServiceLockPaths(this.getZooKeeperRoot(), 
this.getZooCache()));
     if (ueh == Threads.UEH) {
       clientThreadPools = ThreadPools.getServerThreadPools();
     } else {
@@ -1061,8 +1062,7 @@ public class ClientContext implements AccumuloClient {
       // because that client could be closed, and its ZooSession also closed
       // this needs to be fixed; TODO 
https://github.com/apache/accumulo/issues/2301
       var zk = 
info.getZooKeeperSupplier(ZookeeperLockChecker.class.getSimpleName()).get();
-      this.zkLockChecker =
-          new ZookeeperLockChecker(new ZooCache(zk), getZooKeeperRoot(), 
getServerPaths());
+      this.zkLockChecker = new ZookeeperLockChecker(new ZooCache(zk), 
getZooKeeperRoot());
     }
     return this.zkLockChecker;
   }
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java
 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java
index cd9c3b7de7..314a82788a 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java
@@ -38,10 +38,10 @@ public class ZookeeperLockChecker implements 
TabletServerLockChecker {
   private final String root;
   private final ServiceLockPaths lockPaths;
 
-  ZookeeperLockChecker(ZooCache zooCache, String zkRoot, ServiceLockPaths 
serviceLockPaths) {
+  ZookeeperLockChecker(ZooCache zooCache, String zkRoot) {
     this.zc = requireNonNull(zooCache);
     this.root = requireNonNull(zkRoot);
-    this.lockPaths = requireNonNull(serviceLockPaths);
+    this.lockPaths = new ServiceLockPaths(this.root, this.zc);
   }
 
   public boolean doesTabletServerLockExist(String server) {
diff --git 
a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java 
b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java
index 01c1c61032..4c82d085e5 100644
--- a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java
+++ b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java
@@ -18,10 +18,11 @@
  */
 package org.apache.accumulo.core.lock;
 
+import static java.util.Objects.requireNonNull;
+
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-import java.util.Objects;
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
@@ -33,7 +34,6 @@ import java.util.concurrent.FutureTask;
 import java.util.function.Predicate;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat;
 import org.apache.accumulo.core.util.threads.ThreadPoolNames;
@@ -68,8 +68,8 @@ public class ServiceLockPaths {
      * Create a ServiceLockPath for a management process
      */
     private ServiceLockPath(String root, String type) {
-      Objects.requireNonNull(root);
-      this.type = Objects.requireNonNull(type);
+      requireNonNull(root);
+      this.type = requireNonNull(type);
       Preconditions.checkArgument(this.type.equals(Constants.ZGC_LOCK)
           || this.type.equals(Constants.ZMANAGER_LOCK) || 
this.type.equals(Constants.ZMONITOR_LOCK)
           || this.type.equals(Constants.ZTABLE_LOCKS), "Unsupported type: " + 
type);
@@ -84,13 +84,13 @@ public class ServiceLockPaths {
      * Create a ServiceLockPath for ZTABLE_LOCKS
      */
     private ServiceLockPath(String root, String type, String content) {
-      Objects.requireNonNull(root);
-      this.type = Objects.requireNonNull(type);
+      requireNonNull(root);
+      this.type = requireNonNull(type);
       Preconditions.checkArgument(
           this.type.equals(Constants.ZTABLE_LOCKS) || 
this.type.equals(Constants.ZMINI_LOCK),
           "Unsupported type: " + type);
       this.resourceGroup = null;
-      this.server = Objects.requireNonNull(content);
+      this.server = requireNonNull(content);
       this.path = root + this.type + "/" + this.server;
     }
 
@@ -98,14 +98,14 @@ public class ServiceLockPaths {
      * Create a ServiceLockPath for a worker process
      */
     private ServiceLockPath(String root, String type, String resourceGroup, 
String server) {
-      Objects.requireNonNull(root);
-      this.type = Objects.requireNonNull(type);
+      requireNonNull(root);
+      this.type = requireNonNull(type);
       Preconditions.checkArgument(
           this.type.equals(Constants.ZCOMPACTORS) || 
this.type.equals(Constants.ZSSERVERS)
               || this.type.equals(Constants.ZTSERVERS) || 
this.type.equals(Constants.ZDEADTSERVERS),
           "Unsupported type: " + type);
-      this.resourceGroup = Objects.requireNonNull(resourceGroup);
-      this.server = Objects.requireNonNull(server);
+      this.resourceGroup = requireNonNull(resourceGroup);
+      this.server = requireNonNull(server);
       this.path = root + this.type + "/" + this.resourceGroup + "/" + 
this.server;
     }
 
@@ -184,10 +184,12 @@ public class ServiceLockPaths {
 
   private final ExecutorService fetchExectuor;
 
-  private final ClientContext ctx;
+  private final String zkRoot;
+  private final ZooCache zooCache;
 
-  public ServiceLockPaths(ClientContext context) {
-    this.ctx = context;
+  public ServiceLockPaths(String zkRoot, ZooCache zc) {
+    this.zkRoot = requireNonNull(zkRoot);
+    this.zooCache = requireNonNull(zc);
     this.fetchExectuor = ThreadPools.getServerThreadPools()
         
.getPoolBuilder(ThreadPoolNames.SERVICE_LOCK_POOL).numCoreThreads(16).build();
   }
@@ -219,8 +221,8 @@ public class ServiceLockPaths {
    * Parse a ZooKeeper path string and return a ServiceLockPath
    */
   public static ServiceLockPath parse(Optional<String> serverType, String 
path) {
-    Objects.requireNonNull(serverType);
-    Objects.requireNonNull(path);
+    requireNonNull(serverType);
+    requireNonNull(path);
 
     final String type = serverType.orElseGet(() -> determineServerType(path));
 
@@ -253,47 +255,47 @@ public class ServiceLockPaths {
   }
 
   public ServiceLockPath createGarbageCollectorPath() {
-    return new ServiceLockPath(ctx.getZooKeeperRoot(), Constants.ZGC_LOCK);
+    return new ServiceLockPath(zkRoot, Constants.ZGC_LOCK);
   }
 
   public ServiceLockPath createManagerPath() {
-    return new ServiceLockPath(ctx.getZooKeeperRoot(), 
Constants.ZMANAGER_LOCK);
+    return new ServiceLockPath(zkRoot, Constants.ZMANAGER_LOCK);
   }
 
   public ServiceLockPath createMiniPath(String miniUUID) {
-    return new ServiceLockPath(ctx.getZooKeeperRoot(), Constants.ZMINI_LOCK, 
miniUUID);
+    return new ServiceLockPath(zkRoot, Constants.ZMINI_LOCK, miniUUID);
   }
 
   public ServiceLockPath createMonitorPath() {
-    return new ServiceLockPath(ctx.getZooKeeperRoot(), 
Constants.ZMONITOR_LOCK);
+    return new ServiceLockPath(zkRoot, Constants.ZMONITOR_LOCK);
   }
 
   public ServiceLockPath createCompactorPath(String resourceGroup, HostAndPort 
serverAddress) {
-    return new ServiceLockPath(ctx.getZooKeeperRoot(), Constants.ZCOMPACTORS, 
resourceGroup,
+    return new ServiceLockPath(zkRoot, Constants.ZCOMPACTORS, resourceGroup,
         serverAddress.toString());
   }
 
   public ServiceLockPath createScanServerPath(String resourceGroup, 
HostAndPort serverAddress) {
-    return new ServiceLockPath(ctx.getZooKeeperRoot(), Constants.ZSSERVERS, 
resourceGroup,
+    return new ServiceLockPath(zkRoot, Constants.ZSSERVERS, resourceGroup,
         serverAddress.toString());
   }
 
   public ServiceLockPath createTableLocksPath() {
-    return new ServiceLockPath(ctx.getZooKeeperRoot(), Constants.ZTABLE_LOCKS);
+    return new ServiceLockPath(zkRoot, Constants.ZTABLE_LOCKS);
   }
 
   public ServiceLockPath createTableLocksPath(String tableId) {
-    return new ServiceLockPath(ctx.getZooKeeperRoot(), Constants.ZTABLE_LOCKS, 
tableId);
+    return new ServiceLockPath(zkRoot, Constants.ZTABLE_LOCKS, tableId);
   }
 
   public ServiceLockPath createTabletServerPath(String resourceGroup, 
HostAndPort serverAddress) {
-    return new ServiceLockPath(ctx.getZooKeeperRoot(), Constants.ZTSERVERS, 
resourceGroup,
+    return new ServiceLockPath(zkRoot, Constants.ZTSERVERS, resourceGroup,
         serverAddress.toString());
   }
 
   public ServiceLockPath createDeadTabletServerPath(String resourceGroup,
       HostAndPort serverAddress) {
-    return new ServiceLockPath(ctx.getZooKeeperRoot(), 
Constants.ZDEADTSERVERS, resourceGroup,
+    return new ServiceLockPath(zkRoot, Constants.ZDEADTSERVERS, resourceGroup,
         serverAddress.toString());
   }
 
@@ -421,13 +423,12 @@ public class ServiceLockPaths {
       ResourceGroupPredicate resourceGroupPredicate, AddressSelector 
addressSelector,
       boolean withLock) {
 
-    Objects.requireNonNull(serverType);
-    Objects.requireNonNull(resourceGroupPredicate);
-    Objects.requireNonNull(addressSelector);
+    requireNonNull(serverType);
+    requireNonNull(resourceGroupPredicate);
+    requireNonNull(addressSelector);
 
     final Set<ServiceLockPath> results = ConcurrentHashMap.newKeySet();
-    final String typePath = ctx.getZooKeeperRoot() + serverType;
-    final ZooCache cache = ctx.getZooCache();
+    final String typePath = zkRoot + serverType;
 
     if (serverType.equals(Constants.ZGC_LOCK) || 
serverType.equals(Constants.ZMANAGER_LOCK)
         || serverType.equals(Constants.ZMONITOR_LOCK)) {
@@ -436,14 +437,14 @@ public class ServiceLockPaths {
       if (!withLock) {
         results.add(slp);
       } else {
-        Optional<ServiceLockData> sld = ServiceLock.getLockData(cache, slp, 
stat);
+        Optional<ServiceLockData> sld = ServiceLock.getLockData(zooCache, slp, 
stat);
         if (!sld.isEmpty()) {
           results.add(slp);
         }
       }
     } else if (serverType.equals(Constants.ZCOMPACTORS) || 
serverType.equals(Constants.ZSSERVERS)
         || serverType.equals(Constants.ZTSERVERS) || 
serverType.equals(Constants.ZDEADTSERVERS)) {
-      final List<String> resourceGroups = cache.getChildren(typePath);
+      final List<String> resourceGroups = zooCache.getChildren(typePath);
       for (final String group : resourceGroups) {
         if (resourceGroupPredicate.test(group)) {
           final Collection<String> servers;
@@ -451,7 +452,7 @@ public class ServiceLockPaths {
 
           if (addressSelector.getExactAddress() != null) {
             var server = addressSelector.getExactAddress().toString();
-            if (withLock || cache.get(typePath + "/" + group + "/" + server) 
!= null) {
+            if (withLock || zooCache.get(typePath + "/" + group + "/" + 
server) != null) {
               // When withLock is true the server in the list may not exist in 
zookeeper, if it does
               // not exist then no lock will be found later when looking for a 
lock in zookeeper.
               servers = List.of(server);
@@ -460,7 +461,7 @@ public class ServiceLockPaths {
             }
             addressPredicate = s -> true;
           } else {
-            servers = cache.getChildren(typePath + "/" + group);
+            servers = zooCache.getChildren(typePath + "/" + group);
             addressPredicate = addressSelector.getPredicate();
           }
 
@@ -484,7 +485,7 @@ public class ServiceLockPaths {
                 // connection at the same time though.
                 var futureTask = new FutureTask<>(() -> {
                   final ZcStat stat = new ZcStat();
-                  Optional<ServiceLockData> sld = 
ServiceLock.getLockData(cache, slp, stat);
+                  Optional<ServiceLockData> sld = 
ServiceLock.getLockData(zooCache, slp, stat);
                   if (sld.isPresent()) {
                     results.add(slp);
                   }
diff --git 
a/core/src/test/java/org/apache/accumulo/core/clientImpl/ZookeeperLockCheckerTest.java
 
b/core/src/test/java/org/apache/accumulo/core/clientImpl/ZookeeperLockCheckerTest.java
index 7f6f0e37c5..bb7402f78d 100644
--- 
a/core/src/test/java/org/apache/accumulo/core/clientImpl/ZookeeperLockCheckerTest.java
+++ 
b/core/src/test/java/org/apache/accumulo/core/clientImpl/ZookeeperLockCheckerTest.java
@@ -32,7 +32,6 @@ import java.util.function.Predicate;
 import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.lock.ServiceLockPaths;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
@@ -59,8 +58,7 @@ public class ZookeeperLockCheckerTest {
 
   @Test
   public void testInvalidateCache() {
-    var zklc =
-        new ZookeeperLockChecker(zc, context.getZooKeeperRoot(), new 
ServiceLockPaths(context));
+    var zklc = new ZookeeperLockChecker(zc, context.getZooKeeperRoot());
 
     verify(zc);
     reset(zc);
diff --git 
a/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockPathsTest.java 
b/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockPathsTest.java
index e35cbd616b..0232376237 100644
--- a/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockPathsTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockPathsTest.java
@@ -28,6 +28,10 @@ import static 
org.apache.accumulo.core.Constants.ZMONITOR_LOCK;
 import static org.apache.accumulo.core.Constants.ZSSERVERS;
 import static org.apache.accumulo.core.Constants.ZTABLE_LOCKS;
 import static org.apache.accumulo.core.Constants.ZTSERVERS;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
@@ -41,13 +45,14 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.UUID;
 
-import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat;
 import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.lock.ServiceLockPaths.AddressSelector;
 import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath;
 import org.easymock.EasyMock;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
 import com.google.common.net.HostAndPort;
@@ -60,17 +65,26 @@ public class ServiceLockPathsTest {
   private static final String HOSTNAME_NO_LOCK = "localhost:9877";
   private static final HostAndPort hp = HostAndPort.fromString(HOSTNAME);
 
-  @Test
-  public void testPathGeneration() {
+  private ZooCache zc;
+  private ServiceLockPaths paths;
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
+  @BeforeEach
+  public void setupMocks() {
+    zc = createMock(ZooCache.class);
+    paths = new ServiceLockPaths(ROOT, zc);
+  }
 
-    EasyMock.replay(ctx);
+  @AfterEach
+  public void verifyMocks() {
+    verify(zc);
+  }
+
+  @Test
+  public void testPathGeneration() {
+    replay(zc);
 
-    ServiceLockPaths paths = new ServiceLockPaths(ctx);
     // Test management process path creation
-    ServiceLockPath slp = paths.createGarbageCollectorPath();
+    var slp = paths.createGarbageCollectorPath();
     assertNull(slp.getServer());
     assertNull(slp.getResourceGroup());
     assertEquals(ZGC_LOCK, slp.getType());
@@ -132,323 +146,190 @@ public class ServiceLockPathsTest {
     assertEquals(TEST_RESOURCE_GROUP, slp.getResourceGroup());
     assertEquals(ZTSERVERS, slp.getType());
     assertEquals(ROOT + ZTSERVERS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME, slp.toString());
-
-    EasyMock.verify(ctx);
   }
 
   @Test
   public void testGetGarbageCollectorNotRunning() {
+    expect(zc.getChildren(ROOT + ZGC_LOCK)).andReturn(List.of()).anyTimes();
+    replay(zc);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + 
ZGC_LOCK)).andReturn(List.of()).anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx, zc);
-
-    ServiceLockPath slp = ctx.getServerPaths().getGarbageCollector(true);
+    var slp = paths.getGarbageCollector(true);
     assertNull(slp);
-
-    EasyMock.verify(ctx, zc);
-
   }
 
   @Test
   public void testGetGarbageCollectorNoLock() {
+    expect(zc.getChildren(ROOT + 
ZGC_LOCK)).andReturn(List.of(HOSTNAME_NO_LOCK)).anyTimes();
+    replay(zc);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + 
ZGC_LOCK)).andReturn(List.of(HOSTNAME_NO_LOCK))
-        .anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx, zc);
-
-    ServiceLockPath slp = ctx.getServerPaths().getGarbageCollector(false);
+    var slp = paths.getGarbageCollector(false);
     assertNotNull(slp);
     assertNull(slp.getServer());
     assertNull(slp.getResourceGroup());
     assertEquals(ZGC_LOCK, slp.getType());
     assertEquals(ROOT + ZGC_LOCK, slp.toString());
-
-    EasyMock.verify(ctx, zc);
-
   }
 
   @Test
   public void testGetGarbageCollector() {
-
     UUID uuid = UUID.randomUUID();
     String svcLock1 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + 
"#0000000001";
     String svcLock2 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + 
"#0000000002";
-    ServiceLockData sld =
-        new ServiceLockData(uuid, HOSTNAME, ThriftService.GC, 
TEST_RESOURCE_GROUP);
-
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + 
ZGC_LOCK)).andReturn(List.of(svcLock1, svcLock2))
-        .anyTimes();
-    EasyMock
-        .expect(zc.get(EasyMock.eq(ROOT + ZGC_LOCK + "/" + svcLock1), 
EasyMock.isA(ZcStat.class)))
+    var sld = new ServiceLockData(uuid, HOSTNAME, ThriftService.GC, 
TEST_RESOURCE_GROUP);
+
+    expect(zc.getChildren(ROOT + ZGC_LOCK)).andReturn(List.of(svcLock1, 
svcLock2)).anyTimes();
+    expect(zc.get(EasyMock.eq(ROOT + ZGC_LOCK + "/" + svcLock1), 
EasyMock.isA(ZcStat.class)))
         .andReturn(sld.serialize());
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx, zc);
+    replay(zc);
 
-    ServiceLockPath slp = ctx.getServerPaths().getGarbageCollector(true);
+    var slp = paths.getGarbageCollector(true);
     assertNotNull(slp);
     assertNull(slp.getServer());
     assertNull(slp.getResourceGroup());
     assertEquals(ZGC_LOCK, slp.getType());
     assertEquals(ROOT + ZGC_LOCK, slp.toString());
-
-    EasyMock.verify(ctx, zc);
-
   }
 
   @Test
   public void testGetManagerNotRunning() {
+    expect(zc.getChildren(ROOT + 
ZMANAGER_LOCK)).andReturn(List.of()).anyTimes();
+    replay(zc);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + 
ZMANAGER_LOCK)).andReturn(List.of()).anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx, zc);
-
-    ServiceLockPath slp = ctx.getServerPaths().getManager(true);
+    var slp = paths.getManager(true);
     assertNull(slp);
-
-    EasyMock.verify(ctx, zc);
-
   }
 
   @Test
   public void testGetManagerNoLock() {
+    expect(zc.getChildren(ROOT + 
ZMANAGER_LOCK)).andReturn(List.of(HOSTNAME_NO_LOCK)).anyTimes();
+    replay(zc);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + 
ZMANAGER_LOCK)).andReturn(List.of(HOSTNAME_NO_LOCK))
-        .anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx, zc);
-
-    ServiceLockPath slp = ctx.getServerPaths().getManager(false);
+    var slp = paths.getManager(false);
     assertNotNull(slp);
     assertNull(slp.getServer());
     assertNull(slp.getResourceGroup());
     assertEquals(ZMANAGER_LOCK, slp.getType());
     assertEquals(ROOT + ZMANAGER_LOCK, slp.toString());
-
-    EasyMock.verify(ctx, zc);
-
   }
 
   @Test
   public void testGetManager() {
-
     UUID uuid = UUID.randomUUID();
     String svcLock1 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + 
"#0000000001";
     String svcLock2 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + 
"#0000000002";
-    ServiceLockData sld =
-        new ServiceLockData(uuid, HOSTNAME, ThriftService.MANAGER, 
TEST_RESOURCE_GROUP);
-
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + 
ZMANAGER_LOCK)).andReturn(List.of(svcLock1, svcLock2))
-        .anyTimes();
-    EasyMock
-        .expect(
-            zc.get(EasyMock.eq(ROOT + ZMANAGER_LOCK + "/" + svcLock1), 
EasyMock.isA(ZcStat.class)))
+    var sld = new ServiceLockData(uuid, HOSTNAME, ThriftService.MANAGER, 
TEST_RESOURCE_GROUP);
+
+    expect(zc.getChildren(ROOT + ZMANAGER_LOCK)).andReturn(List.of(svcLock1, 
svcLock2)).anyTimes();
+    expect(zc.get(EasyMock.eq(ROOT + ZMANAGER_LOCK + "/" + svcLock1), 
EasyMock.isA(ZcStat.class)))
         .andReturn(sld.serialize());
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx, zc);
+    replay(zc);
 
-    ServiceLockPath slp = ctx.getServerPaths().getManager(true);
+    var slp = paths.getManager(true);
     assertNotNull(slp);
     assertNull(slp.getServer());
     assertNull(slp.getResourceGroup());
     assertEquals(ZMANAGER_LOCK, slp.getType());
     assertEquals(ROOT + ZMANAGER_LOCK, slp.toString());
-
-    EasyMock.verify(ctx, zc);
-
   }
 
   @Test
   public void testGetMonitorNotRunning() {
+    expect(zc.getChildren(ROOT + 
ZMONITOR_LOCK)).andReturn(List.of()).anyTimes();
+    replay(zc);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + 
ZMONITOR_LOCK)).andReturn(List.of()).anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx, zc);
-
-    ServiceLockPath slp = ctx.getServerPaths().getMonitor(true);
+    var slp = paths.getMonitor(true);
     assertNull(slp);
-
-    EasyMock.verify(ctx, zc);
-
   }
 
   @Test
   public void testGetMonitorNoLock() {
+    expect(zc.getChildren(ROOT + 
ZMONITOR_LOCK)).andReturn(List.of(HOSTNAME_NO_LOCK)).anyTimes();
+    replay(zc);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + 
ZMONITOR_LOCK)).andReturn(List.of(HOSTNAME_NO_LOCK))
-        .anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx, zc);
-
-    ServiceLockPath slp = ctx.getServerPaths().getMonitor(false);
+    var slp = paths.getMonitor(false);
     assertNotNull(slp);
     assertNull(slp.getServer());
     assertNull(slp.getResourceGroup());
     assertEquals(ZMONITOR_LOCK, slp.getType());
     assertEquals(ROOT + ZMONITOR_LOCK, slp.toString());
-
-    EasyMock.verify(ctx, zc);
-
   }
 
   @Test
   public void testGetMonitor() {
-
     UUID uuid = UUID.randomUUID();
     String svcLock1 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + 
"#0000000001";
     String svcLock2 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + 
"#0000000002";
-    ServiceLockData sld =
-        new ServiceLockData(uuid, HOSTNAME, ThriftService.NONE, 
TEST_RESOURCE_GROUP);
-
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + 
ZMONITOR_LOCK)).andReturn(List.of(svcLock1, svcLock2))
-        .anyTimes();
-    EasyMock
-        .expect(
-            zc.get(EasyMock.eq(ROOT + ZMONITOR_LOCK + "/" + svcLock1), 
EasyMock.isA(ZcStat.class)))
+    var sld = new ServiceLockData(uuid, HOSTNAME, ThriftService.NONE, 
TEST_RESOURCE_GROUP);
+
+    expect(zc.getChildren(ROOT + ZMONITOR_LOCK)).andReturn(List.of(svcLock1, 
svcLock2)).anyTimes();
+    expect(zc.get(EasyMock.eq(ROOT + ZMONITOR_LOCK + "/" + svcLock1), 
EasyMock.isA(ZcStat.class)))
         .andReturn(sld.serialize());
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx, zc);
+    replay(zc);
 
-    ServiceLockPath slp = ctx.getServerPaths().getMonitor(true);
+    var slp = paths.getMonitor(true);
     assertNotNull(slp);
     assertNull(slp.getServer());
     assertNull(slp.getResourceGroup());
     assertEquals(ZMONITOR_LOCK, slp.getType());
     assertEquals(ROOT + ZMONITOR_LOCK, slp.toString());
-
-    EasyMock.verify(ctx, zc);
-
   }
 
   @Test
   public void testGetCompactorsNotRunning() {
+    expect(zc.getChildren(ROOT + ZCOMPACTORS)).andReturn(List.of()).anyTimes();
+    replay(zc);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + 
ZCOMPACTORS)).andReturn(List.of()).anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx, zc);
-
-    assertThrows(NullPointerException.class,
-        () -> ctx.getServerPaths().getCompactor(null, null, true));
+    assertThrows(NullPointerException.class, () -> paths.getCompactor(null, 
null, true));
     assertThrows(NullPointerException.class,
-        () -> ctx.getServerPaths().getCompactor(rg -> 
rg.equals(TEST_RESOURCE_GROUP), null, true));
-    assertTrue(
-        ctx.getServerPaths().getCompactor(rg -> true, AddressSelector.all(), 
true).isEmpty());
-    assertTrue(ctx.getServerPaths()
-        .getCompactor(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.all(), true).isEmpty());
-    assertTrue(ctx.getServerPaths()
-        .getCompactor(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.exact(hp), true)
+        () -> paths.getCompactor(rg -> rg.equals(TEST_RESOURCE_GROUP), null, 
true));
+    assertTrue(paths.getCompactor(rg -> true, AddressSelector.all(), 
true).isEmpty());
+    assertTrue(paths.getCompactor(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.all(), true)
         .isEmpty());
-
-    EasyMock.verify(ctx, zc);
-
+    assertTrue(
+        paths.getCompactor(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.exact(hp), true)
+            .isEmpty());
   }
 
   @Test
   public void testGetCompactors() {
-
     UUID uuid = UUID.randomUUID();
     String svcLock1 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + 
"#0000000001";
     String svcLock2 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + 
"#0000000002";
-    ServiceLockData sld1 =
-        new ServiceLockData(uuid, HOSTNAME, ThriftService.COMPACTOR, 
TEST_RESOURCE_GROUP);
-    ServiceLockData sld2 =
+    var sld1 = new ServiceLockData(uuid, HOSTNAME, ThriftService.COMPACTOR, 
TEST_RESOURCE_GROUP);
+    var sld2 =
         new ServiceLockData(uuid, HOSTNAME, ThriftService.COMPACTOR, 
DEFAULT_RESOURCE_GROUP_NAME);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZCOMPACTORS))
+    expect(zc.getChildren(ROOT + ZCOMPACTORS))
         .andReturn(List.of(TEST_RESOURCE_GROUP, 
DEFAULT_RESOURCE_GROUP_NAME)).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZCOMPACTORS + "/" + 
TEST_RESOURCE_GROUP))
+    expect(zc.getChildren(ROOT + ZCOMPACTORS + "/" + TEST_RESOURCE_GROUP))
         .andReturn(List.of(HOSTNAME, HOSTNAME_NO_LOCK)).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZCOMPACTORS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME))
+    expect(zc.getChildren(ROOT + ZCOMPACTORS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME))
         .andReturn(List.of(HOSTNAME, HOSTNAME_NO_LOCK)).anyTimes();
-    EasyMock
-        .expect(
-            zc.getChildren(ROOT + ZCOMPACTORS + "/" + TEST_RESOURCE_GROUP + 
"/" + HOSTNAME_NO_LOCK))
+    expect(zc.getChildren(ROOT + ZCOMPACTORS + "/" + TEST_RESOURCE_GROUP + "/" 
+ HOSTNAME_NO_LOCK))
         .andReturn(List.of()).anyTimes();
-    EasyMock
-        .expect(zc.getChildren(
-            ROOT + ZCOMPACTORS + "/" + DEFAULT_RESOURCE_GROUP_NAME + "/" + 
HOSTNAME_NO_LOCK))
+    expect(zc.getChildren(
+        ROOT + ZCOMPACTORS + "/" + DEFAULT_RESOURCE_GROUP_NAME + "/" + 
HOSTNAME_NO_LOCK))
         .andReturn(List.of()).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZCOMPACTORS + "/" + 
TEST_RESOURCE_GROUP + "/" + HOSTNAME))
+    expect(zc.getChildren(ROOT + ZCOMPACTORS + "/" + TEST_RESOURCE_GROUP + "/" 
+ HOSTNAME))
         .andReturn(List.of(svcLock1, svcLock2)).anyTimes();
-    EasyMock
-        .expect(
-            zc.getChildren(ROOT + ZCOMPACTORS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME + "/" + HOSTNAME))
+    expect(zc.getChildren(ROOT + ZCOMPACTORS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME + "/" + HOSTNAME))
         .andReturn(List.of(svcLock1, svcLock2)).anyTimes();
-    EasyMock.expect(zc.get(
+    expect(zc.get(
         EasyMock
             .eq(ROOT + ZCOMPACTORS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME + "/" + svcLock1),
         EasyMock.isA(ZcStat.class))).andReturn(sld1.serialize()).anyTimes();
-    EasyMock
-        .expect(zc.get(EasyMock.eq(ROOT + ZCOMPACTORS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME + "/"
-            + HOSTNAME + "/" + svcLock1), EasyMock.isA(ZcStat.class)))
-        .andReturn(sld2.serialize()).anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock
-        .expect(zc.getChildren(ROOT + ZCOMPACTORS + "/" + TEST_RESOURCE_GROUP 
+ "/localhost:1234"))
+    expect(zc.get(EasyMock.eq(
+        ROOT + ZCOMPACTORS + "/" + DEFAULT_RESOURCE_GROUP_NAME + "/" + 
HOSTNAME + "/" + svcLock1),
+        EasyMock.isA(ZcStat.class))).andReturn(sld2.serialize()).anyTimes();
+
+    expect(zc.getChildren(ROOT + ZCOMPACTORS + "/" + TEST_RESOURCE_GROUP + 
"/localhost:1234"))
         .andReturn(null).anyTimes();
-    EasyMock.expect(zc.get(ROOT + ZCOMPACTORS + "/" + TEST_RESOURCE_GROUP + 
"/localhost:1234"))
+    expect(zc.get(ROOT + ZCOMPACTORS + "/" + TEST_RESOURCE_GROUP + 
"/localhost:1234"))
         .andReturn(null).anyTimes();
-    EasyMock.replay(ctx, zc);
+    replay(zc);
 
     // query for all
-    Set<ServiceLockPath> results =
-        ctx.getServerPaths().getCompactor(rg -> true, AddressSelector.all(), 
false);
+    Set<ServiceLockPath> results = paths.getCompactor(rg -> true, 
AddressSelector.all(), false);
     assertEquals(4, results.size());
     for (ServiceLockPath path : results) {
       assertEquals(ZCOMPACTORS, path.getType());
@@ -465,10 +346,10 @@ public class ServiceLockPathsTest {
     }
 
     // query for all with locks
-    results = ctx.getServerPaths().getCompactor(rg -> true, 
AddressSelector.all(), true);
+    results = paths.getCompactor(rg -> true, AddressSelector.all(), true);
     assertEquals(2, results.size());
     Iterator<ServiceLockPath> iter = results.iterator();
-    ServiceLockPath slp1 = iter.next();
+    var slp1 = iter.next();
     assertEquals(HOSTNAME, slp1.getServer());
     assertEquals(ZCOMPACTORS, slp1.getType());
     if (slp1.getResourceGroup().equals(TEST_RESOURCE_GROUP)) {
@@ -480,7 +361,7 @@ public class ServiceLockPathsTest {
     } else {
       fail("wrong resource group");
     }
-    ServiceLockPath slp2 = iter.next();
+    var slp2 = iter.next();
     assertEquals(HOSTNAME, slp2.getServer());
     assertEquals(ZCOMPACTORS, slp2.getType());
     if (slp2.getResourceGroup().equals(TEST_RESOURCE_GROUP)) {
@@ -494,13 +375,12 @@ public class ServiceLockPathsTest {
     }
 
     // query for all in non-existent resource group
-    results = ctx.getServerPaths().getCompactor(rg -> 
rg.equals("FAKE_RESOURCE_GROUP"),
-        AddressSelector.all(), true);
+    results =
+        paths.getCompactor(rg -> rg.equals("FAKE_RESOURCE_GROUP"), 
AddressSelector.all(), true);
     assertEquals(0, results.size());
 
     // query for all in test resource group
-    results = ctx.getServerPaths().getCompactor(rg -> 
rg.equals(TEST_RESOURCE_GROUP),
-        AddressSelector.all(), true);
+    results = paths.getCompactor(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.all(), true);
     assertEquals(1, results.size());
     iter = results.iterator();
     slp1 = iter.next();
@@ -510,8 +390,8 @@ public class ServiceLockPathsTest {
     assertEquals(ROOT + ZCOMPACTORS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME, slp1.toString());
 
     // query for a specific server
-    results = ctx.getServerPaths().getCompactor(rg -> 
rg.equals(TEST_RESOURCE_GROUP),
-        AddressSelector.exact(hp), true);
+    results =
+        paths.getCompactor(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.exact(hp), true);
     assertEquals(1, results.size());
     iter = results.iterator();
     slp1 = iter.next();
@@ -522,99 +402,70 @@ public class ServiceLockPathsTest {
 
     // query for a wrong server
     for (boolean withLock : new boolean[] {true, false}) {
-      results = ctx.getServerPaths().getCompactor(rg -> 
rg.equals(TEST_RESOURCE_GROUP),
+      results = paths.getCompactor(rg -> rg.equals(TEST_RESOURCE_GROUP),
           AddressSelector.exact(HostAndPort.fromString("localhost:1234")), 
withLock);
       assertEquals(0, results.size());
-      results = ctx.getServerPaths().getCompactor(rg -> 
rg.equals(TEST_RESOURCE_GROUP),
+      results = paths.getCompactor(rg -> rg.equals(TEST_RESOURCE_GROUP),
           AddressSelector.matching(hp -> hp.equals("localhost:1234")), 
withLock);
       assertEquals(0, results.size());
     }
-
-    EasyMock.verify(ctx, zc);
-
   }
 
   @Test
   public void testGetScanServersNotRunning() {
+    expect(zc.getChildren(ROOT + ZSSERVERS)).andReturn(List.of()).anyTimes();
+    replay(zc);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + 
ZSSERVERS)).andReturn(List.of()).anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx, zc);
-
+    assertThrows(NullPointerException.class, () -> paths.getScanServer(null, 
null, true));
     assertThrows(NullPointerException.class,
-        () -> ctx.getServerPaths().getScanServer(null, null, true));
-    assertThrows(NullPointerException.class,
-        () -> ctx.getServerPaths().getScanServer(rg -> 
rg.equals(TEST_RESOURCE_GROUP), null, true));
+        () -> paths.getScanServer(rg -> rg.equals(TEST_RESOURCE_GROUP), null, 
true));
+    assertTrue(paths.getScanServer(rg -> true, AddressSelector.all(), 
true).isEmpty());
     assertTrue(
-        ctx.getServerPaths().getScanServer(rg -> true, AddressSelector.all(), 
true).isEmpty());
-    assertTrue(ctx.getServerPaths()
-        .getScanServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.all(), true)
-        .isEmpty());
-    assertTrue(ctx.getServerPaths()
-        .getScanServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.exact(hp), true)
-        .isEmpty());
-
-    EasyMock.verify(ctx, zc);
-
+        paths.getScanServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.all(), true)
+            .isEmpty());
+    assertTrue(
+        paths.getScanServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.exact(hp), true)
+            .isEmpty());
   }
 
   @Test
   public void testGetScanServers() {
-
     UUID uuid = UUID.randomUUID();
     String svcLock1 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + 
"#0000000001";
     String svcLock2 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + 
"#0000000002";
-    ServiceLockData sld1 =
-        new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, 
TEST_RESOURCE_GROUP);
-    ServiceLockData sld2 =
+    var sld1 = new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, 
TEST_RESOURCE_GROUP);
+    var sld2 =
         new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, 
DEFAULT_RESOURCE_GROUP_NAME);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZSSERVERS))
+    expect(zc.getChildren(ROOT + ZSSERVERS))
         .andReturn(List.of(TEST_RESOURCE_GROUP, 
DEFAULT_RESOURCE_GROUP_NAME)).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZSSERVERS + "/" + 
TEST_RESOURCE_GROUP))
+    expect(zc.getChildren(ROOT + ZSSERVERS + "/" + TEST_RESOURCE_GROUP))
         .andReturn(List.of(HOSTNAME, HOSTNAME_NO_LOCK)).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZSSERVERS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME))
+    expect(zc.getChildren(ROOT + ZSSERVERS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME))
         .andReturn(List.of(HOSTNAME, HOSTNAME_NO_LOCK)).anyTimes();
-    EasyMock
-        .expect(
-            zc.getChildren(ROOT + ZSSERVERS + "/" + TEST_RESOURCE_GROUP + "/" 
+ HOSTNAME_NO_LOCK))
+    expect(zc.getChildren(ROOT + ZSSERVERS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME_NO_LOCK))
         .andReturn(List.of()).anyTimes();
-    EasyMock
-        .expect(zc.getChildren(
-            ROOT + ZSSERVERS + "/" + DEFAULT_RESOURCE_GROUP_NAME + "/" + 
HOSTNAME_NO_LOCK))
+    expect(zc
+        .getChildren(ROOT + ZSSERVERS + "/" + DEFAULT_RESOURCE_GROUP_NAME + 
"/" + HOSTNAME_NO_LOCK))
         .andReturn(List.of()).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZSSERVERS + "/" + 
TEST_RESOURCE_GROUP + "/" + HOSTNAME))
+    expect(zc.getChildren(ROOT + ZSSERVERS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME))
         .andReturn(List.of(svcLock1, svcLock2)).anyTimes();
-    EasyMock
-        .expect(
-            zc.getChildren(ROOT + ZSSERVERS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME + "/" + HOSTNAME))
+    expect(zc.getChildren(ROOT + ZSSERVERS + "/" + DEFAULT_RESOURCE_GROUP_NAME 
+ "/" + HOSTNAME))
         .andReturn(List.of(svcLock1, svcLock2)).anyTimes();
-    EasyMock.expect(zc.get(
+    expect(zc.get(
         EasyMock.eq(ROOT + ZSSERVERS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME + "/" + svcLock1),
         EasyMock.isA(ZcStat.class))).andReturn(sld1.serialize()).anyTimes();
-    EasyMock.expect(zc.get(
+    expect(zc.get(
         EasyMock.eq(
             ROOT + ZSSERVERS + "/" + DEFAULT_RESOURCE_GROUP_NAME + "/" + 
HOSTNAME + "/" + svcLock1),
         EasyMock.isA(ZcStat.class))).andReturn(sld2.serialize()).anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock
-        .expect(zc.getChildren(ROOT + ZSSERVERS + "/" + TEST_RESOURCE_GROUP + 
"/localhost:1234"))
+
+    expect(zc.getChildren(ROOT + ZSSERVERS + "/" + TEST_RESOURCE_GROUP + 
"/localhost:1234"))
         .andReturn(null).anyTimes();
-    EasyMock.replay(ctx, zc);
+    replay(zc);
 
     // query for all
-    Set<ServiceLockPath> results =
-        ctx.getServerPaths().getScanServer(rg -> true, AddressSelector.all(), 
false);
+    Set<ServiceLockPath> results = paths.getScanServer(rg -> true, 
AddressSelector.all(), false);
     assertEquals(4, results.size());
     for (ServiceLockPath path : results) {
       assertEquals(ZSSERVERS, path.getType());
@@ -631,10 +482,10 @@ public class ServiceLockPathsTest {
     }
 
     // query for all with lock
-    results = ctx.getServerPaths().getScanServer(rg -> true, 
AddressSelector.all(), true);
+    results = paths.getScanServer(rg -> true, AddressSelector.all(), true);
     assertEquals(2, results.size());
     Iterator<ServiceLockPath> iter = results.iterator();
-    ServiceLockPath slp1 = iter.next();
+    var slp1 = iter.next();
     assertEquals(HOSTNAME, slp1.getServer());
     assertEquals(ZSSERVERS, slp1.getType());
     if (slp1.getResourceGroup().equals(TEST_RESOURCE_GROUP)) {
@@ -645,7 +496,7 @@ public class ServiceLockPathsTest {
     } else {
       fail("wrong resource group");
     }
-    ServiceLockPath slp2 = iter.next();
+    var slp2 = iter.next();
     assertEquals(HOSTNAME, slp2.getServer());
     assertEquals(ZSSERVERS, slp2.getType());
     if (slp2.getResourceGroup().equals(TEST_RESOURCE_GROUP)) {
@@ -658,13 +509,13 @@ public class ServiceLockPathsTest {
     }
 
     // query for all in non-existent resource group
-    results = ctx.getServerPaths().getScanServer(rg -> 
rg.equals("FAKE_RESOURCE_GROUP"),
-        AddressSelector.all(), true);
+    results =
+        paths.getScanServer(rg -> rg.equals("FAKE_RESOURCE_GROUP"), 
AddressSelector.all(), true);
     assertEquals(0, results.size());
 
     // query for all in test resource group
-    results = ctx.getServerPaths().getScanServer(rg -> 
rg.equals(TEST_RESOURCE_GROUP),
-        AddressSelector.all(), true);
+    results =
+        paths.getScanServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.all(), true);
     assertEquals(1, results.size());
     iter = results.iterator();
     slp1 = iter.next();
@@ -674,8 +525,8 @@ public class ServiceLockPathsTest {
     assertEquals(ROOT + ZSSERVERS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME, slp1.toString());
 
     // query for a specific server
-    results = ctx.getServerPaths().getScanServer(rg -> 
rg.equals(TEST_RESOURCE_GROUP),
-        AddressSelector.exact(hp), true);
+    results =
+        paths.getScanServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.exact(hp), true);
     assertEquals(1, results.size());
     iter = results.iterator();
     slp1 = iter.next();
@@ -685,95 +536,66 @@ public class ServiceLockPathsTest {
     assertEquals(ROOT + ZSSERVERS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME, slp1.toString());
 
     // query for a wrong server
-    results = ctx.getServerPaths().getScanServer(rg -> 
rg.equals(TEST_RESOURCE_GROUP),
+    results = paths.getScanServer(rg -> rg.equals(TEST_RESOURCE_GROUP),
         AddressSelector.exact(HostAndPort.fromString("localhost:1234")), true);
     assertEquals(0, results.size());
-
-    EasyMock.verify(ctx, zc);
-
   }
 
   @Test
   public void testGetTabletServersNotRunning() {
+    expect(zc.getChildren(ROOT + ZTSERVERS)).andReturn(List.of()).anyTimes();
+    replay(zc);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + 
ZTSERVERS)).andReturn(List.of()).anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx, zc);
-
+    assertThrows(NullPointerException.class, () -> paths.getTabletServer(null, 
null, true));
     assertThrows(NullPointerException.class,
-        () -> ctx.getServerPaths().getTabletServer(null, null, true));
-    assertThrows(NullPointerException.class, () -> ctx.getServerPaths()
-        .getTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP), null, true));
+        () -> paths.getTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
null, true));
+    assertTrue(paths.getTabletServer(rg -> true, AddressSelector.all(), 
true).isEmpty());
     assertTrue(
-        ctx.getServerPaths().getTabletServer(rg -> true, 
AddressSelector.all(), true).isEmpty());
-    assertTrue(ctx.getServerPaths()
-        .getTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.all(), true)
-        .isEmpty());
-    assertTrue(ctx.getServerPaths()
-        .getTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.exact(hp), true)
-        .isEmpty());
-
-    EasyMock.verify(ctx, zc);
-
+        paths.getTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.all(), true)
+            .isEmpty());
+    assertTrue(
+        paths.getTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.exact(hp), true)
+            .isEmpty());
   }
 
   @Test
   public void testGetTabletServers() {
-
     UUID uuid = UUID.randomUUID();
     String svcLock1 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + 
"#0000000001";
     String svcLock2 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + 
"#0000000002";
-    ServiceLockData sld1 =
-        new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, 
TEST_RESOURCE_GROUP);
-    ServiceLockData sld2 =
+    var sld1 = new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, 
TEST_RESOURCE_GROUP);
+    var sld2 =
         new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, 
DEFAULT_RESOURCE_GROUP_NAME);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZTSERVERS))
+    expect(zc.getChildren(ROOT + ZTSERVERS))
         .andReturn(List.of(TEST_RESOURCE_GROUP, 
DEFAULT_RESOURCE_GROUP_NAME)).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZTSERVERS + "/" + 
TEST_RESOURCE_GROUP))
+    expect(zc.getChildren(ROOT + ZTSERVERS + "/" + TEST_RESOURCE_GROUP))
         .andReturn(List.of(HOSTNAME, HOSTNAME_NO_LOCK)).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZTSERVERS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME))
+    expect(zc.getChildren(ROOT + ZTSERVERS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME))
         .andReturn(List.of(HOSTNAME, HOSTNAME_NO_LOCK)).anyTimes();
-    EasyMock
-        .expect(
-            zc.getChildren(ROOT + ZTSERVERS + "/" + TEST_RESOURCE_GROUP + "/" 
+ HOSTNAME_NO_LOCK))
+    expect(zc.getChildren(ROOT + ZTSERVERS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME_NO_LOCK))
         .andReturn(List.of()).anyTimes();
-    EasyMock
-        .expect(zc.getChildren(
-            ROOT + ZTSERVERS + "/" + DEFAULT_RESOURCE_GROUP_NAME + "/" + 
HOSTNAME_NO_LOCK))
+    expect(zc
+        .getChildren(ROOT + ZTSERVERS + "/" + DEFAULT_RESOURCE_GROUP_NAME + 
"/" + HOSTNAME_NO_LOCK))
         .andReturn(List.of()).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZTSERVERS + "/" + 
TEST_RESOURCE_GROUP + "/" + HOSTNAME))
+    expect(zc.getChildren(ROOT + ZTSERVERS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME))
         .andReturn(List.of(svcLock1, svcLock2)).anyTimes();
-    EasyMock
-        .expect(
-            zc.getChildren(ROOT + ZTSERVERS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME + "/" + HOSTNAME))
+    expect(zc.getChildren(ROOT + ZTSERVERS + "/" + DEFAULT_RESOURCE_GROUP_NAME 
+ "/" + HOSTNAME))
         .andReturn(List.of(svcLock1, svcLock2)).anyTimes();
-    EasyMock.expect(zc.get(
+    expect(zc.get(
         EasyMock.eq(ROOT + ZTSERVERS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME + "/" + svcLock1),
         EasyMock.isA(ZcStat.class))).andReturn(sld1.serialize()).anyTimes();
-    EasyMock.expect(zc.get(
+    expect(zc.get(
         EasyMock.eq(
             ROOT + ZTSERVERS + "/" + DEFAULT_RESOURCE_GROUP_NAME + "/" + 
HOSTNAME + "/" + svcLock1),
         EasyMock.isA(ZcStat.class))).andReturn(sld2.serialize()).anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock
-        .expect(zc.getChildren(ROOT + ZTSERVERS + "/" + TEST_RESOURCE_GROUP + 
"/localhost:1234"))
+
+    expect(zc.getChildren(ROOT + ZTSERVERS + "/" + TEST_RESOURCE_GROUP + 
"/localhost:1234"))
         .andReturn(null).anyTimes();
-    EasyMock.replay(ctx, zc);
+    replay(zc);
 
     // query for all
-    Set<ServiceLockPath> results =
-        ctx.getServerPaths().getTabletServer(rg -> true, 
AddressSelector.all(), false);
+    Set<ServiceLockPath> results = paths.getTabletServer(rg -> true, 
AddressSelector.all(), false);
     assertEquals(4, results.size());
     for (ServiceLockPath path : results) {
       assertEquals(ZTSERVERS, path.getType());
@@ -790,10 +612,10 @@ public class ServiceLockPathsTest {
     }
 
     // query for all with lock
-    results = ctx.getServerPaths().getTabletServer(rg -> true, 
AddressSelector.all(), true);
+    results = paths.getTabletServer(rg -> true, AddressSelector.all(), true);
     assertEquals(2, results.size());
     Iterator<ServiceLockPath> iter = results.iterator();
-    ServiceLockPath slp1 = iter.next();
+    var slp1 = iter.next();
     assertEquals(HOSTNAME, slp1.getServer());
     assertEquals(ZTSERVERS, slp1.getType());
     if (slp1.getResourceGroup().equals(TEST_RESOURCE_GROUP)) {
@@ -804,7 +626,7 @@ public class ServiceLockPathsTest {
     } else {
       fail("wrong resource group");
     }
-    ServiceLockPath slp2 = iter.next();
+    var slp2 = iter.next();
     assertEquals(HOSTNAME, slp2.getServer());
     assertEquals(ZTSERVERS, slp2.getType());
     if (slp2.getResourceGroup().equals(TEST_RESOURCE_GROUP)) {
@@ -817,13 +639,13 @@ public class ServiceLockPathsTest {
     }
 
     // query for all in non-existent resource group
-    results = ctx.getServerPaths().getTabletServer(rg -> 
rg.equals("FAKE_RESOURCE_GROUP"),
-        AddressSelector.all(), true);
+    results =
+        paths.getTabletServer(rg -> rg.equals("FAKE_RESOURCE_GROUP"), 
AddressSelector.all(), true);
     assertEquals(0, results.size());
 
     // query for all in test resource group
-    results = ctx.getServerPaths().getTabletServer(rg -> 
rg.equals(TEST_RESOURCE_GROUP),
-        AddressSelector.all(), true);
+    results =
+        paths.getTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.all(), true);
     assertEquals(1, results.size());
     iter = results.iterator();
     slp1 = iter.next();
@@ -833,8 +655,8 @@ public class ServiceLockPathsTest {
     assertEquals(ROOT + ZTSERVERS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME, slp1.toString());
 
     // query for a specific server
-    results = ctx.getServerPaths().getTabletServer(rg -> 
rg.equals(TEST_RESOURCE_GROUP),
-        AddressSelector.exact(hp), true);
+    results = paths.getTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.exact(hp),
+        true);
     assertEquals(1, results.size());
     iter = results.iterator();
     slp1 = iter.next();
@@ -844,93 +666,68 @@ public class ServiceLockPathsTest {
     assertEquals(ROOT + ZTSERVERS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME, slp1.toString());
 
     // query for a wrong server
-    results = ctx.getServerPaths().getTabletServer(rg -> 
rg.equals(TEST_RESOURCE_GROUP),
+    results = paths.getTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP),
         AddressSelector.exact(HostAndPort.fromString("localhost:1234")), true);
     assertEquals(0, results.size());
-
-    EasyMock.verify(ctx, zc);
-
   }
 
   @Test
   public void testGetDeadTabletServersNone() {
+    expect(zc.getChildren(ROOT + 
ZDEADTSERVERS)).andReturn(List.of()).anyTimes();
+    replay(zc);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + 
ZDEADTSERVERS)).andReturn(List.of()).anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx, zc);
-
+    assertThrows(NullPointerException.class, () -> 
paths.getDeadTabletServer(null, null, false));
     assertThrows(NullPointerException.class,
-        () -> ctx.getServerPaths().getDeadTabletServer(null, null, false));
-    assertThrows(NullPointerException.class, () -> ctx.getServerPaths()
-        .getDeadTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP), null, 
false));
-    assertTrue(ctx.getServerPaths().getDeadTabletServer(rg -> true, 
AddressSelector.all(), false)
-        .isEmpty());
-    assertTrue(ctx.getServerPaths()
+        () -> paths.getDeadTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
null, false));
+    assertTrue(paths.getDeadTabletServer(rg -> true, AddressSelector.all(), 
false).isEmpty());
+    assertTrue(paths
         .getDeadTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.all(), false)
         .isEmpty());
-    assertTrue(ctx.getServerPaths()
+    assertTrue(paths
         .getDeadTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.exact(hp), false)
         .isEmpty());
-
-    EasyMock.verify(ctx, zc);
-
   }
 
   @Test
   public void testGetDeadTabletServers() {
-
     UUID uuid = UUID.randomUUID();
     String svcLock1 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + 
"#0000000001";
     String svcLock2 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + 
"#0000000002";
-    ServiceLockData sld1 =
-        new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, 
TEST_RESOURCE_GROUP);
-    ServiceLockData sld2 =
+    var sld1 = new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, 
TEST_RESOURCE_GROUP);
+    var sld2 =
         new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, 
DEFAULT_RESOURCE_GROUP_NAME);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    ZooCache zc = EasyMock.createMock(ZooCache.class);
-
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getZooCache()).andReturn(zc).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZDEADTSERVERS))
+    expect(zc.getChildren(ROOT + ZDEADTSERVERS))
         .andReturn(List.of(TEST_RESOURCE_GROUP, 
DEFAULT_RESOURCE_GROUP_NAME)).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZDEADTSERVERS + "/" + 
TEST_RESOURCE_GROUP))
+    expect(zc.getChildren(ROOT + ZDEADTSERVERS + "/" + TEST_RESOURCE_GROUP))
         .andReturn(List.of(HOSTNAME)).anyTimes();
-    EasyMock.expect(zc.getChildren(ROOT + ZDEADTSERVERS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME))
+    expect(zc.getChildren(ROOT + ZDEADTSERVERS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME))
         .andReturn(List.of(HOSTNAME)).anyTimes();
-    EasyMock
-        .expect(zc.getChildren(ROOT + ZDEADTSERVERS + "/" + 
TEST_RESOURCE_GROUP + "/" + HOSTNAME))
+    expect(zc.getChildren(ROOT + ZDEADTSERVERS + "/" + TEST_RESOURCE_GROUP + 
"/" + HOSTNAME))
         .andReturn(List.of(svcLock1, svcLock2)).anyTimes();
-    EasyMock.expect(zc.get(ROOT + ZDEADTSERVERS + "/" + TEST_RESOURCE_GROUP + 
"/" + HOSTNAME))
+    expect(zc.get(ROOT + ZDEADTSERVERS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME))
         .andReturn(new byte[0]).anyTimes();
-    EasyMock
-        .expect(zc
-            .getChildren(ROOT + ZDEADTSERVERS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME + "/" + HOSTNAME))
+    expect(
+        zc.getChildren(ROOT + ZDEADTSERVERS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME + "/" + HOSTNAME))
         .andReturn(List.of(svcLock1, svcLock2)).anyTimes();
-    EasyMock.expect(zc.get(
+    expect(zc.get(
         EasyMock
             .eq(ROOT + ZDEADTSERVERS + "/" + TEST_RESOURCE_GROUP + "/" + 
HOSTNAME + "/" + svcLock1),
         EasyMock.isA(ZcStat.class))).andReturn(sld1.serialize()).anyTimes();
-    EasyMock
-        .expect(zc.get(EasyMock.eq(ROOT + ZDEADTSERVERS + "/" + 
DEFAULT_RESOURCE_GROUP_NAME + "/"
-            + HOSTNAME + "/" + svcLock1), EasyMock.isA(ZcStat.class)))
-        .andReturn(sld2.serialize()).anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.expect(zc.get(ROOT + ZDEADTSERVERS + "/" + TEST_RESOURCE_GROUP + 
"/localhost:1234"))
+    expect(zc.get(EasyMock.eq(
+        ROOT + ZDEADTSERVERS + "/" + DEFAULT_RESOURCE_GROUP_NAME + "/" + 
HOSTNAME + "/" + svcLock1),
+        EasyMock.isA(ZcStat.class))).andReturn(sld2.serialize()).anyTimes();
+
+    expect(zc.get(ROOT + ZDEADTSERVERS + "/" + TEST_RESOURCE_GROUP + 
"/localhost:1234"))
         .andReturn(null).anyTimes();
-    EasyMock.replay(ctx, zc);
+    replay(zc);
 
     // query for all
     Set<ServiceLockPath> results =
-        ctx.getServerPaths().getDeadTabletServer(rg -> true, 
AddressSelector.all(), false);
+        paths.getDeadTabletServer(rg -> true, AddressSelector.all(), false);
     assertEquals(2, results.size());
     Iterator<ServiceLockPath> iter = results.iterator();
-    ServiceLockPath slp1 = iter.next();
+    var slp1 = iter.next();
     assertEquals(HOSTNAME, slp1.getServer());
     assertEquals(ZDEADTSERVERS, slp1.getType());
     if (slp1.getResourceGroup().equals(TEST_RESOURCE_GROUP)) {
@@ -942,7 +739,7 @@ public class ServiceLockPathsTest {
     } else {
       fail("wrong resource group");
     }
-    ServiceLockPath slp2 = iter.next();
+    var slp2 = iter.next();
     assertEquals(HOSTNAME, slp2.getServer());
     assertEquals(ZDEADTSERVERS, slp2.getType());
     if (slp2.getResourceGroup().equals(TEST_RESOURCE_GROUP)) {
@@ -956,13 +753,13 @@ public class ServiceLockPathsTest {
     }
 
     // query for all in non-existent resource group
-    results = ctx.getServerPaths().getDeadTabletServer(rg -> 
rg.equals("FAKE_RESOURCE_GROUP"),
+    results = paths.getDeadTabletServer(rg -> rg.equals("FAKE_RESOURCE_GROUP"),
         AddressSelector.all(), false);
     assertEquals(0, results.size());
 
     // query for all in test resource group
-    results = ctx.getServerPaths().getDeadTabletServer(rg -> 
rg.equals(TEST_RESOURCE_GROUP),
-        AddressSelector.all(), false);
+    results = paths.getDeadTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP), 
AddressSelector.all(),
+        false);
     assertEquals(1, results.size());
     iter = results.iterator();
     slp1 = iter.next();
@@ -973,7 +770,7 @@ public class ServiceLockPathsTest {
         slp1.toString());
 
     // query for a specific server
-    results = ctx.getServerPaths().getDeadTabletServer(rg -> 
rg.equals(TEST_RESOURCE_GROUP),
+    results = paths.getDeadTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP),
         AddressSelector.exact(hp), false);
     assertEquals(1, results.size());
     iter = results.iterator();
@@ -985,31 +782,24 @@ public class ServiceLockPathsTest {
         slp1.toString());
 
     // query for a wrong server
-    results = ctx.getServerPaths().getDeadTabletServer(rg -> 
rg.equals(TEST_RESOURCE_GROUP),
+    results = paths.getDeadTabletServer(rg -> rg.equals(TEST_RESOURCE_GROUP),
         AddressSelector.exact(HostAndPort.fromString("localhost:1234")), 
false);
     assertEquals(0, results.size());
-
-    EasyMock.verify(ctx, zc);
-
   }
 
   @Test
   public void testTableLocksPath() {
-
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx);
+    replay(zc);
 
     // Only table lock creation is supported because the existing code
     // uses a ServiceLockPath with it.
-    ServiceLockPath slp = ctx.getServerPaths().createTableLocksPath();
+    var slp = paths.createTableLocksPath();
     assertNull(slp.getServer());
     assertNull(slp.getResourceGroup());
     assertEquals(ZTABLE_LOCKS, slp.getType());
     assertEquals(ROOT + ZTABLE_LOCKS, slp.toString());
 
-    slp = ctx.getServerPaths().createTableLocksPath("1");
+    slp = paths.createTableLocksPath("1");
     assertEquals("1", slp.getServer());
     assertNull(slp.getResourceGroup());
     assertEquals(ZTABLE_LOCKS, slp.getType());
@@ -1022,25 +812,18 @@ public class ServiceLockPathsTest {
         () -> ServiceLockPaths.parse(Optional.of(ZTABLE_LOCKS), ROOT + 
ZTABLE_LOCKS));
     assertThrows(IllegalArgumentException.class,
         () -> ServiceLockPaths.parse(Optional.of(ZTABLE_LOCKS), ROOT + 
ZTABLE_LOCKS + "/1"));
-
-    EasyMock.verify(ctx);
-
   }
 
   @Test
   public void testMiniPath() {
+    replay(zc);
 
-    ClientContext ctx = EasyMock.createMock(ClientContext.class);
-    EasyMock.expect(ctx.getZooKeeperRoot()).andReturn(ROOT).anyTimes();
-    EasyMock.expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
-    EasyMock.replay(ctx);
-
-    assertThrows(NullPointerException.class, () -> 
ctx.getServerPaths().createMiniPath(null));
+    assertThrows(NullPointerException.class, () -> paths.createMiniPath(null));
 
     // Only mini lock creation is supported because the existing code
     // uses a ServiceLockPath with it.
     String miniUUID = UUID.randomUUID().toString();
-    ServiceLockPath slp = ctx.getServerPaths().createMiniPath(miniUUID);
+    var slp = paths.createMiniPath(miniUUID);
     assertEquals(miniUUID, slp.getServer());
     assertNull(slp.getResourceGroup());
     assertEquals(ZMINI_LOCK, slp.getType());
@@ -1056,9 +839,5 @@ public class ServiceLockPathsTest {
     assertNull(slp.getResourceGroup());
     assertEquals(ZMINI_LOCK, slp.getType());
     assertEquals(ROOT + ZMINI_LOCK + "/" + miniUUID, slp.toString());
-
-    EasyMock.verify(ctx);
-
   }
-
 }
diff --git 
a/server/base/src/test/java/org/apache/accumulo/server/util/AdminTest.java 
b/server/base/src/test/java/org/apache/accumulo/server/util/AdminTest.java
index 694541105a..3e77560e49 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/AdminTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/AdminTest.java
@@ -101,8 +101,6 @@ public class AdminTest {
 
     String serverPath = group + "/" + server;
     String validZLockEphemeralNode = "zlock#" + UUID.randomUUID() + 
"#0000000000";
-    expect(ctx.getZooKeeperRoot()).andReturn(root).anyTimes();
-    expect(ctx.getZooCache()).andReturn(zc).anyTimes();
     
expect(zc.getChildren(type)).andReturn(List.of(Constants.DEFAULT_RESOURCE_GROUP_NAME))
         .anyTimes();
     expect(zc.getChildren(group)).andReturn(List.of(server)).anyTimes();
@@ -116,7 +114,7 @@ public class AdminTest {
           stat.setEphemeralOwner(session);
           return new byte[0];
         });
-    expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
+    expect(ctx.getServerPaths()).andReturn(new ServiceLockPaths(root, 
zc)).anyTimes();
     replay(ctx, zc);
 
     assertEquals(server + "[" + Long.toHexString(session) + "]",
@@ -137,11 +135,9 @@ public class AdminTest {
     String server = "localhost:12345";
 
     String serverPath = group + "/" + server;
-    expect(ctx.getZooKeeperRoot()).andReturn(root).anyTimes();
-    expect(ctx.getZooCache()).andReturn(zc).anyTimes();
     
expect(zc.getChildren(type)).andReturn(List.of(Constants.DEFAULT_RESOURCE_GROUP_NAME));
     expect(zc.getChildren(serverPath)).andReturn(Collections.emptyList());
-    expect(ctx.getServerPaths()).andReturn(new 
ServiceLockPaths(ctx)).anyTimes();
+    expect(ctx.getServerPaths()).andReturn(new ServiceLockPaths(root, 
zc)).anyTimes();
     replay(ctx, zc);
 
     // A server that isn't in ZooKeeper. Can't qualify it, should return the 
original
diff --git 
a/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java
 
b/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java
index 41c5779ada..8236f28640 100644
--- 
a/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java
+++ 
b/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java
@@ -73,7 +73,7 @@ public class ServiceStatusCmdTest {
     expect(context.getZooCache()).andReturn(zooCache).anyTimes();
     expect(context.getZooSession()).andReturn(zooReader).anyTimes();
     expect(context.getZooKeeperRoot()).andReturn(zRoot).anyTimes();
-    expect(context.getServerPaths()).andReturn(new 
ServiceLockPaths(context)).anyTimes();
+    expect(context.getServerPaths()).andReturn(new ServiceLockPaths(zRoot, 
zooCache)).anyTimes();
     replay(context);
   }
 
diff --git 
a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java 
b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java
index 0384bc97b1..b1fce72f83 100644
--- a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java
@@ -61,10 +61,10 @@ import org.apache.accumulo.core.fate.FateStore;
 import org.apache.accumulo.core.fate.ReadOnlyFateStore;
 import org.apache.accumulo.core.fate.user.UserFateStore;
 import org.apache.accumulo.core.fate.zookeeper.MetaFateStore;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.lock.ServiceLockPaths.AddressSelector;
+import org.apache.accumulo.core.zookeeper.ZooSession;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl.ProcessInfo;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -643,8 +643,7 @@ public abstract class FateOpsCommandsIT extends 
ConfigurableMacBase
     } else {
       Method listMethod = MetaFateStore.class.getMethod("list");
       mockedStore = EasyMock.createMockBuilder(MetaFateStore.class)
-          .withConstructor(String.class, ZooReaderWriter.class, 
ZooUtil.LockID.class,
-              Predicate.class)
+          .withConstructor(String.class, ZooSession.class, 
ZooUtil.LockID.class, Predicate.class)
           .withArgs(sctx.getZooKeeperRoot() + Constants.ZFATE, 
sctx.getZooSession(),
               createDummyLockID(), null)
           .addMockedMethod(listMethod).createMock();
diff --git 
a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java
 
b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java
index 5ed4913f8b..13e9d77465 100644
--- 
a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java
+++ 
b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java
@@ -83,6 +83,7 @@ import org.apache.accumulo.harness.SharedMiniClusterBase;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.test.util.Wait;
 import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
@@ -92,6 +93,8 @@ import com.google.common.net.HostAndPort;
 
 public class ManagerAssignmentIT extends SharedMiniClusterBase {
 
+  private static AccumuloClient client;
+
   @BeforeAll
   public static void beforeAll() throws Exception {
     SharedMiniClusterBase.startMiniClusterWithConfig((cfg, core) -> {
@@ -102,119 +105,121 @@ public class ManagerAssignmentIT extends 
SharedMiniClusterBase {
       cfg.setProperty(Property.TSERV_ONDEMAND_UNLOADER_INTERVAL, "10s");
       cfg.setProperty(DefaultOnDemandTabletUnloader.INACTIVITY_THRESHOLD, 
"15");
     });
+    client = Accumulo.newClient().from(getClientProps()).build();
+  }
+
+  @AfterAll
+  public static void afterAll() {
+    client.close();
   }
 
   @BeforeEach
   public void before() throws Exception {
-    try (AccumuloClient client = 
Accumulo.newClient().from(getClientProps()).build()) {
-      Wait.waitFor(() -> countTabletsWithLocation(client, 
AccumuloTable.ROOT.tableId()) > 0);
-      Wait.waitFor(() -> countTabletsWithLocation(client, 
AccumuloTable.METADATA.tableId()) > 0);
-    }
+    Wait.waitFor(() -> countTabletsWithLocation(client, 
AccumuloTable.ROOT.tableId()) > 0);
+    Wait.waitFor(() -> countTabletsWithLocation(client, 
AccumuloTable.METADATA.tableId()) > 0);
   }
 
   @Test
   public void test() throws Exception {
-    try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {
-
-      // Confirm that the root and metadata tables are hosted
-      Locations rootLocations = 
c.tableOperations().locate(AccumuloTable.ROOT.tableName(),
-          Collections.singletonList(new Range()));
-      rootLocations.groupByTablet().keySet()
-          .forEach(tid -> assertNotNull(rootLocations.getTabletLocation(tid)));
-
-      Locations metadataLocations = 
c.tableOperations().locate(AccumuloTable.METADATA.tableName(),
-          Collections.singletonList(new Range()));
-      metadataLocations.groupByTablet().keySet()
-          .forEach(tid -> 
assertNotNull(metadataLocations.getTabletLocation(tid)));
-
-      String tableName = super.getUniqueNames(1)[0];
-      c.tableOperations().create(tableName);
-
-      String tableId = c.tableOperations().tableIdMap().get(tableName);
-
-      // wait for the tablet to exist in the metadata table. The tablet
-      // will not be hosted so the current location will be empty.
-      Wait.waitFor(() -> getTabletMetadata(c, tableId, null) != null, 10000, 
250);
-      TabletMetadata newTablet = getTabletMetadata(c, tableId, null);
-      assertNotNull(newTablet.getExtent());
-      assertFalse(newTablet.hasCurrent());
-      assertNull(newTablet.getLast());
-      assertNull(newTablet.getLocation());
-      assertEquals(TabletAvailability.ONDEMAND, 
newTablet.getTabletAvailability());
-
-      // calling the batch writer will cause the tablet to be hosted
-      try (BatchWriter bw = c.createBatchWriter(tableName)) {
-        Mutation m = new Mutation("a");
-        m.put("b", "c", "d");
-        bw.addMutation(m);
-      }
-      // give it a last location
-      c.tableOperations().flush(tableName, null, null, true);
-
-      TabletMetadata flushed = getTabletMetadata(c, tableId, null);
-      assertTrue(flushed.hasCurrent());
-      assertNotNull(flushed.getLocation());
-      assertEquals(flushed.getLocation().getHostPort(), 
flushed.getLast().getHostPort());
-      assertFalse(flushed.getLocation().getType().equals(LocationType.FUTURE));
-      assertEquals(TabletAvailability.ONDEMAND, 
flushed.getTabletAvailability());
-
-      // take the tablet offline
-      c.tableOperations().offline(tableName, true);
-      TabletMetadata offline = getTabletMetadata(c, tableId, null);
-      assertFalse(offline.hasCurrent());
-      assertNull(offline.getLocation());
-      assertEquals(flushed.getLocation().getHostPort(), 
offline.getLast().getHostPort());
-      assertEquals(TabletAvailability.ONDEMAND, 
offline.getTabletAvailability());
-
-      // put it back online
-      c.tableOperations().online(tableName, true);
-      TabletMetadata online = getTabletMetadata(c, tableId, null);
-      assertTrue(online.hasCurrent());
-      assertNotNull(online.getLocation());
-      assertEquals(online.getLocation().getHostPort(), 
online.getLast().getHostPort());
-      assertEquals(TabletAvailability.ONDEMAND, 
online.getTabletAvailability());
-
-      // set the tablet availability to HOSTED
-      c.tableOperations().setTabletAvailability(tableName, new Range(), 
TabletAvailability.HOSTED);
-
-      Predicate<TabletMetadata> hostedOrCurrentNotNull =
-          t -> (t.getTabletAvailability() == TabletAvailability.HOSTED && 
t.hasCurrent());
-
-      Wait.waitFor(() -> hostedOrCurrentNotNull.test(getTabletMetadata(c, 
tableId, null)), 60000,
-          250);
-
-      final TabletMetadata always = getTabletMetadata(c, tableId, null);
-      assertTrue(hostedOrCurrentNotNull.test(always));
-      assertTrue(always.hasCurrent());
-      assertEquals(flushed.getLocation().getHostPort(), 
always.getLast().getHostPort());
-      assertEquals(TabletAvailability.HOSTED, always.getTabletAvailability());
-
-      // set the hosting availability to never
-      c.tableOperations().setTabletAvailability(tableName, new Range(),
-          TabletAvailability.UNHOSTED);
-      Predicate<TabletMetadata> unhostedOrCurrentNull =
-          t -> (t.getTabletAvailability() == TabletAvailability.UNHOSTED && 
!t.hasCurrent());
-      Wait.waitFor(() -> unhostedOrCurrentNull.test(getTabletMetadata(c, 
tableId, null)), 60000,
-          250);
-
-      final TabletMetadata unhosted = getTabletMetadata(c, tableId, null);
-      assertTrue(unhostedOrCurrentNull.test(unhosted));
-      assertNull(unhosted.getLocation());
-      assertEquals(flushed.getLocation().getHostPort(), 
unhosted.getLast().getHostPort());
-      assertEquals(TabletAvailability.UNHOSTED, 
unhosted.getTabletAvailability());
-
-      // set the tablet availability to ONDEMAND
-      c.tableOperations().setTabletAvailability(tableName, new Range(),
-          TabletAvailability.ONDEMAND);
-      Predicate<TabletMetadata> ondemandHosted =
-          t -> t.getTabletAvailability() == TabletAvailability.ONDEMAND;
-      Wait.waitFor(() -> ondemandHosted.test(getTabletMetadata(c, tableId, 
null)), 60000, 250);
-      final TabletMetadata ondemand = getTabletMetadata(c, tableId, null);
-      assertTrue(ondemandHosted.test(ondemand));
-      assertNull(ondemand.getLocation());
-      assertEquals(flushed.getLocation().getHostPort(), 
ondemand.getLast().getHostPort());
-      assertEquals(TabletAvailability.ONDEMAND, 
ondemand.getTabletAvailability());
+    // Confirm that the root and metadata tables are hosted
+    Locations rootLocations = 
client.tableOperations().locate(AccumuloTable.ROOT.tableName(),
+        Collections.singletonList(new Range()));
+    rootLocations.groupByTablet().keySet()
+        .forEach(tid -> assertNotNull(rootLocations.getTabletLocation(tid)));
+
+    Locations metadataLocations = client.tableOperations()
+        .locate(AccumuloTable.METADATA.tableName(), 
Collections.singletonList(new Range()));
+    metadataLocations.groupByTablet().keySet()
+        .forEach(tid -> 
assertNotNull(metadataLocations.getTabletLocation(tid)));
+
+    String tableName = super.getUniqueNames(1)[0];
+    client.tableOperations().create(tableName);
+
+    String tableId = client.tableOperations().tableIdMap().get(tableName);
+
+    // wait for the tablet to exist in the metadata table. The tablet
+    // will not be hosted so the current location will be empty.
+    Wait.waitFor(() -> getTabletMetadata(client, tableId, null) != null, 
10000, 250);
+    TabletMetadata newTablet = getTabletMetadata(client, tableId, null);
+    assertNotNull(newTablet.getExtent());
+    assertFalse(newTablet.hasCurrent());
+    assertNull(newTablet.getLast());
+    assertNull(newTablet.getLocation());
+    assertEquals(TabletAvailability.ONDEMAND, 
newTablet.getTabletAvailability());
+
+    // calling the batch writer will cause the tablet to be hosted
+    try (BatchWriter bw = client.createBatchWriter(tableName)) {
+      Mutation m = new Mutation("a");
+      m.put("b", "c", "d");
+      bw.addMutation(m);
     }
+    // give it a last location
+    client.tableOperations().flush(tableName, null, null, true);
+
+    TabletMetadata flushed = getTabletMetadata(client, tableId, null);
+    assertTrue(flushed.hasCurrent());
+    assertNotNull(flushed.getLocation());
+    assertEquals(flushed.getLocation().getHostPort(), 
flushed.getLast().getHostPort());
+    assertFalse(flushed.getLocation().getType().equals(LocationType.FUTURE));
+    assertEquals(TabletAvailability.ONDEMAND, flushed.getTabletAvailability());
+
+    // take the tablet offline
+    client.tableOperations().offline(tableName, true);
+    TabletMetadata offline = getTabletMetadata(client, tableId, null);
+    assertFalse(offline.hasCurrent());
+    assertNull(offline.getLocation());
+    assertEquals(flushed.getLocation().getHostPort(), 
offline.getLast().getHostPort());
+    assertEquals(TabletAvailability.ONDEMAND, offline.getTabletAvailability());
+
+    // put it back online
+    client.tableOperations().online(tableName, true);
+    TabletMetadata online = getTabletMetadata(client, tableId, null);
+    assertTrue(online.hasCurrent());
+    assertNotNull(online.getLocation());
+    assertEquals(online.getLocation().getHostPort(), 
online.getLast().getHostPort());
+    assertEquals(TabletAvailability.ONDEMAND, online.getTabletAvailability());
+
+    // set the tablet availability to HOSTED
+    client.tableOperations().setTabletAvailability(tableName, new Range(),
+        TabletAvailability.HOSTED);
+
+    Predicate<TabletMetadata> hostedOrCurrentNotNull =
+        t -> (t.getTabletAvailability() == TabletAvailability.HOSTED && 
t.hasCurrent());
+
+    Wait.waitFor(() -> hostedOrCurrentNotNull.test(getTabletMetadata(client, 
tableId, null)), 60000,
+        250);
+
+    final TabletMetadata always = getTabletMetadata(client, tableId, null);
+    assertTrue(hostedOrCurrentNotNull.test(always));
+    assertTrue(always.hasCurrent());
+    assertEquals(flushed.getLocation().getHostPort(), 
always.getLast().getHostPort());
+    assertEquals(TabletAvailability.HOSTED, always.getTabletAvailability());
+
+    // set the hosting availability to never
+    client.tableOperations().setTabletAvailability(tableName, new Range(),
+        TabletAvailability.UNHOSTED);
+    Predicate<TabletMetadata> unhostedOrCurrentNull =
+        t -> (t.getTabletAvailability() == TabletAvailability.UNHOSTED && 
!t.hasCurrent());
+    Wait.waitFor(() -> unhostedOrCurrentNull.test(getTabletMetadata(client, 
tableId, null)), 60000,
+        250);
+
+    final TabletMetadata unhosted = getTabletMetadata(client, tableId, null);
+    assertTrue(unhostedOrCurrentNull.test(unhosted));
+    assertNull(unhosted.getLocation());
+    assertEquals(flushed.getLocation().getHostPort(), 
unhosted.getLast().getHostPort());
+    assertEquals(TabletAvailability.UNHOSTED, 
unhosted.getTabletAvailability());
+
+    // set the tablet availability to ONDEMAND
+    client.tableOperations().setTabletAvailability(tableName, new Range(),
+        TabletAvailability.ONDEMAND);
+    Predicate<TabletMetadata> ondemandHosted =
+        t -> t.getTabletAvailability() == TabletAvailability.ONDEMAND;
+    Wait.waitFor(() -> ondemandHosted.test(getTabletMetadata(client, tableId, 
null)), 60000, 250);
+    final TabletMetadata ondemand = getTabletMetadata(client, tableId, null);
+    assertTrue(ondemandHosted.test(ondemand));
+    assertNull(ondemand.getLocation());
+    assertEquals(flushed.getLocation().getHostPort(), 
ondemand.getLast().getHostPort());
+    assertEquals(TabletAvailability.ONDEMAND, 
ondemand.getTabletAvailability());
   }
 
   private String prepTableForScanTest(AccumuloClient c, String tableName) 
throws Exception {
@@ -261,179 +266,167 @@ public class ManagerAssignmentIT extends 
SharedMiniClusterBase {
 
   @Test
   public void testScannerAssignsOneOnDemandTablets() throws Exception {
-    try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {
-      String tableName = super.getUniqueNames(1)[0];
+    String tableName = super.getUniqueNames(1)[0];
 
-      String tableId = prepTableForScanTest(c, tableName);
+    String tableId = prepTableForScanTest(client, tableName);
 
-      Range scanRange = new Range("a", "c");
-      Scanner s = c.createScanner(tableName);
-      s.setRange(scanRange);
-      // Should return keys for a, b, c
-      assertEquals(3, Iterables.size(s));
+    Range scanRange = new Range("a", "c");
+    Scanner s = client.createScanner(tableName);
+    s.setRange(scanRange);
+    // Should return keys for a, b, c
+    assertEquals(3, Iterables.size(s));
 
-      List<TabletStats> stats = getTabletStats(c, tableId);
-      // There should be one tablet online
-      assertEquals(1, stats.size());
-      assertTrue(ClientTabletCache.getInstance((ClientContext) c, 
TableId.of(tableId))
-          .getTabletHostingRequestCount() > 0);
+    List<TabletStats> stats = getTabletStats(client, tableId);
+    // There should be one tablet online
+    assertEquals(1, stats.size());
+    assertTrue(ClientTabletCache.getInstance((ClientContext) client, 
TableId.of(tableId))
+        .getTabletHostingRequestCount() > 0);
 
-    }
   }
 
   @Test
   public void testScannerAssignsMultipleOnDemandTablets() throws Exception {
-    try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {
-      String tableName = super.getUniqueNames(1)[0];
+    String tableName = super.getUniqueNames(1)[0];
 
-      String tableId = prepTableForScanTest(c, tableName);
+    String tableId = prepTableForScanTest(client, tableName);
 
-      try (Scanner s = c.createScanner(tableName)) {
-        s.setRange(new Range("a", "s"));
-        assertEquals(19, Iterables.size(s));
-      }
+    try (Scanner s = client.createScanner(tableName)) {
+      s.setRange(new Range("a", "s"));
+      assertEquals(19, Iterables.size(s));
+    }
 
-      List<TabletStats> stats = getTabletStats(c, tableId);
-      assertEquals(3, stats.size());
-      long hostingRequestCount = ClientTabletCache
-          .getInstance((ClientContext) c, 
TableId.of(tableId)).getTabletHostingRequestCount();
-      assertTrue(hostingRequestCount > 0);
-
-      // Run another scan, the t tablet should get loaded
-      // all others should be loaded.
-      try (Scanner s = c.createScanner(tableName)) {
-        s.setRange(new Range("a", "t"));
-        assertEquals(20, Iterables.size(s));
-      }
+    List<TabletStats> stats = getTabletStats(client, tableId);
+    assertEquals(3, stats.size());
+    long hostingRequestCount = ClientTabletCache
+        .getInstance((ClientContext) client, 
TableId.of(tableId)).getTabletHostingRequestCount();
+    assertTrue(hostingRequestCount > 0);
+
+    // Run another scan, the t tablet should get loaded
+    // all others should be loaded.
+    try (Scanner s = client.createScanner(tableName)) {
+      s.setRange(new Range("a", "t"));
+      assertEquals(20, Iterables.size(s));
+    }
 
-      stats = getTabletStats(c, tableId);
-      assertEquals(3, stats.size());
-      // No more tablets should have been brought online
-      assertEquals(hostingRequestCount, ClientTabletCache
-          .getInstance((ClientContext) c, 
TableId.of(tableId)).getTabletHostingRequestCount());
+    stats = getTabletStats(client, tableId);
+    assertEquals(3, stats.size());
+    // No more tablets should have been brought online
+    assertEquals(hostingRequestCount, ClientTabletCache
+        .getInstance((ClientContext) client, 
TableId.of(tableId)).getTabletHostingRequestCount());
 
-    }
   }
 
   @Test
   public void testBatchScannerAssignsOneOnDemandTablets() throws Exception {
-    try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {
-      String tableName = super.getUniqueNames(1)[0];
+    String tableName = super.getUniqueNames(1)[0];
 
-      String tableId = prepTableForScanTest(c, tableName);
+    String tableId = prepTableForScanTest(client, tableName);
 
-      try (BatchScanner s = c.createBatchScanner(tableName)) {
-        s.setRanges(List.of(new Range("a", "c")));
-        // Should return keys for a, b, c
-        assertEquals(3, Iterables.size(s));
-      }
+    try (BatchScanner s = client.createBatchScanner(tableName)) {
+      s.setRanges(List.of(new Range("a", "c")));
+      // Should return keys for a, b, c
+      assertEquals(3, Iterables.size(s));
+    }
 
-      List<TabletStats> stats = getTabletStats(c, tableId);
-      // There should be one tablet online
-      assertEquals(1, stats.size());
-      assertTrue(ClientTabletCache.getInstance((ClientContext) c, 
TableId.of(tableId))
-          .getTabletHostingRequestCount() > 0);
+    List<TabletStats> stats = getTabletStats(client, tableId);
+    // There should be one tablet online
+    assertEquals(1, stats.size());
+    assertTrue(ClientTabletCache.getInstance((ClientContext) client, 
TableId.of(tableId))
+        .getTabletHostingRequestCount() > 0);
 
-    }
   }
 
   @Test
   public void testBatchScannerAssignsMultipleOnDemandTablets() throws 
Exception {
-    try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {
-      String tableName = super.getUniqueNames(1)[0];
+    String tableName = super.getUniqueNames(1)[0];
 
-      String tableId = prepTableForScanTest(c, tableName);
+    String tableId = prepTableForScanTest(client, tableName);
 
-      try (BatchScanner s = c.createBatchScanner(tableName)) {
-        s.setRanges(List.of(new Range("a", "s")));
-        assertEquals(19, Iterables.size(s));
-      }
+    try (BatchScanner s = client.createBatchScanner(tableName)) {
+      s.setRanges(List.of(new Range("a", "s")));
+      assertEquals(19, Iterables.size(s));
+    }
 
-      List<TabletStats> stats = getTabletStats(c, tableId);
-      assertEquals(3, stats.size());
-      long hostingRequestCount = ClientTabletCache
-          .getInstance((ClientContext) c, 
TableId.of(tableId)).getTabletHostingRequestCount();
-      assertTrue(hostingRequestCount > 0);
+    List<TabletStats> stats = getTabletStats(client, tableId);
+    assertEquals(3, stats.size());
+    long hostingRequestCount = ClientTabletCache
+        .getInstance((ClientContext) client, 
TableId.of(tableId)).getTabletHostingRequestCount();
+    assertTrue(hostingRequestCount > 0);
 
-      // Run another scan, all tablets should be loaded
-      try (BatchScanner s = c.createBatchScanner(tableName)) {
-        s.setRanges(List.of(new Range("a", "t")));
-        assertEquals(20, Iterables.size(s));
-      }
+    // Run another scan, all tablets should be loaded
+    try (BatchScanner s = client.createBatchScanner(tableName)) {
+      s.setRanges(List.of(new Range("a", "t")));
+      assertEquals(20, Iterables.size(s));
+    }
 
-      stats = getTabletStats(c, tableId);
-      assertEquals(3, stats.size());
-      // No more tablets should have been brought online
-      assertEquals(hostingRequestCount, ClientTabletCache
-          .getInstance((ClientContext) c, 
TableId.of(tableId)).getTabletHostingRequestCount());
+    stats = getTabletStats(client, tableId);
+    assertEquals(3, stats.size());
+    // No more tablets should have been brought online
+    assertEquals(hostingRequestCount, ClientTabletCache
+        .getInstance((ClientContext) client, 
TableId.of(tableId)).getTabletHostingRequestCount());
 
-    }
   }
 
   @Test
   public void testBatchWriterAssignsTablets() throws Exception {
-    try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {
-      String tableName = super.getUniqueNames(1)[0];
+    String tableName = super.getUniqueNames(1)[0];
 
-      prepTableForScanTest(c, tableName);
-    }
+    prepTableForScanTest(client, tableName);
   }
 
   @Test
   public void testOpidPreventsAssignment() throws Exception {
-    try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {
-      String tableName = super.getUniqueNames(1)[0];
+    String tableName = super.getUniqueNames(1)[0];
 
-      var tableId = TableId.of(prepTableForScanTest(c, tableName));
+    var tableId = TableId.of(prepTableForScanTest(client, tableName));
 
-      FateInstanceType type = FateInstanceType.fromTableId(tableId);
-      FateId fateId = FateId.from(type, UUID.randomUUID());
+    FateInstanceType type = FateInstanceType.fromTableId(tableId);
+    FateId fateId = FateId.from(type, UUID.randomUUID());
 
-      assertEquals(0, countTabletsWithLocation(c, tableId));
+    assertEquals(0, countTabletsWithLocation(client, tableId));
 
-      assertEquals(Set.of("f", "m", "t"), 
c.tableOperations().listSplits(tableName).stream()
-          .map(Text::toString).collect(Collectors.toSet()));
+    assertEquals(Set.of("f", "m", "t"), 
client.tableOperations().listSplits(tableName).stream()
+        .map(Text::toString).collect(Collectors.toSet()));
 
-      c.securityOperations().grantTablePermission(getPrincipal(),
-          AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
+    client.securityOperations().grantTablePermission(getPrincipal(),
+        AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
 
-      var ample = getCluster().getServerContext().getAmple();
-      var extent = new KeyExtent(tableId, new Text("m"), new Text("f"));
-      var opid = TabletOperationId.from(TabletOperationType.SPLITTING, fateId);
+    var ample = getCluster().getServerContext().getAmple();
+    var extent = new KeyExtent(tableId, new Text("m"), new Text("f"));
+    var opid = TabletOperationId.from(TabletOperationType.SPLITTING, fateId);
 
-      // Set the OperationId on one tablet, which will cause that tablet
-      // to not be assigned
-      ample.mutateTablet(extent).putOperation(opid).mutate();
+    // Set the OperationId on one tablet, which will cause that tablet
+    // to not be assigned
+    ample.mutateTablet(extent).putOperation(opid).mutate();
 
-      // Host all tablets. Can not call the setTabletAvailability api because 
it will block when an
-      // opid is present, so must directly set it in the metadata table.
-      ample.readTablets().forTable(tableId).build()
-          .forEach(tabletMetadata -> 
ample.mutateTablet(tabletMetadata.getExtent())
-              .putTabletAvailability(TabletAvailability.HOSTED).mutate());
+    // Host all tablets. Can not call the setTabletAvailability api because it 
will block when an
+    // opid is present, so must directly set it in the metadata table.
+    ample.readTablets().forTable(tableId).build()
+        .forEach(tabletMetadata -> 
ample.mutateTablet(tabletMetadata.getExtent())
+            .putTabletAvailability(TabletAvailability.HOSTED).mutate());
 
-      Wait.waitFor(() -> countTabletsWithLocation(c, tableId) == 3);
-      assertNull(
-          ample.readTablet(new KeyExtent(tableId, new Text("m"), new 
Text("f"))).getLocation());
+    Wait.waitFor(() -> countTabletsWithLocation(client, tableId) == 3);
+    assertNull(
+        ample.readTablet(new KeyExtent(tableId, new Text("m"), new 
Text("f"))).getLocation());
 
-      // Delete the OperationId column, tablet should be assigned
-      ample.mutateTablet(extent).deleteOperation().mutate();
-      Wait.waitFor(() -> countTabletsWithLocation(c, tableId) == 4);
+    // Delete the OperationId column, tablet should be assigned
+    ample.mutateTablet(extent).deleteOperation().mutate();
+    Wait.waitFor(() -> countTabletsWithLocation(client, tableId) == 4);
 
-      // Set the OperationId on one tablet, which will cause that tablet
-      // to be unhosted
-      ample.mutateTablet(extent).putOperation(opid).mutate();
+    // Set the OperationId on one tablet, which will cause that tablet
+    // to be unhosted
+    ample.mutateTablet(extent).putOperation(opid).mutate();
 
-      // there are four tablets, three should be assigned as one has a 
OperationId
-      Wait.waitFor(() -> countTabletsWithLocation(c, tableId) == 3);
-      assertNull(
-          ample.readTablet(new KeyExtent(tableId, new Text("m"), new 
Text("f"))).getLocation());
+    // there are four tablets, three should be assigned as one has a 
OperationId
+    Wait.waitFor(() -> countTabletsWithLocation(client, tableId) == 3);
+    assertNull(
+        ample.readTablet(new KeyExtent(tableId, new Text("m"), new 
Text("f"))).getLocation());
 
-      // Delete the OperationId column, tablet should be assigned again
-      ample.mutateTablet(extent).deleteOperation().mutate();
+    // Delete the OperationId column, tablet should be assigned again
+    ample.mutateTablet(extent).deleteOperation().mutate();
 
-      // after the operation id is deleted the tablet should be assigned
-      Wait.waitFor(() -> countTabletsWithLocation(c, tableId) == 4);
-    }
+    // after the operation id is deleted the tablet should be assigned
+    Wait.waitFor(() -> countTabletsWithLocation(client, tableId) == 4);
   }
 
   public static void loadDataForScan(AccumuloClient c, String tableName)
@@ -475,150 +468,144 @@ public class ManagerAssignmentIT extends 
SharedMiniClusterBase {
 
     String tableName = getUniqueNames(1)[0];
 
-    try (AccumuloClient client = 
Accumulo.newClient().from(getClientProps()).build()) {
-
-      Wait.waitFor(
-          () -> 
client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER).size() == 1,
-          SECONDS.toMillis(60), SECONDS.toMillis(2));
+    Wait.waitFor(
+        () -> 
client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER).size() == 1,
+        SECONDS.toMillis(60), SECONDS.toMillis(2));
 
-      client.tableOperations().create(tableName);
-      TableId tid = 
TableId.of(client.tableOperations().tableIdMap().get(tableName));
+    client.tableOperations().create(tableName);
+    TableId tid = 
TableId.of(client.tableOperations().tableIdMap().get(tableName));
 
-      // wait for everything to be hosted and balanced
-      client.instanceOperations().waitForBalance();
+    // wait for everything to be hosted and balanced
+    client.instanceOperations().waitForBalance();
 
-      try (var writer = client.createBatchWriter(tableName)) {
-        for (int i = 0; i < 1000000; i++) {
-          Mutation m = new Mutation(String.format("%08d", i));
-          m.put("", "", "");
-          writer.addMutation(m);
-        }
+    try (var writer = client.createBatchWriter(tableName)) {
+      for (int i = 0; i < 1000000; i++) {
+        Mutation m = new Mutation(String.format("%08d", i));
+        m.put("", "", "");
+        writer.addMutation(m);
       }
-      client.tableOperations().flush(tableName, null, null, true);
-
-      final CountDownLatch latch = new CountDownLatch(10);
-
-      Runnable task = () -> {
-        while (true) {
-          try (var scanner = new 
IsolatedScanner(client.createScanner(tableName))) {
-            // TODO maybe do not close scanner? The following limit was placed 
on the stream to
-            // avoid reading all the data possibly leaving a scan session 
active on the tserver
-            AtomicInteger count = new AtomicInteger(0);
-            scanner.forEach(e -> {
-              // let the test thread know that this thread has read some data
-              if (count.incrementAndGet() == 1_000) {
-                latch.countDown();
-              }
-            });
-          } catch (Exception e) {
-            e.printStackTrace();
-            break;
-          }
+    }
+    client.tableOperations().flush(tableName, null, null, true);
+
+    final CountDownLatch latch = new CountDownLatch(10);
+
+    Runnable task = () -> {
+      while (true) {
+        try (var scanner = new 
IsolatedScanner(client.createScanner(tableName))) {
+          // TODO maybe do not close scanner? The following limit was placed 
on the stream to
+          // avoid reading all the data possibly leaving a scan session active 
on the tserver
+          AtomicInteger count = new AtomicInteger(0);
+          scanner.forEach(e -> {
+            // let the test thread know that this thread has read some data
+            if (count.incrementAndGet() == 1_000) {
+              latch.countDown();
+            }
+          });
+        } catch (Exception e) {
+          e.printStackTrace();
+          break;
         }
-      };
-
-      ExecutorService service = Executors.newFixedThreadPool(10);
-      for (int i = 0; i < 10; i++) {
-        service.execute(task);
       }
+    };
 
-      // Wait until all threads are reading some data
-      latch.await();
-
-      // getClusterControl().stopAllServers(ServerType.TABLET_SERVER)
-      // could potentially send a kill -9 to the process. Shut the tablet
-      // servers down in a more graceful way.
-      final Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new 
HashMap<>();
-      ClientTabletCache.getInstance((ClientContext) client, 
tid).binRanges((ClientContext) client,
-          Collections.singletonList(TabletsSection.getRange()), binnedRanges);
-      binnedRanges.keySet().forEach((location) -> {
-        HostAndPort address = HostAndPort.fromString(location);
-        String addressWithSession = address.toString();
-        var zLockPath = getCluster().getServerContext().getServerPaths()
-            .createTabletServerPath(Constants.DEFAULT_RESOURCE_GROUP_NAME, 
address);
-        long sessionId =
-            
ServiceLock.getSessionId(getCluster().getServerContext().getZooCache(), 
zLockPath);
-        if (sessionId != 0) {
-          addressWithSession = address + "[" + Long.toHexString(sessionId) + 
"]";
-        }
+    ExecutorService service = Executors.newFixedThreadPool(10);
+    for (int i = 0; i < 10; i++) {
+      service.execute(task);
+    }
 
-        final String finalAddress = addressWithSession;
-        System.out.println("Attempting to shutdown TabletServer at: " + 
address);
-        try {
-          ThriftClientTypes.MANAGER.executeVoid((ClientContext) client,
-              c -> c.shutdownTabletServer(TraceUtil.traceInfo(),
-                  getCluster().getServerContext().rpcCreds(), finalAddress, 
false));
-        } catch (AccumuloException | AccumuloSecurityException e) {
-          fail("Error shutting down TabletServer", e);
-        }
+    // Wait until all threads are reading some data
+    latch.await();
+
+    // getClusterControl().stopAllServers(ServerType.TABLET_SERVER)
+    // could potentially send a kill -9 to the process. Shut the tablet
+    // servers down in a more graceful way.
+    final Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new 
HashMap<>();
+    ClientTabletCache.getInstance((ClientContext) client, 
tid).binRanges((ClientContext) client,
+        Collections.singletonList(TabletsSection.getRange()), binnedRanges);
+    binnedRanges.keySet().forEach((location) -> {
+      HostAndPort address = HostAndPort.fromString(location);
+      String addressWithSession = address.toString();
+      var zLockPath = getCluster().getServerContext().getServerPaths()
+          .createTabletServerPath(Constants.DEFAULT_RESOURCE_GROUP_NAME, 
address);
+      long sessionId =
+          
ServiceLock.getSessionId(getCluster().getServerContext().getZooCache(), 
zLockPath);
+      if (sessionId != 0) {
+        addressWithSession = address + "[" + Long.toHexString(sessionId) + "]";
+      }
 
-      });
+      final String finalAddress = addressWithSession;
+      System.out.println("Attempting to shutdown TabletServer at: " + address);
+      try {
+        ThriftClientTypes.MANAGER.executeVoid((ClientContext) client,
+            c -> c.shutdownTabletServer(TraceUtil.traceInfo(),
+                getCluster().getServerContext().rpcCreds(), finalAddress, 
false));
+      } catch (AccumuloException | AccumuloSecurityException e) {
+        fail("Error shutting down TabletServer", e);
+      }
 
-      Wait.waitFor(
-          () -> 
client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER).size() == 
0);
-
-      // restart the tablet server for the other tests. Need to call 
stopAllServers
-      // to clear out the process list because we shutdown the TabletServer 
outside
-      // of MAC control.
-      
getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-      getCluster().getClusterControl().start(ServerType.TABLET_SERVER);
-      Wait.waitFor(
-          () -> 
client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER).size() == 1,
-          60_000);
-    }
+    });
+
+    Wait.waitFor(
+        () -> 
client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER).size() == 
0);
+
+    // restart the tablet server for the other tests. Need to call 
stopAllServers
+    // to clear out the process list because we shutdown the TabletServer 
outside
+    // of MAC control.
+    getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
+    getCluster().getClusterControl().start(ServerType.TABLET_SERVER);
+    Wait.waitFor(
+        () -> 
client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER).size() == 1,
+        60_000);
   }
 
   @Test
   public void testShutdownOnlyTServerWithoutUserTable() throws Exception {
 
-    try (AccumuloClient client = 
Accumulo.newClient().from(getClientProps()).build()) {
-
-      Wait.waitFor(
-          () -> 
client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER).size() == 1,
-          SECONDS.toMillis(60), SECONDS.toMillis(2));
-
-      client.instanceOperations().waitForBalance();
-
-      // getClusterControl().stopAllServers(ServerType.TABLET_SERVER)
-      // could potentially send a kill -9 to the process. Shut the tablet
-      // servers down in a more graceful way.
-
-      Locations locs = 
client.tableOperations().locate(AccumuloTable.ROOT.tableName(),
-          Collections.singletonList(TabletsSection.getRange()));
-      
locs.groupByTablet().keySet().stream().map(locs::getTabletLocation).forEach(location
 -> {
-        HostAndPort address = HostAndPort.fromString(location);
-        String addressWithSession = address.toString();
-        var zLockPath = getCluster().getServerContext().getServerPaths()
-            .createTabletServerPath(Constants.DEFAULT_RESOURCE_GROUP_NAME, 
address);
-        long sessionId =
-            
ServiceLock.getSessionId(getCluster().getServerContext().getZooCache(), 
zLockPath);
-        if (sessionId != 0) {
-          addressWithSession = address + "[" + Long.toHexString(sessionId) + 
"]";
-        }
+    Wait.waitFor(
+        () -> 
client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER).size() == 1,
+        SECONDS.toMillis(60), SECONDS.toMillis(2));
+
+    client.instanceOperations().waitForBalance();
+
+    // getClusterControl().stopAllServers(ServerType.TABLET_SERVER)
+    // could potentially send a kill -9 to the process. Shut the tablet
+    // servers down in a more graceful way.
+
+    Locations locs = 
client.tableOperations().locate(AccumuloTable.ROOT.tableName(),
+        Collections.singletonList(TabletsSection.getRange()));
+    
locs.groupByTablet().keySet().stream().map(locs::getTabletLocation).forEach(location
 -> {
+      HostAndPort address = HostAndPort.fromString(location);
+      String addressWithSession = address.toString();
+      var zLockPath = getCluster().getServerContext().getServerPaths()
+          .createTabletServerPath(Constants.DEFAULT_RESOURCE_GROUP_NAME, 
address);
+      long sessionId =
+          
ServiceLock.getSessionId(getCluster().getServerContext().getZooCache(), 
zLockPath);
+      if (sessionId != 0) {
+        addressWithSession = address + "[" + Long.toHexString(sessionId) + "]";
+      }
 
-        final String finalAddress = addressWithSession;
-        System.out.println("Attempting to shutdown TabletServer at: " + 
address);
-        try {
-          ThriftClientTypes.MANAGER.executeVoid((ClientContext) client,
-              c -> c.shutdownTabletServer(TraceUtil.traceInfo(),
-                  getCluster().getServerContext().rpcCreds(), finalAddress, 
false));
-        } catch (AccumuloException | AccumuloSecurityException e) {
-          fail("Error shutting down TabletServer", e);
-        }
+      final String finalAddress = addressWithSession;
+      System.out.println("Attempting to shutdown TabletServer at: " + address);
+      try {
+        ThriftClientTypes.MANAGER.executeVoid((ClientContext) client,
+            c -> c.shutdownTabletServer(TraceUtil.traceInfo(),
+                getCluster().getServerContext().rpcCreds(), finalAddress, 
false));
+      } catch (AccumuloException | AccumuloSecurityException e) {
+        fail("Error shutting down TabletServer", e);
+      }
 
-      });
-      Wait.waitFor(
-          () -> 
client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER).size() == 
0);
-
-      // restart the tablet server for the other tests. Need to call 
stopAllServers
-      // to clear out the process list because we shutdown the TabletServer 
outside
-      // of MAC control.
-      
getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-      getCluster().getClusterControl().start(ServerType.TABLET_SERVER);
-      Wait.waitFor(
-          () -> 
client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER).size() == 1,
-          60_000);
-    }
+    });
+    Wait.waitFor(
+        () -> 
client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER).size() == 
0);
+
+    // restart the tablet server for the other tests. Need to call 
stopAllServers
+    // to clear out the process list because we shutdown the TabletServer 
outside
+    // of MAC control.
+    getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
+    getCluster().getClusterControl().start(ServerType.TABLET_SERVER);
+    Wait.waitFor(
+        () -> 
client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER).size() == 1,
+        60_000);
   }
 
   public static TabletMetadata getTabletMetadata(AccumuloClient c, String 
tableId, Text endRow) {

Reply via email to