You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by dl...@apache.org on 2022/04/19 18:25:39 UTC

[accumulo] branch main updated: Use Thrift Multiplex components to host multiple services (#2620)

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

dlmarion 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 8be98d6744 Use Thrift Multiplex components to host multiple services (#2620)
8be98d6744 is described below

commit 8be98d6744962ccb9806800943dcd8cc6f2003c1
Author: Dave Marion <dl...@apache.org>
AuthorDate: Tue Apr 19 14:25:35 2022 -0400

    Use Thrift Multiplex components to host multiple services (#2620)
    
    This commit changes all server side Thrift Processors to be
    TMultiplexedProcessors and changes all clients to use the
    TMultiplexedProtocol to support splitting TabletClientService
    into at least two services in a later commit for the ScanServer
    feature.
    
    This commit also modifies the Thrift IDL for TabletClientService
    such that it no longer extends ClientService and modifies
    ManagerClientService such that it no longer extends FateService.
    
    The TabletServer and Manager processes then needed to be
    modified to start two Thrift TProcessors instead of one. This
    required the use of the TMultiplexedProcessor object on the server
    side, which takes one or more Processors and places them in
    a map keyed on the Thrift service name. Clients, when
    communicating with the TabletServer or Manager, then need to
    use a TMultiplexedProtocol with the name of the Thrift service
    that they want to call. This server and client setup logic is located
    in two new classes: ThriftProcessorTypes and ThriftClientTypes
---
 .../core/clientImpl/ConditionalWriterImpl.java     |   5 +-
 .../core/clientImpl/FateManagerClient.java         | 122 +++++++++++++++
 .../core/clientImpl/InstanceOperationsImpl.java    |   9 +-
 .../accumulo/core/clientImpl/ManagerClient.java    |  33 +---
 .../core/clientImpl/ReplicationClient.java         |   6 +-
 .../accumulo/core/clientImpl/ServerClient.java     |  22 +--
 .../core/clientImpl/TableOperationsImpl.java       |  49 +++---
 .../TabletServerBatchReaderIterator.java           |   6 +-
 .../core/clientImpl/TabletServerBatchWriter.java   |   6 +-
 .../accumulo/core/clientImpl/ThriftScanner.java    |  10 +-
 .../apache/accumulo/core/clientImpl/Writer.java    |   3 +-
 .../accumulo/core/rpc/ThriftClientTypes.java       |  97 ++++++++++++
 .../org/apache/accumulo/core/rpc/ThriftUtil.java   |  60 ++------
 .../org/apache/accumulo/core/summary/Gatherer.java |   6 +-
 .../util/compaction/ExternalCompactionUtil.java    |   9 +-
 .../core/manager/thrift/ManagerClientService.java  |  12 +-
 .../tabletserver/thrift/TabletClientService.java   |  12 +-
 core/src/main/thrift/manager.thrift                |   2 +-
 core/src/main/thrift/tabletserver.thrift           |   2 +-
 .../MiniAccumuloClusterControl.java                |   5 +-
 .../accumulo/server/client/BulkImporter.java       |   3 +-
 .../accumulo/server/manager/LiveTServerSet.java    |  26 ++--
 .../manager/balancer/BalancerEnvironmentImpl.java  |   4 +-
 .../server/master/balancer/TabletBalancer.java     |   4 +-
 .../accumulo/server/rpc/ThriftProcessorTypes.java  | 169 +++++++++++++++++++++
 .../org/apache/accumulo/server/util/ECAdmin.java   |   4 +-
 .../server/util/VerifyTabletAssignments.java       |   4 +-
 .../coordinator/CompactionCoordinator.java         |  20 +--
 .../accumulo/coordinator/CompactionFinalizer.java  |   4 +-
 .../org/apache/accumulo/compactor/Compactor.java   |  20 +--
 server/gc/pom.xml                                  |   4 +
 .../apache/accumulo/gc/SimpleGarbageCollector.java |  22 +--
 .../accumulo/manager/FateServiceHandler.java       |   2 +-
 .../java/org/apache/accumulo/manager/Manager.java  |  46 +++---
 .../manager/ManagerClientServiceHandler.java       |   6 +-
 .../manager/tableOps/bulkVer1/LoadFiles.java       |   4 +-
 .../manager/tableOps/bulkVer2/LoadFiles.java       |   4 +-
 .../java/org/apache/accumulo/monitor/Monitor.java  |  11 +-
 .../rest/tservers/TabletServerResource.java        |   3 +-
 ...ClientHandler.java => TabletClientHandler.java} |  23 ++-
 .../org/apache/accumulo/tserver/TabletServer.java  |  50 +++---
 .../accumulo/shell/commands/FateCommand.java       |   4 +-
 .../org/apache/accumulo/test/TotalQueuedIT.java    |   5 +-
 .../test/compaction/ExternalCompactionTServer.java |   7 +-
 .../compaction/ExternalCompactionTestUtils.java    |   9 +-
 ...tingExternalCompactionTabletClientHandler.java} |  10 +-
 .../accumulo/test/functional/BulkFailureIT.java    |   4 +-
 .../accumulo/test/functional/ZombieTServer.java    |  12 +-
 .../accumulo/test/performance/NullTserver.java     |  12 +-
 ...GarbageCollectorCommunicatesWithTServersIT.java |   3 +-
 50 files changed, 668 insertions(+), 307 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
index 9643e0acb0..5d6e7c0335 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
@@ -63,6 +63,7 @@ import org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation;
 import org.apache.accumulo.core.dataImpl.thrift.TConditionalSession;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TMutation;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
@@ -533,9 +534,9 @@ class ConditionalWriterImpl implements ConditionalWriter {
   private TabletClientService.Iface getClient(HostAndPort location) throws TTransportException {
     TabletClientService.Iface client;
     if (timeout < context.getClientTimeoutInMillis())
-      client = ThriftUtil.getTServerClient(location, context, timeout);
+      client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, context, timeout);
     else
-      client = ThriftUtil.getTServerClient(location, context);
+      client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, context);
     return client;
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/FateManagerClient.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/FateManagerClient.java
new file mode 100644
index 0000000000..74d5e800b0
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/FateManagerClient.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.clientImpl;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.manager.thrift.FateService;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.thrift.TServiceClient;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FateManagerClient {
+  private static final Logger log = LoggerFactory.getLogger(FateManagerClient.class);
+
+  public static FateService.Client getConnectionWithRetry(ClientContext context) {
+    while (true) {
+
+      FateService.Client result = getConnection(context);
+      if (result != null)
+        return result;
+      sleepUninterruptibly(250, MILLISECONDS);
+    }
+  }
+
+  private static FateService.Client getConnection(ClientContext context) {
+    checkArgument(context != null, "context is null");
+
+    List<String> locations = context.getManagerLocations();
+
+    if (locations.isEmpty()) {
+      log.debug("No managers...");
+      return null;
+    }
+
+    HostAndPort manager = HostAndPort.fromString(locations.get(0));
+    if (manager.getPort() == 0)
+      return null;
+
+    try {
+      // Manager requests can take a long time: don't ever time out
+      return ThriftUtil.getClientNoTimeout(ThriftClientTypes.FATE, manager, context);
+    } catch (TTransportException tte) {
+      Throwable cause = tte.getCause();
+      if (cause != null && cause instanceof UnknownHostException) {
+        // do not expect to recover from this
+        throw new RuntimeException(tte);
+      }
+      log.debug("Failed to connect to manager=" + manager + ", will retry... ", tte);
+      return null;
+    }
+  }
+
+  public static void close(FateService.Iface iface, ClientContext context) {
+    TServiceClient client = (TServiceClient) iface;
+    if (client != null && client.getInputProtocol() != null
+        && client.getInputProtocol().getTransport() != null) {
+      context.getTransportPool().returnTransport(client.getInputProtocol().getTransport());
+    } else {
+      log.debug("Attempt to close null connection to the manager", new Exception());
+    }
+  }
+
+  public static boolean cancelFateOperation(ClientContext context, long txid)
+      throws AccumuloException, AccumuloSecurityException {
+    while (true) {
+      FateService.Client client = null;
+      try {
+        client = getConnectionWithRetry(context);
+        return client.cancelFateOperation(TraceUtil.traceInfo(), context.rpcCreds(), txid);
+      } catch (TTransportException tte) {
+        log.debug("ManagerClient request failed, retrying ... ", tte);
+        sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+      } catch (ThriftSecurityException e) {
+        throw new AccumuloSecurityException(e.user, e.code, e);
+      } catch (ThriftTableOperationException e) {
+        throw new AccumuloException(e);
+      } catch (ThriftNotActiveServiceException e) {
+        // Let it loop, fetching a new location
+        log.debug("Contacted a Manager which is no longer active, re-creating"
+            + " the connection to the active Manager");
+      } catch (Exception e) {
+        throw new AccumuloException(e);
+      } finally {
+        if (client != null)
+          close(client, context);
+      }
+    }
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
index 91d08a99f5..627a739761 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
@@ -23,7 +23,7 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.stream.Collectors.toList;
 import static org.apache.accumulo.core.rpc.ThriftUtil.createClient;
 import static org.apache.accumulo.core.rpc.ThriftUtil.createTransport;
-import static org.apache.accumulo.core.rpc.ThriftUtil.getTServerClient;
+import static org.apache.accumulo.core.rpc.ThriftUtil.getClient;
 import static org.apache.accumulo.core.rpc.ThriftUtil.returnClient;
 
 import java.util.ArrayList;
@@ -46,6 +46,7 @@ import org.apache.accumulo.core.clientImpl.thrift.ConfigurationType;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
 import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.AddressUtil;
@@ -159,7 +160,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
     final var parsedTserver = HostAndPort.fromString(tserver);
     Client client = null;
     try {
-      client = getTServerClient(parsedTserver, context);
+      client = getClient(ThriftClientTypes.TABLET_SERVER, parsedTserver, context);
 
       List<ActiveScan> as = new ArrayList<>();
       for (var activeScan : client.getActiveScans(TraceUtil.traceInfo(), context.rpcCreds())) {
@@ -193,7 +194,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
     final var parsedTserver = HostAndPort.fromString(tserver);
     Client client = null;
     try {
-      client = getTServerClient(parsedTserver, context);
+      client = getClient(ThriftClientTypes.TABLET_SERVER, parsedTserver, context);
 
       List<ActiveCompaction> as = new ArrayList<>();
       for (var tac : client.getActiveCompactions(TraceUtil.traceInfo(), context.rpcCreds())) {
@@ -263,7 +264,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
   public void ping(String tserver) throws AccumuloException {
     try (
         TTransport transport = createTransport(AddressUtil.parseAddress(tserver, false), context)) {
-      var client = createClient(new Client.Factory(), transport);
+      Client client = createClient(ThriftClientTypes.TABLET_SERVER, transport);
       client.getTabletServerStatus(TraceUtil.traceInfo(), context.rpcCreds());
     } catch (TException e) {
       throw new AccumuloException(e);
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ManagerClient.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ManagerClient.java
index 957bd7cfe4..644b5ffa47 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ManagerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ManagerClient.java
@@ -24,7 +24,6 @@ import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.net.UnknownHostException;
 import java.util.List;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -34,8 +33,8 @@ import org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceExceptio
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.manager.thrift.ManagerClientService;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
-import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.thrift.TServiceClient;
 import org.apache.thrift.transport.TTransportException;
@@ -71,8 +70,7 @@ public class ManagerClient {
 
     try {
       // Manager requests can take a long time: don't ever time out
-      return ThriftUtil.getClientNoTimeout(new ManagerClientService.Client.Factory(), manager,
-          context);
+      return ThriftUtil.getClientNoTimeout(ThriftClientTypes.MANAGER, manager, context);
     } catch (TTransportException tte) {
       Throwable cause = tte.getCause();
       if (cause != null && cause instanceof UnknownHostException) {
@@ -196,31 +194,4 @@ public class ManagerClient {
     }
   }
 
-  public static boolean cancelFateOperation(ClientContext context, long txid)
-      throws AccumuloException, AccumuloSecurityException {
-    while (true) {
-      ManagerClientService.Client client = null;
-      try {
-        client = getConnectionWithRetry(context);
-        return client.cancelFateOperation(TraceUtil.traceInfo(), context.rpcCreds(), txid);
-      } catch (TTransportException tte) {
-        log.debug("ManagerClient request failed, retrying ... ", tte);
-        sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
-      } catch (ThriftSecurityException e) {
-        throw new AccumuloSecurityException(e.user, e.code, e);
-      } catch (ThriftTableOperationException e) {
-        throw new AccumuloException(e);
-      } catch (ThriftNotActiveServiceException e) {
-        // Let it loop, fetching a new location
-        log.debug("Contacted a Manager which is no longer active, re-creating"
-            + " the connection to the active Manager");
-      } catch (Exception e) {
-        throw new AccumuloException(e);
-      } finally {
-        if (client != null)
-          close(client, context);
-      }
-    }
-  }
-
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationClient.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationClient.java
index d5955e52b1..6b779fda1f 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationClient.java
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.replication.thrift.ReplicationCoordinator;
 import org.apache.accumulo.core.replication.thrift.ReplicationServicer;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.fate.zookeeper.ZooReader;
@@ -106,7 +107,7 @@ public class ReplicationClient {
 
     try {
       // Manager requests can take a long time: don't ever time out
-      return ThriftUtil.getClientNoTimeout(new ReplicationCoordinator.Client.Factory(),
+      return ThriftUtil.getClientNoTimeout(ThriftClientTypes.REPLICATION_COORDINATOR,
           coordinatorAddr, context);
     } catch (TTransportException tte) {
       log.debug("Failed to connect to manager coordinator service ({})", coordinatorAddr, tte);
@@ -131,8 +132,7 @@ public class ReplicationClient {
     requireNonNull(server);
 
     try {
-      return ThriftUtil.getClient(new ReplicationServicer.Client.Factory(), server, context,
-          timeout);
+      return ThriftUtil.getClient(ThriftClientTypes.REPLICATION_SERVICER, server, context, timeout);
     } catch (TTransportException tte) {
       log.debug("Failed to connect to servicer ({}), will retry...", server, tte);
       throw tte;
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerClient.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerClient.java
index 00d369e3b6..c61e546cb2 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerClient.java
@@ -31,6 +31,8 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.clientImpl.thrift.ClientService;
 import org.apache.accumulo.core.clientImpl.thrift.ClientService.Client;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
+import org.apache.accumulo.core.rpc.ThriftClientTypes.ThriftClientType;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.ServerServices;
@@ -39,7 +41,6 @@ import org.apache.accumulo.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.thrift.TApplicationException;
 import org.apache.thrift.TServiceClient;
-import org.apache.thrift.TServiceClientFactory;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
@@ -50,14 +51,14 @@ public class ServerClient {
 
   public static <T> T execute(ClientContext context, ClientExecReturn<T,ClientService.Client> exec)
       throws AccumuloException, AccumuloSecurityException {
-    return execute(context, new ClientService.Client.Factory(), exec);
+    return execute(context, ThriftClientTypes.CLIENT, exec);
   }
 
   public static <CT extends TServiceClient,RT> RT execute(ClientContext context,
-      TServiceClientFactory<CT> factory, ClientExecReturn<RT,CT> exec)
+      ThriftClientType<CT,?> type, ClientExecReturn<RT,CT> exec)
       throws AccumuloException, AccumuloSecurityException {
     try {
-      return executeRaw(context, factory, exec);
+      return executeRaw(context, type, exec);
     } catch (ThriftSecurityException e) {
       throw new AccumuloSecurityException(e.user, e.code, e);
     } catch (AccumuloException e) {
@@ -82,16 +83,16 @@ public class ServerClient {
 
   public static <T> T executeRaw(ClientContext context,
       ClientExecReturn<T,ClientService.Client> exec) throws Exception {
-    return executeRaw(context, new ClientService.Client.Factory(), exec);
+    return executeRaw(context, ThriftClientTypes.CLIENT, exec);
   }
 
   public static <CT extends TServiceClient,RT> RT executeRaw(ClientContext context,
-      TServiceClientFactory<CT> factory, ClientExecReturn<RT,CT> exec) throws Exception {
+      ThriftClientType<CT,?> type, ClientExecReturn<RT,CT> exec) throws Exception {
     while (true) {
       CT client = null;
       String server = null;
       try {
-        Pair<String,CT> pair = ServerClient.getConnection(context, factory, true);
+        Pair<String,CT> pair = ServerClient.getConnection(context, type, true);
         server = pair.getFirst();
         client = pair.getSecond();
         return exec.execute(client);
@@ -114,7 +115,7 @@ public class ServerClient {
       String server = null;
       try {
         Pair<String,Client> pair =
-            ServerClient.getConnection(context, new ClientService.Client.Factory(), true);
+            ServerClient.getConnection(context, ThriftClientTypes.CLIENT, true);
         server = pair.getFirst();
         client = pair.getSecond();
         exec.execute(client);
@@ -134,8 +135,7 @@ public class ServerClient {
   static volatile boolean warnedAboutTServersBeingDown = false;
 
   public static <CT extends TServiceClient> Pair<String,CT> getConnection(ClientContext context,
-      TServiceClientFactory<CT> factory, boolean preferCachedConnections)
-      throws TTransportException {
+      ThriftClientType<CT,?> type, boolean preferCachedConnections) throws TTransportException {
     checkArgument(context != null, "context is null");
     long rpcTimeout = context.getClientTimeoutInMillis();
     // create list of servers
@@ -159,7 +159,7 @@ public class ServerClient {
     try {
       Pair<String,TTransport> pair =
           context.getTransportPool().getAnyTransport(servers, preferCachedConnections);
-      CT client = ThriftUtil.createClient(factory, pair.getSecond());
+      CT client = ThriftUtil.createClient(type, pair.getSecond());
       opened = true;
       warnedAboutTServersBeingDown = false;
       return new Pair<>(pair.getFirst(), client);
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
index 60f3d77f98..d3817c403f 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
@@ -89,7 +89,6 @@ import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.client.summary.Summary;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.clientImpl.bulk.BulkImport;
-import org.apache.accumulo.core.clientImpl.thrift.ClientService;
 import org.apache.accumulo.core.clientImpl.thrift.ClientService.Client;
 import org.apache.accumulo.core.clientImpl.thrift.TDiskUsage;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException;
@@ -113,6 +112,7 @@ import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.manager.thrift.FateOperation;
+import org.apache.accumulo.core.manager.thrift.FateService;
 import org.apache.accumulo.core.manager.thrift.ManagerClientService;
 import org.apache.accumulo.core.metadata.MetadataServicer;
 import org.apache.accumulo.core.metadata.MetadataTable;
@@ -121,6 +121,7 @@ import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
@@ -254,9 +255,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   private long beginFateOperation() throws ThriftSecurityException, TException {
     while (true) {
-      ManagerClientService.Iface client = null;
+      FateService.Iface client = null;
       try {
-        client = ManagerClient.getConnectionWithRetry(context);
+        client = FateManagerClient.getConnectionWithRetry(context);
         return client.beginFateOperation(TraceUtil.traceInfo(), context.rpcCreds());
       } catch (TTransportException tte) {
         log.debug("Failed to call beginFateOperation(), retrying ... ", tte);
@@ -266,7 +267,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
         log.debug("Contacted a Manager which is no longer active, retrying");
         sleepUninterruptibly(100, MILLISECONDS);
       } finally {
-        ManagerClient.close(client, context);
+        FateManagerClient.close(client, context);
       }
     }
   }
@@ -277,9 +278,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
       Map<String,String> opts, boolean autoCleanUp)
       throws ThriftSecurityException, TException, ThriftTableOperationException {
     while (true) {
-      ManagerClientService.Iface client = null;
+      FateService.Iface client = null;
       try {
-        client = ManagerClient.getConnectionWithRetry(context);
+        client = FateManagerClient.getConnectionWithRetry(context);
         client.executeFateOperation(TraceUtil.traceInfo(), context.rpcCreds(), opid, op, args, opts,
             autoCleanUp);
         return;
@@ -291,7 +292,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
         log.debug("Contacted a Manager which is no longer active, retrying");
         sleepUninterruptibly(100, MILLISECONDS);
       } finally {
-        ManagerClient.close(client, context);
+        FateManagerClient.close(client, context);
       }
     }
   }
@@ -299,9 +300,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
   private String waitForFateOperation(long opid)
       throws ThriftSecurityException, TException, ThriftTableOperationException {
     while (true) {
-      ManagerClientService.Iface client = null;
+      FateService.Iface client = null;
       try {
-        client = ManagerClient.getConnectionWithRetry(context);
+        client = FateManagerClient.getConnectionWithRetry(context);
         return client.waitForFateOperation(TraceUtil.traceInfo(), context.rpcCreds(), opid);
       } catch (TTransportException tte) {
         log.debug("Failed to call waitForFateOperation(), retrying ... ", tte);
@@ -311,16 +312,16 @@ public class TableOperationsImpl extends TableOperationsHelper {
         log.debug("Contacted a Manager which is no longer active, retrying");
         sleepUninterruptibly(100, MILLISECONDS);
       } finally {
-        ManagerClient.close(client, context);
+        FateManagerClient.close(client, context);
       }
     }
   }
 
   private void finishFateOperation(long opid) throws ThriftSecurityException, TException {
     while (true) {
-      ManagerClientService.Iface client = null;
+      FateService.Iface client = null;
       try {
-        client = ManagerClient.getConnectionWithRetry(context);
+        client = FateManagerClient.getConnectionWithRetry(context);
         client.finishFateOperation(TraceUtil.traceInfo(), context.rpcCreds(), opid);
         break;
       } catch (TTransportException tte) {
@@ -331,7 +332,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
         log.debug("Contacted a Manager which is no longer active, retrying");
         sleepUninterruptibly(100, MILLISECONDS);
       } finally {
-        ManagerClient.close(client, context);
+        FateManagerClient.close(client, context);
       }
     }
   }
@@ -554,7 +555,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
         HostAndPort address = HostAndPort.fromString(tl.tablet_location);
 
         try {
-          TabletClientService.Client client = ThriftUtil.getTServerClient(address, context);
+          TabletClientService.Client client =
+              ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
           try {
 
             OpTimer timer = null;
@@ -1463,7 +1465,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
         // this operation may us a lot of memory... its likely that connections to tabletservers
         // hosting metadata tablets will be cached, so do not use cached
         // connections
-        pair = ServerClient.getConnection(context, new ClientService.Client.Factory(), false);
+        pair = ServerClient.getConnection(context, ThriftClientTypes.CLIENT, false);
         diskUsages = pair.getSecond().getDiskUsage(tableNames, context.rpcCreds());
       } catch (ThriftTableOperationException e) {
         switch (e.getType()) {
@@ -1900,15 +1902,14 @@ public class TableOperationsImpl extends TableOperationsHelper {
           _flush(tableId, startRow, endRow, true);
         }
 
-        TSummaries ret =
-            ServerClient.execute(context, new TabletClientService.Client.Factory(), client -> {
-              TSummaries tsr =
-                  client.startGetSummaries(TraceUtil.traceInfo(), context.rpcCreds(), request);
-              while (!tsr.finished) {
-                tsr = client.contiuneGetSummaries(TraceUtil.traceInfo(), tsr.sessionId);
-              }
-              return tsr;
-            });
+        TSummaries ret = ServerClient.execute(context, ThriftClientTypes.TABLET_SERVER, client -> {
+          TSummaries tsr =
+              client.startGetSummaries(TraceUtil.traceInfo(), context.rpcCreds(), request);
+          while (!tsr.finished) {
+            tsr = client.contiuneGetSummaries(TraceUtil.traceInfo(), tsr.sessionId);
+          }
+          return tsr;
+        });
         return new SummaryCollection(ret).getSummaries();
       }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
index d9ee46cdd9..5cbc595c09 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
@@ -58,6 +58,7 @@ import org.apache.accumulo.core.dataImpl.thrift.MultiScanResult;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyValue;
 import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
@@ -653,9 +654,10 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
       final HostAndPort parsedServer = HostAndPort.fromString(server);
       final TabletClientService.Client client;
       if (timeoutTracker.getTimeOut() < context.getClientTimeoutInMillis())
-        client = ThriftUtil.getTServerClient(parsedServer, context, timeoutTracker.getTimeOut());
+        client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context,
+            timeoutTracker.getTimeOut());
       else
-        client = ThriftUtil.getTServerClient(parsedServer, context);
+        client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context);
 
       try {
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
index 518469103a..dc63da1455 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
@@ -65,6 +65,7 @@ import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.TabletIdImpl;
 import org.apache.accumulo.core.dataImpl.thrift.TMutation;
 import org.apache.accumulo.core.dataImpl.thrift.UpdateErrors;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
@@ -917,9 +918,10 @@ public class TabletServerBatchWriter implements AutoCloseable {
         final TabletClientService.Iface client;
 
         if (timeoutTracker.getTimeOut() < context.getClientTimeoutInMillis())
-          client = ThriftUtil.getTServerClient(parsedServer, context, timeoutTracker.getTimeOut());
+          client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context,
+              timeoutTracker.getTimeOut());
         else
-          client = ThriftUtil.getTServerClient(parsedServer, context);
+          client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context);
 
         try {
           MutationSet allFailures = new MutationSet();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
index 0243a84c55..41e4aeeba1 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
@@ -54,6 +54,7 @@ import org.apache.accumulo.core.dataImpl.thrift.InitialScan;
 import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
 import org.apache.accumulo.core.dataImpl.thrift.ScanResult;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyValue;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
@@ -100,7 +101,8 @@ public class ThriftScanner {
     final HostAndPort parsedServer = HostAndPort.fromString(server);
     try {
       TInfo tinfo = TraceUtil.traceInfo();
-      TabletClientService.Client client = ThriftUtil.getTServerClient(parsedServer, context);
+      TabletClientService.Client client =
+          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context);
       try {
         // not reading whole rows (or stopping on row boundaries) so there is no need to enable
         // isolation below
@@ -450,7 +452,8 @@ public class ThriftScanner {
 
     final TInfo tinfo = TraceUtil.traceInfo();
     final HostAndPort parsedLocation = HostAndPort.fromString(loc.tablet_location);
-    TabletClientService.Client client = ThriftUtil.getTServerClient(parsedLocation, context);
+    TabletClientService.Client client =
+        ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedLocation, context);
 
     String old = Thread.currentThread().getName();
     try {
@@ -583,7 +586,8 @@ public class ThriftScanner {
       HostAndPort parsedLocation = HostAndPort.fromString(scanState.prevLoc.tablet_location);
       TabletClientService.Client client = null;
       try {
-        client = ThriftUtil.getTServerClient(parsedLocation, scanState.context);
+        client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedLocation,
+            scanState.context);
         client.closeScan(tinfo, scanState.scanID);
       } catch (TException e) {
         // ignore this is a best effort
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
index 9425fb840c..9990d6869c 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
@@ -68,7 +69,7 @@ public class Writer {
 
     TabletClientService.Iface client = null;
     try {
-      client = ThriftUtil.getTServerClient(server, context);
+      client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, server, context);
       client.update(TraceUtil.traceInfo(), context.rpcCreds(), extent.toThrift(), m.toThrift(),
           TDurability.DEFAULT);
       return;
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftClientTypes.java b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftClientTypes.java
new file mode 100644
index 0000000000..dd567c2de1
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftClientTypes.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.rpc;
+
+import org.apache.accumulo.core.clientImpl.thrift.ClientService;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
+import org.apache.accumulo.core.compaction.thrift.CompactorService;
+import org.apache.accumulo.core.gc.thrift.GCMonitorService;
+import org.apache.accumulo.core.manager.thrift.FateService;
+import org.apache.accumulo.core.manager.thrift.ManagerClientService;
+import org.apache.accumulo.core.replication.thrift.ReplicationCoordinator;
+import org.apache.accumulo.core.replication.thrift.ReplicationServicer;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.thrift.TServiceClient;
+import org.apache.thrift.TServiceClientFactory;
+import org.apache.thrift.protocol.TMultiplexedProtocol;
+import org.apache.thrift.protocol.TProtocol;
+
+public class ThriftClientTypes {
+
+  public static class ThriftClientType<C extends TServiceClient,
+      F extends TServiceClientFactory<C>> {
+
+    private final String serviceName;
+    private final F clientFactory;
+
+    public ThriftClientType(String serviceName, F clientFactory) {
+      super();
+      this.serviceName = serviceName;
+      this.clientFactory = clientFactory;
+    }
+
+    public String getServiceName() {
+      return serviceName;
+    }
+
+    public F getClientFactory() {
+      return clientFactory;
+    }
+
+    public C getClient(TProtocol prot) {
+      // All server side TProcessors are multiplexed. Wrap this protocol.
+      return clientFactory.getClient(new TMultiplexedProtocol(prot, getServiceName()));
+    }
+
+  }
+
+  public static final ThriftClientType<ClientService.Client,ClientService.Client.Factory> CLIENT =
+      new ThriftClientType<>("ClientService", new ClientService.Client.Factory());
+
+  public static final ThriftClientType<CompactorService.Client,
+      CompactorService.Client.Factory> COMPACTOR =
+          new ThriftClientType<>("CompactorService", new CompactorService.Client.Factory());
+
+  public static final ThriftClientType<CompactionCoordinatorService.Client,
+      CompactionCoordinatorService.Client.Factory> COORDINATOR = new ThriftClientType<>(
+          "CompactionCoordinatorService", new CompactionCoordinatorService.Client.Factory());
+
+  public static final ThriftClientType<FateService.Client,FateService.Client.Factory> FATE =
+      new ThriftClientType<>("FateService", new FateService.Client.Factory());
+
+  public static final ThriftClientType<GCMonitorService.Client,GCMonitorService.Client.Factory> GC =
+      new ThriftClientType<>("GCMonitorService", new GCMonitorService.Client.Factory());
+
+  public static final ThriftClientType<ManagerClientService.Client,
+      ManagerClientService.Client.Factory> MANAGER =
+          new ThriftClientType<>("ManagerClientService", new ManagerClientService.Client.Factory());
+
+  public static final ThriftClientType<ReplicationCoordinator.Client,
+      ReplicationCoordinator.Client.Factory> REPLICATION_COORDINATOR = new ThriftClientType<>(
+          "ReplicationCoordinator", new ReplicationCoordinator.Client.Factory());
+
+  public static final ThriftClientType<ReplicationServicer.Client,
+      ReplicationServicer.Client.Factory> REPLICATION_SERVICER =
+          new ThriftClientType<>("ReplicationServicer", new ReplicationServicer.Client.Factory());
+
+  public static final ThriftClientType<TabletClientService.Client,
+      TabletClientService.Client.Factory> TABLET_SERVER =
+          new ThriftClientType<>("TabletClientService", new TabletClientService.Client.Factory());
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java
index 53dfb8954e..3e63ac1cbf 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java
@@ -34,13 +34,12 @@ import javax.net.ssl.TrustManagerFactory;
 
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.rpc.SaslConnectionParams.SaslMechanism;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.rpc.ThriftClientTypes.ThriftClientType;
 import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.thrift.TException;
 import org.apache.thrift.TServiceClient;
-import org.apache.thrift.TServiceClientFactory;
 import org.apache.thrift.protocol.TProtocolFactory;
 import org.apache.thrift.transport.TSSLTransportFactory;
 import org.apache.thrift.transport.TSaslClientTransport;
@@ -91,52 +90,51 @@ public class ThriftUtil {
   /**
    * Create a Thrift client using the given factory and transport
    */
-  public static <T extends TServiceClient> T createClient(TServiceClientFactory<T> factory,
+  public static <T extends TServiceClient> T createClient(ThriftClientType<T,?> type,
       TTransport transport) {
-    return factory.getClient(protocolFactory.getProtocol(transport),
-        protocolFactory.getProtocol(transport));
+    return type.getClient(protocolFactory.getProtocol(transport));
   }
 
   /**
    * Create a Thrift client using the given factory with a pooled transport (if available), the
    * address, and client context with no timeout.
    *
-   * @param factory
-   *          Thrift client factory
+   * @param type
+   *          Thrift client type
    * @param address
    *          Server address for client to connect to
    * @param context
    *          RPC options
    */
-  public static <T extends TServiceClient> T getClientNoTimeout(TServiceClientFactory<T> factory,
+  public static <T extends TServiceClient> T getClientNoTimeout(ThriftClientType<T,?> type,
       HostAndPort address, ClientContext context) throws TTransportException {
-    return getClient(factory, address, context, 0);
+    return getClient(type, address, context, 0);
   }
 
   /**
    * Create a Thrift client using the given factory with a pooled transport (if available), the
    * address and client context. Client timeout is extracted from the ClientContext
    *
-   * @param factory
-   *          Thrift client factory
+   * @param type
+   *          Thrift client type
    * @param address
    *          Server address for client to connect to
    * @param context
    *          RPC options
    */
-  public static <T extends TServiceClient> T getClient(TServiceClientFactory<T> factory,
+  public static <T extends TServiceClient> T getClient(ThriftClientType<T,?> type,
       HostAndPort address, ClientContext context) throws TTransportException {
     TTransport transport = context.getTransportPool().getTransport(address,
         context.getClientTimeoutInMillis(), context);
-    return createClient(factory, transport);
+    return createClient(type, transport);
   }
 
   /**
    * Create a Thrift client using the given factory with a pooled transport (if available) using the
    * address, client context and timeout
    *
-   * @param factory
-   *          Thrift client factory
+   * @param type
+   *          Thrift client type
    * @param address
    *          Server address for client to connect to
    * @param context
@@ -144,10 +142,10 @@ public class ThriftUtil {
    * @param timeout
    *          Socket timeout which overrides the ClientContext timeout
    */
-  public static <T extends TServiceClient> T getClient(TServiceClientFactory<T> factory,
+  public static <T extends TServiceClient> T getClient(ThriftClientType<T,?> type,
       HostAndPort address, ClientContext context, long timeout) throws TTransportException {
     TTransport transport = context.getTransportPool().getTransport(address, timeout, context);
-    return createClient(factory, transport);
+    return createClient(type, transport);
   }
 
   /**
@@ -162,34 +160,6 @@ public class ThriftUtil {
     }
   }
 
-  /**
-   * Create a TabletServer Thrift client
-   *
-   * @param address
-   *          Server address for client to connect to
-   * @param context
-   *          RPC options
-   */
-  public static TabletClientService.Client getTServerClient(HostAndPort address,
-      ClientContext context) throws TTransportException {
-    return getClient(new TabletClientService.Client.Factory(), address, context);
-  }
-
-  /**
-   * Create a TabletServer Thrift client
-   *
-   * @param address
-   *          Server address for client to connect to
-   * @param context
-   *          Options for connecting to the server
-   * @param timeout
-   *          Socket timeout which overrides the ClientContext timeout
-   */
-  public static TabletClientService.Client getTServerClient(HostAndPort address,
-      ClientContext context, long timeout) throws TTransportException {
-    return getClient(new TabletClientService.Client.Factory(), address, context, timeout);
-  }
-
   /**
    * Create a transport that is not pooled
    *
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java b/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
index 6a0938f00d..23b80ffcab 100644
--- a/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
+++ b/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
@@ -64,10 +64,10 @@ import org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest;
 import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.spi.cache.BlockCache;
 import org.apache.accumulo.core.spi.crypto.CryptoService;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.trace.thrift.TInfo;
@@ -308,7 +308,7 @@ public class Gatherer {
 
       Client client = null;
       try {
-        client = ThriftUtil.getTServerClient(location, ctx);
+        client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, ctx);
         // partition files into smaller chunks so that not too many are sent to a tserver at once
         for (Map<TabletFile,List<TRowRange>> files : partition(allFiles, 500)) {
           if (!pfiles.failedFiles.isEmpty()) {
@@ -563,7 +563,7 @@ public class Gatherer {
 
       TSummaries tSums;
       try {
-        tSums = ServerClient.execute(ctx, new TabletClientService.Client.Factory(), client -> {
+        tSums = ServerClient.execute(ctx, ThriftClientTypes.TABLET_SERVER, client -> {
           TSummaries tsr =
               client.startGetSummariesForPartition(tinfo, ctx.rpcCreds(), req, modulus, remainder);
           while (!tsr.finished && !cancelFlag.get()) {
diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
index 67ff60fc2a..c01f6ca9a9 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.compaction.thrift.CompactorService;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
 import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
@@ -162,7 +163,7 @@ public class ExternalCompactionUtil {
       ClientContext context) throws ThriftSecurityException {
     CompactorService.Client client = null;
     try {
-      client = ThriftUtil.getClient(new CompactorService.Client.Factory(), compactor, context);
+      client = ThriftUtil.getClient(ThriftClientTypes.COMPACTOR, compactor, context);
       return client.getActiveCompactions(TraceUtil.traceInfo(), context.rpcCreds());
     } catch (ThriftSecurityException e) {
       throw e;
@@ -188,7 +189,7 @@ public class ExternalCompactionUtil {
 
     CompactorService.Client client = null;
     try {
-      client = ThriftUtil.getClient(new CompactorService.Client.Factory(), compactorAddr, context);
+      client = ThriftUtil.getClient(ThriftClientTypes.COMPACTOR, compactorAddr, context);
       TExternalCompactionJob job =
           client.getRunningCompaction(TraceUtil.traceInfo(), context.rpcCreds());
       if (job.getExternalCompactionId() != null) {
@@ -207,7 +208,7 @@ public class ExternalCompactionUtil {
       ClientContext context) {
     CompactorService.Client client = null;
     try {
-      client = ThriftUtil.getClient(new CompactorService.Client.Factory(), compactorAddr, context);
+      client = ThriftUtil.getClient(ThriftClientTypes.COMPACTOR, compactorAddr, context);
       String secid = client.getRunningCompactionId(TraceUtil.traceInfo(), context.rpcCreds());
       if (!secid.isEmpty()) {
         return ExternalCompactionId.of(secid);
@@ -311,7 +312,7 @@ public class ExternalCompactionUtil {
       String ecid) {
     CompactorService.Client client = null;
     try {
-      client = ThriftUtil.getClient(new CompactorService.Client.Factory(), compactorAddr, context);
+      client = ThriftUtil.getClient(ThriftClientTypes.COMPACTOR, compactorAddr, context);
       client.cancel(TraceUtil.traceInfo(), context.rpcCreds(), ecid);
     } catch (TException e) {
       LOG.debug("Failed to cancel compactor {} for {}", compactorAddr, ecid, e);
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java
index 24dc777557..de50ae7600 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java
@@ -27,7 +27,7 @@ package org.apache.accumulo.core.manager.thrift;
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class ManagerClientService {
 
-  public interface Iface extends FateService.Iface {
+  public interface Iface {
 
     public long initiateFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
@@ -67,7 +67,7 @@ public class ManagerClientService {
 
   }
 
-  public interface AsyncIface extends FateService .AsyncIface {
+  public interface AsyncIface {
 
     public void initiateFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
 
@@ -107,7 +107,7 @@ public class ManagerClientService {
 
   }
 
-  public static class Client extends FateService.Client implements Iface {
+  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
     public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
       public Factory() {}
       public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
@@ -643,7 +643,7 @@ public class ManagerClientService {
     }
 
   }
-  public static class AsyncClient extends FateService.AsyncClient implements AsyncIface {
+  public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
       private org.apache.thrift.async.TAsyncClientManager clientManager;
       private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
@@ -1382,7 +1382,7 @@ public class ManagerClientService {
 
   }
 
-  public static class Processor<I extends Iface> extends FateService.Processor<I> implements org.apache.thrift.TProcessor {
+  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
     private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName());
     public Processor(I iface) {
       super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
@@ -1970,7 +1970,7 @@ public class ManagerClientService {
 
   }
 
-  public static class AsyncProcessor<I extends AsyncIface> extends FateService.AsyncProcessor<I> {
+  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
     private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(AsyncProcessor.class.getName());
     public AsyncProcessor(I iface) {
       super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
index dc3b23aabe..8d799d2c5e 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
@@ -27,7 +27,7 @@ package org.apache.accumulo.core.tabletserver.thrift;
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class TabletClientService {
 
-  public interface Iface extends org.apache.accumulo.core.clientImpl.thrift.ClientService.Iface {
+  public interface Iface {
 
     public org.apache.accumulo.core.dataImpl.thrift.InitialScan startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<jav [...]
 
@@ -111,7 +111,7 @@ public class TabletClientService {
 
   }
 
-  public interface AsyncIface extends org.apache.accumulo.core.clientImpl.thrift.ClientService .AsyncIface {
+  public interface AsyncIface {
 
     public void startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,jav [...]
 
@@ -195,7 +195,7 @@ public class TabletClientService {
 
   }
 
-  public static class Client extends org.apache.accumulo.core.clientImpl.thrift.ClientService.Client implements Iface {
+  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
     public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
       public Factory() {}
       public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
@@ -1198,7 +1198,7 @@ public class TabletClientService {
     }
 
   }
-  public static class AsyncClient extends org.apache.accumulo.core.clientImpl.thrift.ClientService.AsyncClient implements AsyncIface {
+  public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
       private org.apache.thrift.async.TAsyncClientManager clientManager;
       private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
@@ -2866,7 +2866,7 @@ public class TabletClientService {
 
   }
 
-  public static class Processor<I extends Iface> extends org.apache.accumulo.core.clientImpl.thrift.ClientService.Processor<I> implements org.apache.thrift.TProcessor {
+  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
     private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName());
     public Processor(I iface) {
       super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
@@ -4029,7 +4029,7 @@ public class TabletClientService {
 
   }
 
-  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.accumulo.core.clientImpl.thrift.ClientService.AsyncProcessor<I> {
+  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
     private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(AsyncProcessor.class.getName());
     public AsyncProcessor(I iface) {
       super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
diff --git a/core/src/main/thrift/manager.thrift b/core/src/main/thrift/manager.thrift
index 28eea0ae80..23cb0b0acc 100644
--- a/core/src/main/thrift/manager.thrift
+++ b/core/src/main/thrift/manager.thrift
@@ -156,7 +156,7 @@ service FateService {
   
 }
 
-service ManagerClientService extends FateService {
+service ManagerClientService {
 
   // table management methods
   i64 initiateFlush(
diff --git a/core/src/main/thrift/tabletserver.thrift b/core/src/main/thrift/tabletserver.thrift
index 2a68b722bf..5293e41616 100644
--- a/core/src/main/thrift/tabletserver.thrift
+++ b/core/src/main/thrift/tabletserver.thrift
@@ -196,7 +196,7 @@ struct TCompactionStats{
   3:i64 fileSize;
 }
 
-service TabletClientService extends client.ClientService {
+service TabletClientService {
 
   // scan a range of keys
   data.InitialScan startScan(
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
index d0a580cd01..2cb2994370 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
@@ -38,6 +38,7 @@ import org.apache.accumulo.coordinator.CompactionCoordinator;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
 import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.HostAndPort;
@@ -134,8 +135,8 @@ public class MiniAccumuloClusterControl implements ClusterControl {
     if (coordinatorHost.isEmpty()) {
       throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper");
     }
-    CompactionCoordinatorService.Client client = ThriftUtil.getClient(
-        new CompactionCoordinatorService.Client.Factory(), coordinatorHost.get(), context);
+    CompactionCoordinatorService.Client client =
+        ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, coordinatorHost.get(), context);
     try {
       return client.getRunningCompactions(TraceUtil.traceInfo(), context.rpcCreds());
     } finally {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
index f6ecdc735a..0694a19a20 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
@@ -54,6 +54,7 @@ import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
@@ -566,7 +567,7 @@ public class BulkImporter {
     try {
       long timeInMillis = context.getConfiguration().getTimeInMillis(Property.TSERV_BULK_TIMEOUT);
       TabletClientService.Iface client =
-          ThriftUtil.getTServerClient(location, context, timeInMillis);
+          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, context, timeInMillis);
       try {
         HashMap<KeyExtent,Map<String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files =
             new HashMap<>();
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java
index fbfec53016..72809d2ad2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java
@@ -31,11 +31,13 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.clientImpl.thrift.ClientService;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.tabletserver.thrift.TUnloadTabletGoal;
@@ -98,12 +100,12 @@ public class LiveTServerSet implements Watcher {
         // see ACCUMULO-3597
         try (TTransport transport = ThriftUtil.createTransport(address, context)) {
           TabletClientService.Client client =
-              ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport);
+              ThriftUtil.createClient(ThriftClientTypes.TABLET_SERVER, transport);
           loadTablet(client, lock, extent);
         }
       } else {
         TabletClientService.Client client =
-            ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, context);
+            ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
         try {
           loadTablet(client, lock, extent);
         } finally {
@@ -115,7 +117,7 @@ public class LiveTServerSet implements Watcher {
     public void unloadTablet(ServiceLock lock, KeyExtent extent, TUnloadTabletGoal goal,
         long requestTime) throws TException {
       TabletClientService.Client client =
-          ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, context);
+          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
       try {
         client.unloadTablet(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock),
             extent.toThrift(), goal, requestTime);
@@ -134,7 +136,7 @@ public class LiveTServerSet implements Watcher {
 
       try (TTransport transport = ThriftUtil.createTransport(address, context)) {
         TabletClientService.Client client =
-            ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport);
+            ThriftUtil.createClient(ThriftClientTypes.TABLET_SERVER, transport);
         TabletServerStatus status =
             client.getTabletServerStatus(TraceUtil.traceInfo(), context.rpcCreds());
         if (status != null) {
@@ -146,7 +148,7 @@ public class LiveTServerSet implements Watcher {
 
     public void halt(ServiceLock lock) throws TException, ThriftSecurityException {
       TabletClientService.Client client =
-          ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, context);
+          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
       try {
         client.halt(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock));
       } finally {
@@ -156,7 +158,7 @@ public class LiveTServerSet implements Watcher {
 
     public void fastHalt(ServiceLock lock) throws TException {
       TabletClientService.Client client =
-          ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, context);
+          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
       try {
         client.fastHalt(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock));
       } finally {
@@ -167,7 +169,7 @@ public class LiveTServerSet implements Watcher {
     public void flush(ServiceLock lock, TableId tableId, byte[] startRow, byte[] endRow)
         throws TException {
       TabletClientService.Client client =
-          ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, context);
+          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
       try {
         client.flush(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock),
             tableId.canonical(), startRow == null ? null : ByteBuffer.wrap(startRow),
@@ -179,7 +181,7 @@ public class LiveTServerSet implements Watcher {
 
     public void chop(ServiceLock lock, KeyExtent extent) throws TException {
       TabletClientService.Client client =
-          ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, context);
+          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
       try {
         client.chop(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock), extent.toThrift());
       } finally {
@@ -190,7 +192,7 @@ public class LiveTServerSet implements Watcher {
     public void splitTablet(KeyExtent extent, Text splitPoint)
         throws TException, ThriftSecurityException, NotServingTabletException {
       TabletClientService.Client client =
-          ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, context);
+          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
       try {
         client.splitTablet(TraceUtil.traceInfo(), context.rpcCreds(), extent.toThrift(),
             ByteBuffer.wrap(splitPoint.getBytes(), 0, splitPoint.getLength()));
@@ -202,7 +204,7 @@ public class LiveTServerSet implements Watcher {
     public void compact(ServiceLock lock, String tableId, byte[] startRow, byte[] endRow)
         throws TException {
       TabletClientService.Client client =
-          ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, context);
+          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
       try {
         client.compact(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock), tableId,
             startRow == null ? null : ByteBuffer.wrap(startRow),
@@ -213,8 +215,8 @@ public class LiveTServerSet implements Watcher {
     }
 
     public boolean isActive(long tid) throws TException {
-      TabletClientService.Client client =
-          ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, context);
+      ClientService.Client client =
+          ThriftUtil.getClient(ThriftClientTypes.CLIENT, address, context);
       try {
         return client.isActive(TraceUtil.traceInfo(), tid);
       } finally {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java b/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java
index 3e28f6b445..90615343b6 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java
@@ -37,6 +37,7 @@ import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
 import org.apache.accumulo.core.manager.balancer.TabletStatisticsImpl;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.spi.balancer.BalancerEnvironment;
 import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
@@ -84,8 +85,7 @@ public class BalancerEnvironmentImpl extends ServiceEnvironmentImpl implements B
       TableId tableId) throws AccumuloException, AccumuloSecurityException {
     log.debug("Scanning tablet server {} for table {}", tabletServerId, tableId);
     try {
-      TabletClientService.Client client = ThriftUtil.getClient(
-          new TabletClientService.Client.Factory(),
+      TabletClientService.Client client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER,
           HostAndPort.fromParts(tabletServerId.getHost(), tabletServerId.getPort()), getContext());
       try {
         return client
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
index 360a5a7c03..4fcd222e38 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
@@ -35,6 +35,7 @@ import org.apache.accumulo.core.manager.balancer.AssignmentParamsImpl;
 import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.spi.balancer.BalancerEnvironment;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
@@ -261,7 +262,8 @@ public abstract class TabletBalancer
   public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId tableId)
       throws ThriftSecurityException, TException {
     log.debug("Scanning tablet server {} for table {}", tserver, tableId);
-    Client client = ThriftUtil.getClient(new Client.Factory(), tserver.getHostAndPort(), context);
+    Client client =
+        ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, tserver.getHostAndPort(), context);
     try {
       return client.getTabletStats(TraceUtil.traceInfo(), context.rpcCreds(), tableId.canonical());
     } catch (TTransportException e) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java
new file mode 100644
index 0000000000..caf044f4d9
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.rpc;
+
+import org.apache.accumulo.core.clientImpl.thrift.ClientService;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
+import org.apache.accumulo.core.compaction.thrift.CompactorService;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.gc.thrift.GCMonitorService;
+import org.apache.accumulo.core.manager.thrift.FateService;
+import org.apache.accumulo.core.manager.thrift.ManagerClientService;
+import org.apache.accumulo.core.replication.thrift.ReplicationCoordinator;
+import org.apache.accumulo.core.replication.thrift.ReplicationServicer;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
+import org.apache.accumulo.core.rpc.ThriftClientTypes.ThriftClientType;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.client.ClientServiceHandler;
+import org.apache.thrift.TBaseProcessor;
+import org.apache.thrift.TMultiplexedProcessor;
+import org.apache.thrift.TProcessor;
+import org.apache.thrift.TServiceClient;
+import org.apache.thrift.TServiceClientFactory;
+
+public class ThriftProcessorTypes {
+
+  private static class ProcessorType<C extends TServiceClient,F extends TServiceClientFactory<C>>
+      extends ThriftClientType<C,F> {
+
+    public ProcessorType(ThriftClientType<C,F> type) {
+      super(type.getServiceName(), type.getClientFactory());
+    }
+
+    private <I,H extends I,P extends TBaseProcessor<?>> TProcessor getTProcessor(
+        Class<P> processorClass, Class<I> interfaceClass, H serviceHandler, ServerContext context,
+        AccumuloConfiguration conf) {
+      I rpcProxy = TraceUtil.wrapService(serviceHandler);
+      if (context.getThriftServerType() == ThriftServerType.SASL) {
+        @SuppressWarnings("unchecked")
+        Class<H> clazz = (Class<H>) serviceHandler.getClass();
+        rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, clazz, conf);
+      }
+      try {
+        return processorClass.getConstructor(interfaceClass).newInstance(rpcProxy);
+      } catch (ReflectiveOperationException e) {
+        throw new IllegalArgumentException("Error constructing TProcessor instance", e);
+      }
+    }
+  }
+
+  private static final ProcessorType<ClientService.Client,ClientService.Client.Factory> CLIENT =
+      new ProcessorType<>(ThriftClientTypes.CLIENT);
+
+  private static final ProcessorType<CompactorService.Client,
+      CompactorService.Client.Factory> COMPACTOR = new ProcessorType<>(ThriftClientTypes.COMPACTOR);
+
+  private static final ProcessorType<CompactionCoordinatorService.Client,
+      CompactionCoordinatorService.Client.Factory> COORDINATOR =
+          new ProcessorType<>(ThriftClientTypes.COORDINATOR);
+
+  private static final ProcessorType<FateService.Client,FateService.Client.Factory> FATE =
+      new ProcessorType<>(ThriftClientTypes.FATE);
+
+  private static final ProcessorType<GCMonitorService.Client,GCMonitorService.Client.Factory> GC =
+      new ProcessorType<>(ThriftClientTypes.GC);
+
+  private static final ProcessorType<ManagerClientService.Client,
+      ManagerClientService.Client.Factory> MANAGER = new ProcessorType<>(ThriftClientTypes.MANAGER);
+
+  private static final ProcessorType<ReplicationCoordinator.Client,
+      ReplicationCoordinator.Client.Factory> REPLICATION_COORDINATOR =
+          new ProcessorType<>(ThriftClientTypes.REPLICATION_COORDINATOR);
+
+  private static final ProcessorType<ReplicationServicer.Client,
+      ReplicationServicer.Client.Factory> REPLICATION_SERVICER =
+          new ProcessorType<>(ThriftClientTypes.REPLICATION_SERVICER);
+
+  private static final ProcessorType<TabletClientService.Client,
+      TabletClientService.Client.Factory> TABLET_SERVER =
+          new ProcessorType<>(ThriftClientTypes.TABLET_SERVER);
+
+  public static TMultiplexedProcessor getCompactorTProcessor(CompactorService.Iface serviceHandler,
+      ServerContext context, AccumuloConfiguration conf) {
+    TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor();
+    muxProcessor.registerProcessor(COMPACTOR.getServiceName(),
+        COMPACTOR.getTProcessor(CompactorService.Processor.class, CompactorService.Iface.class,
+            serviceHandler, context, conf));
+    return muxProcessor;
+  }
+
+  public static TMultiplexedProcessor getCoordinatorTProcessor(
+      CompactionCoordinatorService.Iface serviceHandler, ServerContext context,
+      AccumuloConfiguration conf) {
+    TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor();
+    muxProcessor.registerProcessor(COORDINATOR.getServiceName(),
+        COORDINATOR.getTProcessor(CompactionCoordinatorService.Processor.class,
+            CompactionCoordinatorService.Iface.class, serviceHandler, context, conf));
+    return muxProcessor;
+  }
+
+  public static TMultiplexedProcessor getGcTProcessor(GCMonitorService.Iface serviceHandler,
+      ServerContext context, AccumuloConfiguration conf) {
+    TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor();
+    muxProcessor.registerProcessor(GC.getServiceName(),
+        GC.getTProcessor(GCMonitorService.Processor.class, GCMonitorService.Iface.class,
+            serviceHandler, context, conf));
+    return muxProcessor;
+  }
+
+  public static TMultiplexedProcessor getManagerTProcessor(FateService.Iface fateServiceHandler,
+      ManagerClientService.Iface managerServiceHandler, ServerContext context,
+      AccumuloConfiguration conf) {
+    TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor();
+    muxProcessor.registerProcessor(FATE.getServiceName(), FATE.getTProcessor(
+        FateService.Processor.class, FateService.Iface.class, fateServiceHandler, context, conf));
+    muxProcessor.registerProcessor(MANAGER.getServiceName(),
+        MANAGER.getTProcessor(ManagerClientService.Processor.class,
+            ManagerClientService.Iface.class, managerServiceHandler, context, conf));
+    return muxProcessor;
+  }
+
+  public static TMultiplexedProcessor getReplicationCoordinatorTProcessor(
+      ReplicationCoordinator.Iface serviceHandler, ServerContext context,
+      AccumuloConfiguration conf) {
+    TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor();
+    muxProcessor.registerProcessor(REPLICATION_COORDINATOR.getServiceName(),
+        REPLICATION_COORDINATOR.getTProcessor(ReplicationCoordinator.Processor.class,
+            ReplicationCoordinator.Iface.class, serviceHandler, context, conf));
+    return muxProcessor;
+  }
+
+  public static TMultiplexedProcessor getReplicationClientTProcessor(
+      ReplicationServicer.Iface serviceHandler, ServerContext context, AccumuloConfiguration conf) {
+    TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor();
+    muxProcessor.registerProcessor(REPLICATION_SERVICER.getServiceName(),
+        REPLICATION_SERVICER.getTProcessor(ReplicationServicer.Processor.class,
+            ReplicationServicer.Iface.class, serviceHandler, context, conf));
+    return muxProcessor;
+  }
+
+  public static TMultiplexedProcessor getTabletServerTProcessor(ClientServiceHandler clientHandler,
+      TabletClientService.Iface tserverHandler, ServerContext context, AccumuloConfiguration conf) {
+    TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor();
+    muxProcessor.registerProcessor(CLIENT.getServiceName(), CLIENT.getTProcessor(
+        ClientService.Processor.class, ClientService.Iface.class, clientHandler, context, conf));
+    muxProcessor.registerProcessor(TABLET_SERVER.getServiceName(),
+        TABLET_SERVER.getTProcessor(TabletClientService.Processor.class,
+            TabletClientService.Iface.class, tserverHandler, context, conf));
+    return muxProcessor;
+  }
+
+}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java b/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java
index 982dbaf5dc..717fe9c3e7 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java
@@ -22,6 +22,7 @@ import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
 import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.singletons.SingletonManager;
 import org.apache.accumulo.core.singletons.SingletonManager.Mode;
@@ -201,8 +202,7 @@ public class ECAdmin implements KeywordExecutable {
     HostAndPort address = coordinatorHost.get();
     CompactionCoordinatorService.Client coordinatorClient;
     try {
-      coordinatorClient =
-          ThriftUtil.getClient(new CompactionCoordinatorService.Client.Factory(), address, context);
+      coordinatorClient = ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, address, context);
     } catch (Exception e) {
       throw new IllegalStateException("Unable to get Compaction coordinator at " + address, e);
     }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java b/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
index 50fe982894..134189f1ab 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
@@ -45,6 +45,7 @@ import org.apache.accumulo.core.dataImpl.thrift.TColumn;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TRange;
 import org.apache.accumulo.core.metadata.MetadataServicer;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
@@ -159,7 +160,8 @@ public class VerifyTabletAssignments {
   private static void checkTabletServer(ClientContext context,
       Entry<HostAndPort,List<KeyExtent>> entry, HashSet<KeyExtent> failures)
       throws ThriftSecurityException, TException, NoSuchScanIDException {
-    TabletClientService.Iface client = ThriftUtil.getTServerClient(entry.getKey(), context);
+    TabletClientService.Iface client =
+        ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, entry.getKey(), context);
 
     Map<TKeyExtent,List<TRange>> batch = new TreeMap<>();
 
diff --git a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
index 0245f869ba..615b749400 100644
--- a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
+++ b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
@@ -42,7 +42,6 @@ import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
-import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService.Iface;
 import org.apache.accumulo.core.compaction.thrift.TCompactionState;
 import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate;
 import org.apache.accumulo.core.compaction.thrift.TExternalCompaction;
@@ -55,6 +54,7 @@ import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.metrics.MetricsUtil;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
 import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
@@ -77,12 +77,12 @@ import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.manager.LiveTServerSet;
 import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
 import org.apache.accumulo.server.rpc.ServerAddress;
-import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
 import org.apache.accumulo.server.rpc.TServerUtils;
-import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.rpc.ThriftProcessorTypes;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
 import org.apache.accumulo.server.security.SecurityOperation;
 import org.apache.thrift.TException;
+import org.apache.thrift.TProcessor;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 import org.apache.zookeeper.KeeperException;
@@ -226,13 +226,13 @@ public class CompactionCoordinator extends AbstractServer
    *           host unknown
    */
   protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
-    Iface rpcProxy = TraceUtil.wrapService(this);
-    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
-      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
-          getConfiguration());
+    TProcessor processor = null;
+    try {
+      processor =
+          ThriftProcessorTypes.getCoordinatorTProcessor(this, getContext(), getConfiguration());
+    } catch (Exception e) {
+      throw new RuntimeException("Error creating thrift server processor", e);
     }
-    final CompactionCoordinatorService.Processor<Iface> processor =
-        new CompactionCoordinatorService.Processor<>(rpcProxy);
     Property maxMessageSizeProperty =
         (aconf.get(Property.COMPACTION_COORDINATOR_MAX_MESSAGE_SIZE) != null
             ? Property.COMPACTION_COORDINATOR_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE);
@@ -495,7 +495,7 @@ public class CompactionCoordinator extends AbstractServer
     ServerContext serverContext = getContext();
     TTransport transport =
         serverContext.getTransportPool().getTransport(connection.getAddress(), 0, serverContext);
-    return ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport);
+    return ThriftUtil.createClient(ThriftClientTypes.TABLET_SERVER, transport);
   }
 
   /**
diff --git a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionFinalizer.java b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionFinalizer.java
index da0cef0d18..1efaaf3a41 100644
--- a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionFinalizer.java
+++ b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionFinalizer.java
@@ -45,6 +45,7 @@ import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
@@ -120,8 +121,7 @@ public class CompactionFinalizer {
 
     TabletClientService.Client client = null;
     try {
-      client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), loc.getHostAndPort(),
-          context);
+      client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, loc.getHostAndPort(), context);
       if (ecfs.getFinalState() == FinalState.FINISHED) {
         LOG.debug("Notifying tserver {} that compaction {} has finished.", loc, ecfs);
         client.compactionJobFinished(TraceUtil.traceInfo(), context.rpcCreds(),
diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
index ffb3825a79..feab8f2889 100644
--- a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
+++ b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
@@ -51,7 +51,6 @@ import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
 import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService.Client;
 import org.apache.accumulo.core.compaction.thrift.CompactorService;
-import org.apache.accumulo.core.compaction.thrift.CompactorService.Iface;
 import org.apache.accumulo.core.compaction.thrift.TCompactionState;
 import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate;
 import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
@@ -70,6 +69,7 @@ import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metrics.MetricsProducer;
 import org.apache.accumulo.core.metrics.MetricsUtil;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
@@ -102,13 +102,13 @@ import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceeded
 import org.apache.accumulo.server.compaction.RetryableThriftFunction;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.rpc.ServerAddress;
-import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
 import org.apache.accumulo.server.rpc.TServerUtils;
-import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.rpc.ThriftProcessorTypes;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
 import org.apache.accumulo.server.security.SecurityOperation;
 import org.apache.hadoop.fs.Path;
 import org.apache.thrift.TException;
+import org.apache.thrift.TProcessor;
 import org.apache.thrift.transport.TTransportException;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -138,8 +138,6 @@ public class Compactor extends AbstractServer implements MetricsProducer, Compac
   private static final long TIME_BETWEEN_CANCEL_CHECKS = MINUTES.toMillis(5);
 
   private static final long TEN_MEGABYTES = 10485760;
-  private static final CompactionCoordinatorService.Client.Factory COORDINATOR_CLIENT_FACTORY =
-      new CompactionCoordinatorService.Client.Factory();
 
   protected static final CompactionJobHolder JOB_HOLDER = new CompactionJobHolder();
 
@@ -338,11 +336,13 @@ public class Compactor extends AbstractServer implements MetricsProducer, Compac
    *           host unknown
    */
   protected ServerAddress startCompactorClientService() throws UnknownHostException {
-    Iface rpcProxy = TraceUtil.wrapService(this);
-    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
-      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, getClass(), getConfiguration());
+    TProcessor processor = null;
+    try {
+      processor =
+          ThriftProcessorTypes.getCompactorTProcessor(this, getContext(), getConfiguration());
+    } catch (Exception e) {
+      throw new RuntimeException("Error creating thrift server processor", e);
     }
-    final CompactorService.Processor<Iface> processor = new CompactorService.Processor<>(rpcProxy);
     Property maxMessageSizeProperty = (aconf.get(Property.COMPACTOR_MAX_MESSAGE_SIZE) != null
         ? Property.COMPACTOR_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE);
     ServerAddress sp = TServerUtils.startServer(getContext(), getHostname(),
@@ -524,7 +524,7 @@ public class Compactor extends AbstractServer implements MetricsProducer, Compac
       throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper");
     }
     LOG.trace("CompactionCoordinator address is: {}", coordinatorHost.get());
-    return ThriftUtil.getClient(COORDINATOR_CLIENT_FACTORY, coordinatorHost.get(), getContext());
+    return ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, coordinatorHost.get(), getContext());
   }
 
   /**
diff --git a/server/gc/pom.xml b/server/gc/pom.xml
index d010f7e270..285ac69d8b 100644
--- a/server/gc/pom.xml
+++ b/server/gc/pom.xml
@@ -72,6 +72,10 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-client-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.thrift</groupId>
+      <artifactId>libthrift</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.apache.zookeeper</groupId>
       <artifactId>zookeeper</artifactId>
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index 22ebd4888d..f42cfdf463 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -49,7 +49,6 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.gc.thrift.GCMonitorService.Iface;
-import org.apache.accumulo.core.gc.thrift.GCMonitorService.Processor;
 import org.apache.accumulo.core.gc.thrift.GCStatus;
 import org.apache.accumulo.core.gc.thrift.GcCycleStats;
 import org.apache.accumulo.core.manager.state.tables.TableState;
@@ -87,11 +86,11 @@ import org.apache.accumulo.server.gc.GcVolumeUtil;
 import org.apache.accumulo.server.manager.LiveTServerSet;
 import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.rpc.ServerAddress;
-import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
 import org.apache.accumulo.server.rpc.TServerUtils;
-import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.rpc.ThriftProcessorTypes;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.thrift.TProcessor;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -662,18 +661,13 @@ public class SimpleGarbageCollector extends AbstractServer implements Iface {
   }
 
   private HostAndPort startStatsService() {
-    Iface rpcProxy = TraceUtil.wrapService(this);
-    final Processor<Iface> processor;
-    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
-      Iface tcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, getClass(), getConfiguration());
-      processor = new Processor<>(tcProxy);
-    } else {
-      processor = new Processor<>(rpcProxy);
-    }
-    IntStream port = getConfiguration().getPortStream(Property.GC_PORT);
-    HostAndPort[] addresses = TServerUtils.getHostAndPorts(getHostname(), port);
-    long maxMessageSize = getConfiguration().getAsBytes(Property.GENERAL_MAX_MESSAGE_SIZE);
+
     try {
+      TProcessor processor =
+          ThriftProcessorTypes.getGcTProcessor(this, getContext(), getConfiguration());
+      IntStream port = getConfiguration().getPortStream(Property.GC_PORT);
+      HostAndPort[] addresses = TServerUtils.getHostAndPorts(getHostname(), port);
+      long maxMessageSize = getConfiguration().getAsBytes(Property.GENERAL_MAX_MESSAGE_SIZE);
       ServerAddress server = TServerUtils.startTServer(getConfiguration(),
           getContext().getThriftServerType(), processor, this.getClass().getSimpleName(),
           "GC Monitor Service", 2, ThreadPools.DEFAULT_TIMEOUT_MILLISECS, 1000, maxMessageSize,
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
index 8b543e657e..39abe4ea4e 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
@@ -96,7 +96,7 @@ import org.slf4j.Logger;
 
 class FateServiceHandler implements FateService.Iface {
 
-  protected final Manager manager;
+  private final Manager manager;
   protected static final Logger log = Manager.log;
 
   public FateServiceHandler(Manager manager) {
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
index 45fe3a9115..37286f69b9 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
@@ -67,8 +67,7 @@ import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl;
 import org.apache.accumulo.core.manager.balancer.TServerStatusImpl;
 import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
 import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.manager.thrift.ManagerClientService.Iface;
-import org.apache.accumulo.core.manager.thrift.ManagerClientService.Processor;
+import org.apache.accumulo.core.manager.thrift.ManagerClientService;
 import org.apache.accumulo.core.manager.thrift.ManagerGoalState;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
 import org.apache.accumulo.core.manager.thrift.ManagerState;
@@ -127,9 +126,8 @@ import org.apache.accumulo.server.manager.state.TabletServerState;
 import org.apache.accumulo.server.manager.state.TabletStateStore;
 import org.apache.accumulo.server.rpc.HighlyAvailableServiceWrapper;
 import org.apache.accumulo.server.rpc.ServerAddress;
-import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
 import org.apache.accumulo.server.rpc.TServerUtils;
-import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.rpc.ThriftProcessorTypes;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
 import org.apache.accumulo.server.security.SecurityOperation;
 import org.apache.accumulo.server.security.delegation.AuthenticationTokenKeyManager;
@@ -142,6 +140,7 @@ import org.apache.accumulo.server.util.TableInfoUtil;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.thrift.TException;
+import org.apache.thrift.TProcessor;
 import org.apache.thrift.server.TServer;
 import org.apache.thrift.transport.TTransportException;
 import org.apache.zookeeper.KeeperException;
@@ -279,7 +278,8 @@ public class Manager extends AbstractServer
 
   private Future<Void> upgradeMetadataFuture;
 
-  private ManagerClientServiceHandler clientHandler;
+  private FateServiceHandler fateServiceHandler;
+  private ManagerClientServiceHandler managerClientHandler;
 
   private int assignedOrHosted(TableId tableId) {
     int result = 0;
@@ -1014,20 +1014,18 @@ public class Manager extends AbstractServer
     // ACCUMULO-4424 Put up the Thrift servers before getting the lock as a sign of process health
     // when a hot-standby
     //
+    // Start the Manager's Fate Service
+    fateServiceHandler = new FateServiceHandler(this);
+    managerClientHandler = new ManagerClientServiceHandler(this);
     // Start the Manager's Client service
-    clientHandler = new ManagerClientServiceHandler(this);
     // Ensure that calls before the manager gets the lock fail
-    Iface haProxy = HighlyAvailableServiceWrapper.service(clientHandler, this);
-    Iface rpcProxy = TraceUtil.wrapService(haProxy);
-    final Processor<Iface> processor;
-    if (context.getThriftServerType() == ThriftServerType.SASL) {
-      Iface tcredsProxy = TCredentialsUpdatingWrapper.service(rpcProxy, clientHandler.getClass(),
-          getConfiguration());
-      processor = new Processor<>(tcredsProxy);
-    } else {
-      processor = new Processor<>(rpcProxy);
-    }
+    ManagerClientService.Iface haProxy =
+        HighlyAvailableServiceWrapper.service(managerClientHandler, this);
+
     ServerAddress sa;
+    TProcessor processor = ThriftProcessorTypes.getManagerTProcessor(fateServiceHandler, haProxy,
+        getContext(), getConfiguration());
+
     try {
       sa = TServerUtils.startServer(context, getHostname(), Property.MANAGER_CLIENTPORT, processor,
           "Manager", "Manager Client Service Handler", null, Property.MANAGER_MINTHREADS,
@@ -1368,12 +1366,18 @@ public class Manager extends AbstractServer
     var impl = new org.apache.accumulo.manager.replication.ManagerReplicationCoordinator(this);
     ReplicationCoordinator.Iface haReplicationProxy =
         HighlyAvailableServiceWrapper.service(impl, this);
-    ReplicationCoordinator.Processor<ReplicationCoordinator.Iface> replicationCoordinatorProcessor =
-        new ReplicationCoordinator.Processor<>(TraceUtil.wrapService(haReplicationProxy));
+
+    TProcessor processor = null;
+    try {
+      processor = ThriftProcessorTypes.getReplicationCoordinatorTProcessor(haReplicationProxy,
+          getContext(), getConfiguration());
+    } catch (Exception e) {
+      throw new RuntimeException("Error creating thrift server processor", e);
+    }
+
     ServerAddress replAddress = TServerUtils.startServer(context, getHostname(),
-        Property.MANAGER_REPLICATION_COORDINATOR_PORT, replicationCoordinatorProcessor,
-        "Manager Replication Coordinator", "Replication Coordinator", null,
-        Property.MANAGER_REPLICATION_COORDINATOR_MINTHREADS, null,
+        Property.MANAGER_REPLICATION_COORDINATOR_PORT, processor, "Manager Replication Coordinator",
+        "Replication Coordinator", null, Property.MANAGER_REPLICATION_COORDINATOR_MINTHREADS, null,
         Property.MANAGER_REPLICATION_COORDINATOR_THREADCHECK, Property.GENERAL_MAX_MESSAGE_SIZE);
 
     log.info("Started replication coordinator service at " + replAddress.address);
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
index feb27797a0..1e2e5886de 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
@@ -96,15 +96,15 @@ import org.slf4j.LoggerFactory;
 
 import com.google.protobuf.InvalidProtocolBufferException;
 
-public class ManagerClientServiceHandler extends FateServiceHandler
-    implements ManagerClientService.Iface {
+public class ManagerClientServiceHandler implements ManagerClientService.Iface {
 
   private static final Logger log = Manager.log;
   private static final Logger drainLog =
       LoggerFactory.getLogger("org.apache.accumulo.manager.ManagerDrainImpl");
+  private final Manager manager;
 
   protected ManagerClientServiceHandler(Manager manager) {
-    super(manager);
+    this.manager = manager;
   }
 
   @Override
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/LoadFiles.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/LoadFiles.java
index 827631e524..32b9006465 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/LoadFiles.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/LoadFiles.java
@@ -44,6 +44,7 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.master.thrift.BulkImportState;
 import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.HostAndPort;
@@ -166,7 +167,8 @@ class LoadFiles extends ManagerRepo {
               long timeInMillis =
                   manager.getConfiguration().getTimeInMillis(Property.MANAGER_BULK_TIMEOUT);
               server = servers[random.nextInt(servers.length)].getHostAndPort();
-              client = ThriftUtil.getTServerClient(server, manager.getContext(), timeInMillis);
+              client = ThriftUtil.getClient(ThriftClientTypes.CLIENT, server, manager.getContext(),
+                  timeInMillis);
               List<String> attempt1 = Collections.singletonList(file);
               log.debug("Asking " + server + " to bulk import " + file);
               List<String> fail =
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java
index 384b6c05db..1e5327c58c 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java
@@ -52,6 +52,7 @@ import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
@@ -169,7 +170,8 @@ class LoadFiles extends ManagerRepo {
 
           TabletClientService.Client client = null;
           try {
-            client = ThriftUtil.getTServerClient(server, manager.getContext(), timeInMillis);
+            client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, server,
+                manager.getContext(), timeInMillis);
             client.loadFiles(TraceUtil.traceInfo(), manager.getContext().rpcCreds(), tid,
                 bulkDir.toString(), tabletFiles, setTime);
           } catch (TException ex) {
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
index 9f22197471..d6403bc220 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
@@ -58,6 +58,7 @@ import org.apache.accumulo.core.manager.thrift.ManagerClientService;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
 import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
@@ -427,7 +428,7 @@ public class Monitor extends AbstractServer implements HighlyAvailableService {
         address = new ServerServices(new String(zk.getData(path + "/" + locks.get(0)), UTF_8))
             .getAddress(Service.GC_CLIENT);
         GCMonitorService.Client client =
-            ThriftUtil.getClient(new GCMonitorService.Client.Factory(), address, context);
+            ThriftUtil.getClient(ThriftClientTypes.GC, address, context);
         try {
           result = client.getStatus(TraceUtil.traceInfo(), context.rpcCreds());
         } finally {
@@ -675,8 +676,8 @@ public class Monitor extends AbstractServer implements HighlyAvailableService {
   private CompactionCoordinatorService.Client getCoordinator(HostAndPort address) {
     if (coordinatorClient == null) {
       try {
-        coordinatorClient = ThriftUtil.getClient(new CompactionCoordinatorService.Client.Factory(),
-            address, getContext());
+        coordinatorClient =
+            ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, address, getContext());
       } catch (Exception e) {
         log.error("Unable to get Compaction coordinator at {}", address);
         throw new IllegalStateException(coordinatorMissingMsg, e);
@@ -691,7 +692,7 @@ public class Monitor extends AbstractServer implements HighlyAvailableService {
       final HostAndPort parsedServer = HostAndPort.fromString(server);
       Client tserver = null;
       try {
-        tserver = ThriftUtil.getTServerClient(parsedServer, context);
+        tserver = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context);
         List<ActiveScan> scans = tserver.getActiveScans(null, context.rpcCreds());
         allScans.put(parsedServer, new ScanStats(scans));
         scansFetchedNanos = System.nanoTime();
@@ -719,7 +720,7 @@ public class Monitor extends AbstractServer implements HighlyAvailableService {
       final HostAndPort parsedServer = HostAndPort.fromString(server);
       Client tserver = null;
       try {
-        tserver = ThriftUtil.getTServerClient(parsedServer, context);
+        tserver = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context);
         var compacts = tserver.getActiveCompactions(null, context.rpcCreds());
         allCompactions.put(parsedServer, new CompactionStats(compacts));
         compactsFetchedNanos = System.nanoTime();
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
index d8c570c720..4bf96425e1 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
@@ -42,6 +42,7 @@ import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
 import org.apache.accumulo.core.master.thrift.RecoveryStatus;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.tabletserver.thrift.ActionStats;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
@@ -184,7 +185,7 @@ public class TabletServerResource {
     try {
       ClientContext context = monitor.getContext();
       TabletClientService.Client client =
-          ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, context);
+          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
       try {
         for (String tableId : mmi.tableMap.keySet()) {
           tsStats.addAll(client.getTabletStats(TraceUtil.traceInfo(), context.rpcCreds(), tableId));
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftClientHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java
similarity index 98%
rename from server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftClientHandler.java
rename to server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java
index 9802e0c99c..a03fa9ae11 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftClientHandler.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java
@@ -121,7 +121,7 @@ import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.server.client.ClientServiceHandler;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.compaction.CompactionInfo;
 import org.apache.accumulo.server.compaction.FileCompactor;
 import org.apache.accumulo.server.conf.TableConfiguration;
@@ -129,6 +129,8 @@ import org.apache.accumulo.server.data.ServerMutation;
 import org.apache.accumulo.server.fs.TooManyFilesException;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher;
 import org.apache.accumulo.tserver.ConditionCheckerContext.ConditionChecker;
 import org.apache.accumulo.tserver.RowLocks.RowLock;
@@ -162,21 +164,26 @@ import com.google.common.collect.Collections2;
 import io.opentelemetry.api.trace.Span;
 import io.opentelemetry.context.Scope;
 
-public class ThriftClientHandler extends ClientServiceHandler implements TabletClientService.Iface {
+public class TabletClientHandler implements TabletClientService.Iface {
 
-  private static final Logger log = LoggerFactory.getLogger(ThriftClientHandler.class);
+  private static final Logger log = LoggerFactory.getLogger(TabletClientHandler.class);
   private final long MAX_TIME_TO_WAIT_FOR_SCAN_RESULT_MILLIS;
   private static final long RECENTLY_SPLIT_MILLIES = MINUTES.toMillis(1);
   private final TabletServer server;
+  protected final TransactionWatcher watcher;
+  protected final ServerContext context;
+  protected final SecurityOperation security;
   private final WriteTracker writeTracker = new WriteTracker();
   private final RowLocks rowLocks = new RowLocks();
 
-  public ThriftClientHandler(TabletServer server) {
-    super(server.getContext(), new TransactionWatcher(server.getContext()));
+  public TabletClientHandler(TabletServer server, TransactionWatcher watcher) {
+    this.context = server.getContext();
+    this.watcher = watcher;
+    this.security = AuditedSecurityOperation.getInstance(context);
     this.server = server;
     MAX_TIME_TO_WAIT_FOR_SCAN_RESULT_MILLIS = server.getContext().getConfiguration()
         .getTimeInMillis(Property.TSERV_SCAN_RESULTS_MAX_TIMEOUT);
-    log.debug("{} created", ThriftClientHandler.class.getName());
+    log.debug("{} created", TabletClientHandler.class.getName());
   }
 
   @Override
@@ -190,7 +197,7 @@ public class ThriftClientHandler extends ClientServiceHandler implements TabletC
     }
 
     try {
-      return transactionWatcher.run(Constants.BULK_ARBITRATOR_TYPE, tid, () -> {
+      return watcher.run(Constants.BULK_ARBITRATOR_TYPE, tid, () -> {
         List<TKeyExtent> failures = new ArrayList<>();
 
         for (Entry<TKeyExtent,Map<String,MapFileInfo>> entry : files.entrySet()) {
@@ -236,7 +243,7 @@ public class ThriftClientHandler extends ClientServiceHandler implements TabletC
           SecurityErrorCode.PERMISSION_DENIED);
     }
 
-    transactionWatcher.runQuietly(Constants.BULK_ARBITRATOR_TYPE, tid, () -> {
+    watcher.runQuietly(Constants.BULK_ARBITRATOR_TYPE, tid, () -> {
       tabletImports.forEach((tke, fileMap) -> {
         Map<TabletFile,MapFileInfo> newFileMap = new HashMap<>();
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 578d689504..5d306db6c9 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -82,12 +82,10 @@ import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.core.metrics.MetricsUtil;
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Processor;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.ComparablePair;
 import org.apache.accumulo.core.util.Halt;
@@ -112,6 +110,7 @@ import org.apache.accumulo.server.GarbageCollectionLogger;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.TabletLevel;
+import org.apache.accumulo.server.client.ClientServiceHandler;
 import org.apache.accumulo.server.compaction.CompactionWatcher;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
@@ -121,9 +120,8 @@ import org.apache.accumulo.server.log.WalStateManager;
 import org.apache.accumulo.server.log.WalStateManager.WalMarkerException;
 import org.apache.accumulo.server.manager.recovery.RecoveryPath;
 import org.apache.accumulo.server.rpc.ServerAddress;
-import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
 import org.apache.accumulo.server.rpc.TServerUtils;
-import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.rpc.ThriftProcessorTypes;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
 import org.apache.accumulo.server.security.SecurityOperation;
 import org.apache.accumulo.server.security.SecurityUtil;
@@ -133,6 +131,7 @@ import org.apache.accumulo.server.util.FileSystemMonitor;
 import org.apache.accumulo.server.util.ServerBulkImportStatus;
 import org.apache.accumulo.server.util.time.RelativeTime;
 import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
+import org.apache.accumulo.server.zookeeper.TransactionWatcher;
 import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
 import org.apache.accumulo.tserver.TabletStatsKeeper.Operation;
 import org.apache.accumulo.tserver.compactions.Compactable;
@@ -398,7 +397,8 @@ public class TabletServer extends AbstractServer {
 
   private final AtomicLong totalQueuedMutationSize = new AtomicLong(0);
   private final ReentrantLock recoveryLock = new ReentrantLock(true);
-  private ThriftClientHandler clientHandler;
+  private ClientServiceHandler clientHandler;
+  private TabletClientHandler thriftClientHandler;
   private final ServerBulkImportStatus bulkImportStatus = new ServerBulkImportStatus();
   private CompactionManager compactionManager;
 
@@ -582,16 +582,20 @@ public class TabletServer extends AbstractServer {
         return null;
       }
       // log.info("Listener API to manager has been opened");
-      return ThriftUtil.getClient(new ManagerClientService.Client.Factory(), address, getContext());
+      return ThriftUtil.getClient(ThriftClientTypes.MANAGER, address, getContext());
     } catch (Exception e) {
       log.warn("Issue with managerConnection (" + address + ") " + e, e);
     }
     return null;
   }
 
+  protected ClientServiceHandler newClientHandler(TransactionWatcher watcher) {
+    return new ClientServiceHandler(context, watcher);
+  }
+
   // exists to be overridden in tests
-  protected ThriftClientHandler getThriftClientHandler() {
-    return new ThriftClientHandler(this);
+  protected TabletClientHandler newTabletClientHandler(TransactionWatcher watcher) {
+    return new TabletClientHandler(this, watcher);
   }
 
   private void returnManagerConnection(ManagerClientService.Client client) {
@@ -600,16 +604,12 @@ public class TabletServer extends AbstractServer {
 
   private HostAndPort startTabletClientService() throws UnknownHostException {
     // start listening for client connection last
-    clientHandler = getThriftClientHandler();
-    Iface rpcProxy = TraceUtil.wrapService(clientHandler);
-    final Processor<Iface> processor;
-    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
-      Iface tcredProxy = TCredentialsUpdatingWrapper.service(rpcProxy, ThriftClientHandler.class,
-          getConfiguration());
-      processor = new Processor<>(tcredProxy);
-    } else {
-      processor = new Processor<>(rpcProxy);
-    }
+    TransactionWatcher watcher = new TransactionWatcher(context);
+    clientHandler = newClientHandler(watcher);
+    thriftClientHandler = newTabletClientHandler(watcher);
+
+    TProcessor processor = ThriftProcessorTypes.getTabletServerTProcessor(clientHandler,
+        thriftClientHandler, getContext(), getConfiguration());
     HostAndPort address = startServer(getConfiguration(), clientAddress.getHost(), processor);
     log.info("address = {}", address);
     return address;
@@ -619,11 +619,13 @@ public class TabletServer extends AbstractServer {
   private void startReplicationService() throws UnknownHostException {
     final var handler =
         new org.apache.accumulo.tserver.replication.ReplicationServicerHandler(this);
-    ReplicationServicer.Iface rpcProxy = TraceUtil.wrapService(handler);
-    ReplicationServicer.Iface repl =
-        TCredentialsUpdatingWrapper.service(rpcProxy, handler.getClass(), getConfiguration());
-    ReplicationServicer.Processor<ReplicationServicer.Iface> processor =
-        new ReplicationServicer.Processor<>(repl);
+    TProcessor processor = null;
+    try {
+      processor = ThriftProcessorTypes.getReplicationClientTProcessor(handler, getContext(),
+          getConfiguration());
+    } catch (Exception e) {
+      throw new RuntimeException("Error creating thrift server processor", e);
+    }
     Property maxMessageSizeProperty =
         getConfiguration().get(Property.TSERV_MAX_MESSAGE_SIZE) != null
             ? Property.TSERV_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE;
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
index d029070883..56412d8deb 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
@@ -35,7 +35,7 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.ManagerClient;
+import org.apache.accumulo.core.clientImpl.FateManagerClient;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.fate.AdminUtil;
@@ -272,7 +272,7 @@ public class FateCommand extends Command {
       boolean cancelTx =
           line != null && (line.equalsIgnoreCase("y") || line.equalsIgnoreCase("yes"));
       if (cancelTx) {
-        boolean cancelled = ManagerClient.cancelFateOperation(context, txid);
+        boolean cancelled = FateManagerClient.cancelFateOperation(context, txid);
         if (cancelled) {
           shellState.getWriter()
               .println("FaTE transaction " + txid + " was cancelled or already completed.");
diff --git a/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java b/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java
index 678b645ef5..1487f5b247 100644
--- a/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.util.HostAndPort;
@@ -126,8 +127,8 @@ public class TotalQueuedIT extends ConfigurableMacBase {
   private long getSyncs(AccumuloClient c) throws Exception {
     ServerContext context = getServerContext();
     for (String address : c.instanceOperations().getTabletServers()) {
-      TabletClientService.Client client =
-          ThriftUtil.getTServerClient(HostAndPort.fromString(address), context);
+      TabletClientService.Client client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER,
+          HostAndPort.fromString(address), context);
       TabletServerStatus status = client.getTabletServerStatus(null, context.rpcCreds());
       return status.syncs;
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java
index 2cac5fa7d9..1a0bf32e04 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java
@@ -19,8 +19,9 @@
 package org.apache.accumulo.test.compaction;
 
 import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.zookeeper.TransactionWatcher;
+import org.apache.accumulo.tserver.TabletClientHandler;
 import org.apache.accumulo.tserver.TabletServer;
-import org.apache.accumulo.tserver.ThriftClientHandler;
 
 public class ExternalCompactionTServer extends TabletServer {
 
@@ -29,8 +30,8 @@ public class ExternalCompactionTServer extends TabletServer {
   }
 
   @Override
-  protected ThriftClientHandler getThriftClientHandler() {
-    return new NonCommittingExternalCompactionThriftClientHandler(this);
+  protected TabletClientHandler newTabletClientHandler(TransactionWatcher watcher) {
+    return new NonCommittingExternalCompactionTabletClientHandler(this, watcher);
   }
 
   public static void main(String[] args) throws Exception {
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java
index 71476d9d3c..efc6d18626 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java
@@ -58,6 +58,7 @@ import org.apache.accumulo.core.metadata.schema.ExternalCompactionFinalState;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner;
 import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher;
@@ -245,8 +246,8 @@ public class ExternalCompactionTestUtils {
     if (coordinatorHost.isEmpty()) {
       throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper");
     }
-    CompactionCoordinatorService.Client client = ThriftUtil.getClient(
-        new CompactionCoordinatorService.Client.Factory(), coordinatorHost.get(), context);
+    CompactionCoordinatorService.Client client =
+        ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, coordinatorHost.get(), context);
     try {
       TExternalCompactionList running =
           client.getRunningCompactions(TraceUtil.traceInfo(), context.rpcCreds());
@@ -263,8 +264,8 @@ public class ExternalCompactionTestUtils {
     if (coordinatorHost.isEmpty()) {
       throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper");
     }
-    CompactionCoordinatorService.Client client = ThriftUtil.getClient(
-        new CompactionCoordinatorService.Client.Factory(), coordinatorHost.get(), context);
+    CompactionCoordinatorService.Client client =
+        ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, coordinatorHost.get(), context);
     try {
       TExternalCompactionList completed =
           client.getCompletedCompactions(TraceUtil.traceInfo(), context.rpcCreds());
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/NonCommittingExternalCompactionThriftClientHandler.java b/test/src/main/java/org/apache/accumulo/test/compaction/NonCommittingExternalCompactionTabletClientHandler.java
similarity index 83%
rename from test/src/main/java/org/apache/accumulo/test/compaction/NonCommittingExternalCompactionThriftClientHandler.java
rename to test/src/main/java/org/apache/accumulo/test/compaction/NonCommittingExternalCompactionTabletClientHandler.java
index bbdbfde661..2f2b9a216a 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/NonCommittingExternalCompactionThriftClientHandler.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/NonCommittingExternalCompactionTabletClientHandler.java
@@ -23,15 +23,17 @@ import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.server.zookeeper.TransactionWatcher;
+import org.apache.accumulo.tserver.TabletClientHandler;
 import org.apache.accumulo.tserver.TabletServer;
-import org.apache.accumulo.tserver.ThriftClientHandler;
 import org.apache.thrift.TException;
 
-public class NonCommittingExternalCompactionThriftClientHandler extends ThriftClientHandler
+public class NonCommittingExternalCompactionTabletClientHandler extends TabletClientHandler
     implements TabletClientService.Iface {
 
-  public NonCommittingExternalCompactionThriftClientHandler(TabletServer server) {
-    super(server);
+  public NonCommittingExternalCompactionTabletClientHandler(TabletServer server,
+      TransactionWatcher watcher) {
+    super(server, watcher);
   }
 
   @Override
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java
index 72cb18394c..145aa114f7 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java
@@ -56,6 +56,7 @@ import org.apache.accumulo.core.dataImpl.thrift.MapFileInfo;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
@@ -308,7 +309,8 @@ public class BulkFailureIT extends AccumuloClusterHarness {
         .fromString(locator.locateTablet(context, new Text(""), false, true).tablet_location);
 
     long timeInMillis = context.getConfiguration().getTimeInMillis(Property.TSERV_BULK_TIMEOUT);
-    TabletClientService.Iface client = ThriftUtil.getTServerClient(location, context, timeInMillis);
+    TabletClientService.Iface client =
+        ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, context, timeInMillis);
     return client;
   }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
index 36e927da22..b02761e54f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
@@ -47,7 +47,6 @@ import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.rpc.ServerAddress;
 import org.apache.accumulo.server.rpc.TServerUtils;
 import org.apache.accumulo.server.rpc.ThriftServerType;
-import org.apache.accumulo.server.zookeeper.TransactionWatcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -59,17 +58,13 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
  */
 public class ZombieTServer {
 
-  public static class ThriftClientHandler
-      extends org.apache.accumulo.test.performance.NullTserver.ThriftClientHandler {
+  public static class ZombieTServerThriftClientHandler
+      extends org.apache.accumulo.test.performance.NullTserver.NullTServerThriftClientHandler {
 
     int statusCount = 0;
 
     boolean halted = false;
 
-    ThriftClientHandler(ServerContext context, TransactionWatcher watcher) {
-      super(context, watcher);
-    }
-
     @Override
     public synchronized void fastHalt(TInfo tinfo, TCredentials credentials, String lock) {
       halted = true;
@@ -102,8 +97,7 @@ public class ZombieTServer {
   public static void main(String[] args) throws Exception {
     int port = random.nextInt(30000) + 2000;
     var context = new ServerContext(SiteConfiguration.auto());
-    TransactionWatcher watcher = new TransactionWatcher(context);
-    final ThriftClientHandler tch = new ThriftClientHandler(context, watcher);
+    final ZombieTServerThriftClientHandler tch = new ZombieTServerThriftClientHandler();
     Processor<Iface> processor = new Processor<>(tch);
     ServerAddress serverPort =
         TServerUtils.startTServer(context.getConfiguration(), ThriftServerType.CUSTOM_HS_HA,
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
index f557c373e3..ac754e63d2 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
@@ -74,13 +74,11 @@ import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.client.ClientServiceHandler;
 import org.apache.accumulo.server.manager.state.Assignment;
 import org.apache.accumulo.server.manager.state.MetaDataTableScanner;
 import org.apache.accumulo.server.manager.state.TabletStateStore;
 import org.apache.accumulo.server.rpc.TServerUtils;
 import org.apache.accumulo.server.rpc.ThriftServerType;
-import org.apache.accumulo.server.zookeeper.TransactionWatcher;
 import org.apache.thrift.TException;
 
 import com.beust.jcommander.Parameter;
@@ -92,15 +90,10 @@ import com.beust.jcommander.Parameter;
  */
 public class NullTserver {
 
-  public static class ThriftClientHandler extends ClientServiceHandler
-      implements TabletClientService.Iface {
+  public static class NullTServerThriftClientHandler implements TabletClientService.Iface {
 
     private long updateSession = 1;
 
-    public ThriftClientHandler(ServerContext context, TransactionWatcher watcher) {
-      super(context, watcher);
-    }
-
     @Override
     public long startUpdate(TInfo tinfo, TCredentials credentials, TDurability durability) {
       return updateSession++;
@@ -324,8 +317,7 @@ public class NullTserver {
         (int) DefaultConfiguration.getInstance().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
     var siteConfig = SiteConfiguration.auto();
     ServerContext context = ServerContext.override(siteConfig, opts.iname, opts.keepers, zkTimeOut);
-    TransactionWatcher watcher = new TransactionWatcher(context);
-    ThriftClientHandler tch = new ThriftClientHandler(context, watcher);
+    NullTServerThriftClientHandler tch = new NullTServerThriftClientHandler();
     Processor<Iface> processor = new Processor<>(tch);
     TServerUtils.startTServer(context.getConfiguration(), ThriftServerType.CUSTOM_HS_HA, processor,
         "NullTServer", "null tserver", 2, ThreadPools.DEFAULT_TIMEOUT_MILLISECS, 1000,
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java b/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
index a23bd13f22..ca46098705 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
@@ -52,6 +52,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationTable;
+import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
@@ -393,7 +394,7 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacB
     // Get the active WALs from that server
     log.info("Fetching active WALs from {}", tserver);
 
-    Client cli = ThriftUtil.getTServerClient(tserver, context);
+    Client cli = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, tserver, context);
     List<String> activeWalsForTserver =
         cli.getActiveLogs(TraceUtil.traceInfo(), context.rpcCreds());