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 dca3ae8  Set default UncaughtExceptionHandler in client, enable 
override (#2554)
dca3ae8 is described below

commit dca3ae8d056a24510aa9d86347d9d6338613c4b3
Author: Dave Marion <dlmar...@apache.org>
AuthorDate: Tue Mar 15 07:29:39 2022 -0400

    Set default UncaughtExceptionHandler in client, enable override (#2554)
    
    When an ExecutorService's task or Thread in Accumulo encounter an
    unhandled exception the default UncaughtExceptionHandler will be
    invoked. An Exception is logged, while an Error is logged and then
    the VM is terminated. This change sets the UncaughtExceptionHandler
    to one that logs the exception/error in the client and does not terminate
    the client VM. It also allows clients to supply their own
    UncaughtExceptionHandler implementation.
    
    Closes #2331 and supersedes #2346
---
 .../DefaultContextClassLoaderFactory.java          |   4 +-
 .../accumulo/core/client/AccumuloClient.java       |  11 ++
 .../accumulo/core/client/ZooKeeperInstance.java    |   2 +-
 .../accumulo/core/clientImpl/ClientContext.java    |  74 ++++++++++++-
 .../core/clientImpl/ConditionalWriterImpl.java     |  10 +-
 .../core/clientImpl/InstanceOperationsImpl.java    |   3 +-
 .../accumulo/core/clientImpl/ScannerIterator.java  |  21 ++--
 .../core/clientImpl/TableOperationsImpl.java       |   3 +-
 .../core/clientImpl/TabletServerBatchReader.java   |   3 +-
 .../core/clientImpl/TabletServerBatchWriter.java   |  12 +-
 .../accumulo/core/clientImpl/bulk/BulkImport.java  |   6 +-
 .../accumulo/core/file/BloomFilterLayer.java       |   4 +-
 .../file/blockfile/cache/lru/LruBlockCache.java    |   4 +-
 .../blockfile/cache/tinylfu/TinyLfuBlockCache.java |   4 +-
 .../util/compaction/ExternalCompactionUtil.java    |   8 +-
 .../util/ratelimit/SharedRateLimiterFactory.java   |   3 +-
 .../core/util/threads/NamedThreadFactory.java      |  17 +--
 .../accumulo/core/util/threads/ThreadPools.java    | 122 ++++++++++++---------
 .../apache/accumulo/core/util/threads/Threads.java |  31 ++++--
 .../main/java/org/apache/accumulo/fate/Fate.java   |   7 +-
 .../clientImpl/TabletServerBatchReaderTest.java    |   3 +
 .../core/file/rfile/MultiThreadedRFileTest.java    |   4 +-
 .../org/apache/accumulo/server/ServerContext.java  |   7 +-
 .../accumulo/server/client/BulkImporter.java       |  11 +-
 .../accumulo/server/fs/VolumeManagerImpl.java      |   3 +-
 .../accumulo/server/problems/ProblemReports.java   |   4 +-
 .../apache/accumulo/server/rpc/TServerUtils.java   |   8 +-
 .../accumulo/server/util/FileSystemMonitor.java    |   4 +-
 .../server/util/RemoveEntriesForMissingFiles.java  |   3 +-
 .../server/util/VerifyTabletAssignments.java       |   3 +-
 .../coordinator/CompactionCoordinator.java         |   8 +-
 .../accumulo/coordinator/CompactionFinalizer.java  |   8 +-
 .../org/apache/accumulo/compactor/Compactor.java   |   6 +-
 .../apache/accumulo/gc/SimpleGarbageCollector.java |   4 +-
 .../java/org/apache/accumulo/manager/Manager.java  |   4 +-
 .../org/apache/accumulo/manager/ManagerTime.java   |   7 +-
 .../manager/metrics/ReplicationMetrics.java        |   4 +-
 .../accumulo/manager/metrics/fate/FateMetrics.java |   4 +-
 .../accumulo/manager/recovery/RecoveryManager.java |   3 +-
 .../manager/tableOps/bulkVer1/BulkImport.java      |   2 +-
 .../manager/tableOps/bulkVer1/LoadFiles.java       |   4 +-
 .../manager/upgrade/UpgradeCoordinator.java        |   5 +-
 .../org/apache/accumulo/tserver/TabletServer.java  |  10 +-
 .../tserver/TabletServerResourceManager.java       |  45 ++++----
 .../tserver/compactions/CompactionService.java     |   4 +-
 .../compactions/InternalCompactionExecutor.java    |   4 +-
 .../org/apache/accumulo/tserver/log/LogSorter.java |   4 +-
 .../accumulo/tserver/log/TabletServerLogger.java   |   3 +-
 .../metrics/CompactionExecutorsMetrics.java        |   4 +-
 .../tserver/replication/AccumuloReplicaSystem.java |   3 +-
 .../accumulo/tserver/session/SessionManager.java   |   4 +-
 .../accumulo/test/BalanceWithOfflineTableIT.java   |   3 +-
 .../test/functional/BatchWriterFlushIT.java        |   4 +-
 53 files changed, 331 insertions(+), 215 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java
 
b/core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java
index 6baaa64..7c77845 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java
@@ -63,7 +63,9 @@ public class DefaultContextClassLoaderFactory implements 
ContextClassLoaderFacto
 
   private static void startCleanupThread(final AccumuloConfiguration conf,
       final Supplier<Map<String,String>> contextConfigSupplier) {
-    ScheduledFuture<?> future = 
ThreadPools.createGeneralScheduledExecutorService(conf)
+    ScheduledFuture<?> future = ThreadPools.getClientThreadPools((t, e) -> {
+      LOG.error("context classloader cleanup thread has failed.", e);
+    }).createGeneralScheduledExecutorService(conf)
         .scheduleWithFixedDelay(Threads.createNamedRunnable(className + 
"-cleanup", () -> {
           LOG.trace("{}-cleanup thread, properties: {}", className, conf);
           Set<String> contextsInUse = 
contextConfigSupplier.get().keySet().stream()
diff --git 
a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java 
b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
index c677d01..39f8886 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.core.client;
 
+import java.lang.Thread.UncaughtExceptionHandler;
 import java.net.URL;
 import java.nio.file.Path;
 import java.util.Properties;
@@ -347,6 +348,16 @@ public interface AccumuloClient extends AutoCloseable {
   interface ClientFactory<T> {
 
     /**
+     * Override default handling of uncaught exceptions in client threads
+     *
+     * @param ueh
+     *          UncaughtExceptionHandler implementation
+     * @return AccumuloClient or Properties
+     * @since 2.1.0
+     */
+    ClientFactory<T> withUncaughtExceptionHandler(UncaughtExceptionHandler 
ueh);
+
+    /**
      * Builds AccumuloClient or client Properties
      *
      * @return AccumuloClient or Properties
diff --git 
a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java 
b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index 7e81280..1b7af13 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -192,7 +192,7 @@ public class ZooKeeperInstance implements Instance {
     ClientInfo info = new ClientInfoImpl(properties, token);
     AccumuloConfiguration serverConf = 
ClientConfConverter.toAccumuloConf(properties);
     return new org.apache.accumulo.core.clientImpl.ConnectorImpl(
-        new ClientContext(SingletonReservation.noop(), info, serverConf));
+        new ClientContext(SingletonReservation.noop(), info, serverConf, 
null));
   }
 
   @Override
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index 88fab7b..5c0abf8 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -25,6 +25,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static 
org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
 
+import java.lang.Thread.UncaughtExceptionHandler;
 import java.net.URL;
 import java.nio.file.Path;
 import java.util.Collections;
@@ -32,6 +33,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Properties;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
 import java.util.function.Function;
 import java.util.function.Supplier;
 
@@ -61,6 +66,7 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.KeyValue;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.manager.state.tables.TableState;
@@ -78,6 +84,8 @@ import org.apache.accumulo.core.singletons.SingletonManager;
 import org.apache.accumulo.core.singletons.SingletonReservation;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.tables.TableZooHelper;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
@@ -130,6 +138,9 @@ public class ClientContext implements AccumuloClient {
   @SuppressWarnings("deprecation")
   private org.apache.accumulo.core.client.admin.ReplicationOperations 
replicationops = null;
   private final SingletonReservation singletonReservation;
+  private final ThreadPools clientThreadPools;
+  private ThreadPoolExecutor cleanupThreadPool;
+  private ThreadPoolExecutor scannerReadaheadPool;
 
   private void ensureOpen() {
     if (closed) {
@@ -149,7 +160,7 @@ public class ClientContext implements AccumuloClient {
    * {@link ClientBuilderImpl#buildClient}
    */
   public ClientContext(SingletonReservation reservation, ClientInfo info,
-      AccumuloConfiguration serverConf) {
+      AccumuloConfiguration serverConf, UncaughtExceptionHandler ueh) {
     this.info = info;
     this.hadoopConf = info.getHadoopConf();
     zooReader = new ZooReader(info.getZooKeepers(), 
info.getZooKeepersSessionTimeOut());
@@ -164,6 +175,18 @@ public class ClientContext implements AccumuloClient {
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    if (ueh == Threads.UEH) {
+      clientThreadPools = ThreadPools.getServerThreadPools();
+    } else {
+      // Provide a default UEH that just logs the error
+      if (ueh == null) {
+        clientThreadPools = ThreadPools.getClientThreadPools((t, e) -> {
+          log.error("Caught an Exception in client background thread: {}. 
Thread is dead.", t, e);
+        });
+      } else {
+        clientThreadPools = ThreadPools.getClientThreadPools(ueh);
+      }
+    }
   }
 
   public Ample getAmple() {
@@ -171,6 +194,33 @@ public class ClientContext implements AccumuloClient {
     return new AmpleImpl(this);
   }
 
+  public synchronized Future<List<KeyValue>>
+      submitScannerReadAheadTask(Callable<List<KeyValue>> c) {
+    ensureOpen();
+    if (scannerReadaheadPool == null) {
+      scannerReadaheadPool = clientThreadPools.createThreadPool(0, 
Integer.MAX_VALUE, 3L, SECONDS,
+          "Accumulo scanner read ahead thread", new SynchronousQueue<>(), 
true);
+    }
+    return scannerReadaheadPool.submit(c);
+  }
+
+  public synchronized void executeCleanupTask(Runnable r) {
+    ensureOpen();
+    if (cleanupThreadPool == null) {
+      cleanupThreadPool = clientThreadPools.createFixedThreadPool(1, 3, 
SECONDS,
+          "Conditional Writer Cleanup Thread", true);
+    }
+    this.cleanupThreadPool.execute(r);
+  }
+
+  /**
+   * @return ThreadPools instance optionally configured with client 
UncaughtExceptionHandler
+   */
+  public ThreadPools threadPools() {
+    ensureOpen();
+    return clientThreadPools;
+  }
+
   /**
    * Retrieve the credentials used to construct this context
    */
@@ -728,7 +778,7 @@ public class ClientContext implements AccumuloClient {
   }
 
   @Override
-  public void close() {
+  public synchronized void close() {
     closed = true;
     if (thriftTransportPool != null) {
       thriftTransportPool.shutdown();
@@ -736,6 +786,12 @@ public class ClientContext implements AccumuloClient {
     if (tableZooHelper != null) {
       tableZooHelper.close();
     }
+    if (scannerReadaheadPool != null) {
+      scannerReadaheadPool.shutdownNow(); // abort all tasks, client is 
shutting down
+    }
+    if (cleanupThreadPool != null) {
+      cleanupThreadPool.shutdown(); // wait for shutdown tasks to execute
+    }
     singletonReservation.close();
   }
 
@@ -746,6 +802,7 @@ public class ClientContext implements AccumuloClient {
     private Properties properties = new Properties();
     private AuthenticationToken token = null;
     private final Function<ClientBuilderImpl<T>,T> builderFunction;
+    private UncaughtExceptionHandler ueh = null;
 
     public ClientBuilderImpl(Function<ClientBuilderImpl<T>,T> builderFunction) 
{
       this.builderFunction = builderFunction;
@@ -760,6 +817,10 @@ public class ClientContext implements AccumuloClient {
       return new ClientInfoImpl(properties);
     }
 
+    private UncaughtExceptionHandler getUncaughtExceptionHandler() {
+      return ueh;
+    }
+
     @Override
     public T build() {
       return builderFunction.apply(this);
@@ -771,7 +832,7 @@ public class ClientContext implements AccumuloClient {
         // ClientContext closes reservation unless a RuntimeException is thrown
         ClientInfo info = cbi.getClientInfo();
         AccumuloConfiguration config = 
ClientConfConverter.toAccumuloConf(info.getProperties());
-        return new ClientContext(reservation, info, config);
+        return new ClientContext(reservation, info, config, 
cbi.getUncaughtExceptionHandler());
       } catch (RuntimeException e) {
         reservation.close();
         throw e;
@@ -939,6 +1000,13 @@ public class ClientContext implements AccumuloClient {
     public void setProperty(ClientProperty property, Integer value) {
       setProperty(property, Integer.toString(value));
     }
+
+    @Override
+    public ClientFactory<T> 
withUncaughtExceptionHandler(UncaughtExceptionHandler ueh) {
+      this.ueh = ueh;
+      return this;
+    }
+
   }
 
   public ZooReader getZooReader() {
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 d566bae..9643e0a 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
@@ -40,7 +40,6 @@ import java.util.concurrent.Delayed;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
@@ -90,9 +89,6 @@ import org.apache.thrift.transport.TTransportException;
 
 class ConditionalWriterImpl implements ConditionalWriter {
 
-  private static ThreadPoolExecutor cleanupThreadPool =
-      ThreadPools.createFixedThreadPool(1, 3, SECONDS, "Conditional Writer 
Cleanup Thread", true);
-
   private static final int MAX_SLEEP = 30000;
 
   private Authorizations auths;
@@ -365,8 +361,8 @@ class ConditionalWriterImpl implements ConditionalWriter {
     this.context = context;
     this.auths = config.getAuthorizations();
     this.ve = new VisibilityEvaluator(config.getAuthorizations());
-    this.threadPool = 
ThreadPools.createScheduledExecutorService(config.getMaxWriteThreads(),
-        this.getClass().getSimpleName(), false);
+    this.threadPool = context.threadPools().createScheduledExecutorService(
+        config.getMaxWriteThreads(), this.getClass().getSimpleName(), false);
     this.locator = new SyncingTabletLocator(context, tableId);
     this.serverQueues = new HashMap<>();
     this.tableId = tableId;
@@ -803,7 +799,7 @@ class ConditionalWriterImpl implements ConditionalWriter {
   @Override
   public void close() {
     threadPool.shutdownNow();
-    
cleanupThreadPool.execute(Threads.createNamedRunnable("ConditionalWriterCleanupTask",
+    
context.executeCleanupTask(Threads.createNamedRunnable("ConditionalWriterCleanupTask",
         new CleanupTask(getActiveSessions())));
   }
 
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 61740f0..91d08a9 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
@@ -53,7 +53,6 @@ import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import 
org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
 import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
-import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransport;
@@ -220,7 +219,7 @@ public class InstanceOperationsImpl implements 
InstanceOperations {
 
     int numThreads = Math.max(4, Math.min((tservers.size() + 
compactors.size()) / 10, 256));
     var executorService =
-        ThreadPools.createFixedThreadPool(numThreads, "getactivecompactions", 
false);
+        context.threadPools().createFixedThreadPool(numThreads, 
"getactivecompactions", false);
     try {
       List<Future<List<ActiveCompaction>>> futures = new ArrayList<>();
 
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
index b349333..f998845 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
@@ -18,8 +18,6 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
-import static java.util.concurrent.TimeUnit.SECONDS;
-
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -27,8 +25,7 @@ import java.util.Map.Entry;
 import java.util.NoSuchElementException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.client.SampleNotPresentException;
 import org.apache.accumulo.core.client.TableDeletedException;
@@ -40,7 +37,6 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
@@ -65,15 +61,14 @@ public class ScannerIterator implements 
Iterator<Entry<Key,Value>> {
 
   private ScannerImpl.Reporter reporter;
 
-  private static ThreadPoolExecutor readaheadPool =
-      ThreadPools.createThreadPool(0, Integer.MAX_VALUE, 3L, SECONDS,
-          "Accumulo scanner read ahead thread", new SynchronousQueue<>(), 
true);
+  private final ClientContext context;
 
-  private boolean closed = false;
+  private AtomicBoolean closed = new AtomicBoolean(false);
 
   ScannerIterator(ClientContext context, TableId tableId, Authorizations 
authorizations,
       Range range, int size, long timeOut, ScannerOptions options, boolean 
isolated,
       long readaheadThreshold, ScannerImpl.Reporter reporter) {
+    this.context = context;
     this.timeOut = timeOut;
     this.readaheadThreshold = readaheadThreshold;
 
@@ -126,11 +121,11 @@ public class ScannerIterator implements 
Iterator<Entry<Key,Value>> {
 
   void close() {
     // run actual close operation in the background so this does not block.
-    readaheadPool.execute(() -> {
+    context.executeCleanupTask(() -> {
       synchronized (scanState) {
         // this is synchronized so its mutually exclusive with readBatch()
         try {
-          closed = true;
+          closed.set(true);
           ThriftScanner.close(scanState);
         } catch (Exception e) {
           LoggerFactory.getLogger(ScannerIterator.class)
@@ -142,7 +137,7 @@ public class ScannerIterator implements 
Iterator<Entry<Key,Value>> {
 
   private void initiateReadAhead() {
     Preconditions.checkState(readAheadOperation == null);
-    readAheadOperation = readaheadPool.submit(this::readBatch);
+    readAheadOperation = context.submitScannerReadAheadTask(this::readBatch);
   }
 
   private List<KeyValue> readBatch() throws Exception {
@@ -152,7 +147,7 @@ public class ScannerIterator implements 
Iterator<Entry<Key,Value>> {
     do {
       synchronized (scanState) {
         // this is synchronized so its mutually exclusive with closing
-        Preconditions.checkState(!closed, "Scanner was closed");
+        Preconditions.checkState(!closed.get(), "Scanner was closed");
         batch = ThriftScanner.scan(scanState.context, scanState, timeOut);
       }
     } while (batch != null && batch.isEmpty());
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 996d74f..60f3d77 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
@@ -136,7 +136,6 @@ import org.apache.accumulo.core.util.MapCounter;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.TextUtil;
-import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.fate.util.Retry;
 import org.apache.hadoop.fs.FileStatus;
@@ -487,7 +486,7 @@ public class TableOperationsImpl extends 
TableOperationsHelper {
     CountDownLatch latch = new CountDownLatch(splits.size());
     AtomicReference<Exception> exception = new AtomicReference<>(null);
 
-    ExecutorService executor = ThreadPools.createFixedThreadPool(16, 
"addSplits", false);
+    ExecutorService executor = context.threadPools().createFixedThreadPool(16, 
"addSplits", false);
     try {
       executor.execute(
           new SplitTask(new SplitEnv(tableName, tableId, executor, latch, 
exception), splits));
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java
 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java
index 55e70e3..71f7396 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java
@@ -36,7 +36,6 @@ import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.cleaner.CleanerUtil;
-import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -72,7 +71,7 @@ public class TabletServerBatchReader extends ScannerOptions 
implements BatchScan
     this.tableName = tableName;
     this.numThreads = numQueryThreads;
 
-    queryThreadPool = ThreadPools.createFixedThreadPool(numQueryThreads,
+    queryThreadPool = 
context.threadPools().createFixedThreadPool(numQueryThreads,
         "batch scanner " + batchReaderInstance + "-", false);
     // Call shutdown on this thread pool in case the caller does not call 
close().
     cleanable = CleanerUtil.shutdownThreadPoolExecutor(queryThreadPool, 
closed, log);
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 b2374bc..5184691 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
@@ -202,8 +202,8 @@ public class TabletServerBatchWriter implements 
AutoCloseable {
 
   public TabletServerBatchWriter(ClientContext context, BatchWriterConfig 
config) {
     this.context = context;
-    this.executor =
-        
ThreadPools.createGeneralScheduledExecutorService(this.context.getConfiguration());
+    this.executor = context.threadPools()
+        
.createGeneralScheduledExecutorService(this.context.getConfiguration());
     this.failedMutations = new FailedMutations();
     this.maxMem = config.getMaxMemory();
     this.maxLatency = config.getMaxLatency(MILLISECONDS) <= 0 ? Long.MAX_VALUE
@@ -655,11 +655,11 @@ public class TabletServerBatchWriter implements 
AutoCloseable {
     public MutationWriter(int numSendThreads) {
       serversMutations = new HashMap<>();
       queued = new HashSet<>();
-      sendThreadPool =
-          ThreadPools.createFixedThreadPool(numSendThreads, 
this.getClass().getName(), false);
+      sendThreadPool = 
context.threadPools().createFixedThreadPool(numSendThreads,
+          this.getClass().getName(), false);
       locators = new HashMap<>();
-      binningThreadPool =
-          ThreadPools.createFixedThreadPool(1, "BinMutations", new 
SynchronousQueue<>(), false);
+      binningThreadPool = context.threadPools().createFixedThreadPool(1, 
"BinMutations",
+          new SynchronousQueue<>(), false);
       binningThreadPool.setRejectedExecutionHandler(new 
ThreadPoolExecutor.CallerRunsPolicy());
     }
 
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
index eef25cc..b20785a 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
@@ -75,7 +75,6 @@ import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.spi.crypto.CryptoService;
-import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.fate.util.Retry;
 import org.apache.commons.io.FilenameUtils;
@@ -476,10 +475,11 @@ public class BulkImport implements 
ImportDestinationArguments, ImportMappingOpti
     if (this.executor != null) {
       executor = this.executor;
     } else if (numThreads > 0) {
-      executor = service = ThreadPools.createFixedThreadPool(numThreads, 
"BulkImportThread", false);
+      executor = service =
+          context.threadPools().createFixedThreadPool(numThreads, 
"BulkImportThread", false);
     } else {
       String threads = 
context.getConfiguration().get(ClientProperty.BULK_LOAD_THREADS.getKey());
-      executor = service = ThreadPools.createFixedThreadPool(
+      executor = service = context.threadPools().createFixedThreadPool(
           ConfigurationTypeHelper.getNumThreads(threads), "BulkImportThread", 
false);
     }
 
diff --git 
a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java 
b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
index 3b4c041..10e8db3 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
@@ -79,8 +79,8 @@ public class BloomFilterLayer {
     }
 
     if (maxLoadThreads > 0) {
-      loadThreadPool =
-          ThreadPools.createThreadPool(0, maxLoadThreads, 60, SECONDS, 
"bloom-loader", false);
+      loadThreadPool = ThreadPools.getServerThreadPools().createThreadPool(0, 
maxLoadThreads, 60,
+          SECONDS, "bloom-loader", false);
     }
 
     return loadThreadPool;
diff --git 
a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/lru/LruBlockCache.java
 
b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/lru/LruBlockCache.java
index 8ed3b6b..4c02f4d 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/lru/LruBlockCache.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/lru/LruBlockCache.java
@@ -102,8 +102,8 @@ public class LruBlockCache extends 
SynchronousLoadingBlockCache implements Block
   private final EvictionThread evictionThread;
 
   /** Statistics thread schedule pool (for heavy debugging, could remove) */
-  private final ScheduledExecutorService scheduleThreadPool =
-      ThreadPools.createScheduledExecutorService(1, "LRUBlockCacheStats", 
true);
+  private final ScheduledExecutorService scheduleThreadPool = 
ThreadPools.getServerThreadPools()
+      .createScheduledExecutorService(1, "LRUBlockCacheStats", true);
 
   /** Current size of cache */
   private final AtomicLong size;
diff --git 
a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/tinylfu/TinyLfuBlockCache.java
 
b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/tinylfu/TinyLfuBlockCache.java
index 9819923..2bc3186 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/tinylfu/TinyLfuBlockCache.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/tinylfu/TinyLfuBlockCache.java
@@ -60,8 +60,8 @@ public final class TinyLfuBlockCache implements BlockCache {
   private final Cache<String,Block> cache;
   private final Policy.Eviction<String,Block> policy;
   private final int maxSize;
-  private final ScheduledExecutorService statsExecutor =
-      ThreadPools.createScheduledExecutorService(1, 
"TinyLfuBlockCacheStatsExecutor", true);
+  private final ScheduledExecutorService statsExecutor = 
ThreadPools.getServerThreadPools()
+      .createScheduledExecutorService(1, "TinyLfuBlockCacheStatsExecutor", 
true);
 
   public TinyLfuBlockCache(Configuration conf, CacheType type) {
     cache = Caffeine.newBuilder()
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 a89932d..67ff60f 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
@@ -231,8 +231,8 @@ public class ExternalCompactionUtil {
    */
   public static List<RunningCompaction> 
getCompactionsRunningOnCompactors(ClientContext context) {
     final List<RunningCompactionFuture> rcFutures = new ArrayList<>();
-    final ExecutorService executor =
-        ThreadPools.createFixedThreadPool(16, "CompactorRunningCompactions", 
false);
+    final ExecutorService executor = 
ThreadPools.getServerThreadPools().createFixedThreadPool(16,
+        "CompactorRunningCompactions", false);
 
     getCompactorAddrs(context).forEach((q, hp) -> {
       hp.forEach(hostAndPort -> {
@@ -259,8 +259,8 @@ public class ExternalCompactionUtil {
 
   public static Collection<ExternalCompactionId>
       getCompactionIdsRunningOnCompactors(ClientContext context) {
-    final ExecutorService executor =
-        ThreadPools.createFixedThreadPool(16, "CompactorRunningCompactions", 
false);
+    final ExecutorService executor = 
ThreadPools.getServerThreadPools().createFixedThreadPool(16,
+        "CompactorRunningCompactions", false);
 
     List<Future<ExternalCompactionId>> futures = new ArrayList<>();
 
diff --git 
a/core/src/main/java/org/apache/accumulo/core/util/ratelimit/SharedRateLimiterFactory.java
 
b/core/src/main/java/org/apache/accumulo/core/util/ratelimit/SharedRateLimiterFactory.java
index ae61337..e7e3a8c 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/util/ratelimit/SharedRateLimiterFactory.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/util/ratelimit/SharedRateLimiterFactory.java
@@ -56,7 +56,8 @@ public class SharedRateLimiterFactory {
     if (instance == null) {
       instance = new SharedRateLimiterFactory();
 
-      ScheduledThreadPoolExecutor svc = 
ThreadPools.createGeneralScheduledExecutorService(conf);
+      ScheduledThreadPoolExecutor svc =
+          
ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(conf);
       updateTaskFuture = svc.scheduleWithFixedDelay(Threads
           .createNamedRunnable("SharedRateLimiterFactory update polling", 
instance::updateAll),
           UPDATE_RATE, UPDATE_RATE, MILLISECONDS);
diff --git 
a/core/src/main/java/org/apache/accumulo/core/util/threads/NamedThreadFactory.java
 
b/core/src/main/java/org/apache/accumulo/core/util/threads/NamedThreadFactory.java
index 4c87dd5..080c7c5 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/util/threads/NamedThreadFactory.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/util/threads/NamedThreadFactory.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.core.util.threads;
 
+import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.OptionalInt;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -29,17 +30,19 @@ class NamedThreadFactory implements ThreadFactory {
 
   private static final String FORMAT = "%s-%s-%d";
 
-  private AtomicInteger threadNum = new AtomicInteger(1);
-  private String name;
-  private OptionalInt priority;
+  private final AtomicInteger threadNum = new AtomicInteger(1);
+  private final String name;
+  private final OptionalInt priority;
+  private final UncaughtExceptionHandler handler;
 
-  NamedThreadFactory(String name) {
-    this(name, OptionalInt.empty());
+  NamedThreadFactory(String name, UncaughtExceptionHandler ueh) {
+    this(name, OptionalInt.empty(), ueh);
   }
 
-  NamedThreadFactory(String name, OptionalInt priority) {
+  NamedThreadFactory(String name, OptionalInt priority, 
UncaughtExceptionHandler ueh) {
     this.name = name;
     this.priority = priority;
+    this.handler = ueh;
   }
 
   @Override
@@ -52,6 +55,6 @@ class NamedThreadFactory implements ThreadFactory {
       threadName =
           String.format(FORMAT, name, r.getClass().getSimpleName(), 
threadNum.getAndIncrement());
     }
-    return Threads.createThread(threadName, priority, r);
+    return Threads.createThread(threadName, priority, r, handler);
   }
 }
diff --git 
a/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java 
b/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
index afa05c0..73cb308 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
@@ -21,6 +21,7 @@ package org.apache.accumulo.core.util.threads;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
 
+import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.Iterator;
 import java.util.List;
 import java.util.OptionalInt;
@@ -65,8 +66,18 @@ public class ThreadPools {
   // the number of seconds before we allow a thread to terminate with non-use.
   public static final long DEFAULT_TIMEOUT_MILLISECS = 180000L;
 
+  private static final ThreadPools SERVER_INSTANCE = new 
ThreadPools(Threads.UEH);
+
+  public static final ThreadPools getServerThreadPools() {
+    return SERVER_INSTANCE;
+  }
+
+  public static final ThreadPools 
getClientThreadPools(UncaughtExceptionHandler ueh) {
+    return new ThreadPools(ueh);
+  }
+
   private static final ThreadPoolExecutor SCHEDULED_FUTURE_CHECKER_POOL =
-      createFixedThreadPool(1, "Scheduled Future Checker", false);
+      getServerThreadPools().createFixedThreadPool(1, "Scheduled Future 
Checker", false);
 
   private static final ConcurrentLinkedQueue<ScheduledFuture<?>> 
CRITICAL_RUNNING_TASKS =
       new ConcurrentLinkedQueue<>();
@@ -216,6 +227,12 @@ public class ThreadPools {
     resizePool(pool, () -> conf.getCount(p), p.getKey());
   }
 
+  private final UncaughtExceptionHandler handler;
+
+  private ThreadPools(UncaughtExceptionHandler ueh) {
+    handler = ueh;
+  }
+
   /**
    * Create a thread pool based on a thread pool related property
    *
@@ -235,7 +252,7 @@ public class ThreadPools {
    *           if property is not handled
    */
   @SuppressWarnings("deprecation")
-  public static ThreadPoolExecutor createExecutorService(final 
AccumuloConfiguration conf,
+  public ThreadPoolExecutor createExecutorService(final AccumuloConfiguration 
conf,
       final Property p, boolean emitThreadPoolMetrics) {
 
     switch (p) {
@@ -304,7 +321,7 @@ public class ThreadPools {
    *          over long time periods.
    * @return ThreadPoolExecutor
    */
-  public static ThreadPoolExecutor createFixedThreadPool(int numThreads, final 
String name,
+  public ThreadPoolExecutor createFixedThreadPool(int numThreads, final String 
name,
       boolean emitThreadPoolMetrics) {
     return createFixedThreadPool(numThreads, DEFAULT_TIMEOUT_MILLISECS, 
MILLISECONDS, name,
         emitThreadPoolMetrics);
@@ -328,7 +345,7 @@ public class ThreadPools {
    *          over long time periods.
    * @return ThreadPoolExecutor
    */
-  public static ThreadPoolExecutor createFixedThreadPool(int numThreads, final 
String name,
+  public ThreadPoolExecutor createFixedThreadPool(int numThreads, final String 
name,
       BlockingQueue<Runnable> queue, boolean emitThreadPoolMetrics) {
     return createThreadPool(numThreads, numThreads, DEFAULT_TIMEOUT_MILLISECS, 
MILLISECONDS, name,
         queue, emitThreadPoolMetrics);
@@ -354,8 +371,8 @@ public class ThreadPools {
    *          over long time periods.
    * @return ThreadPoolExecutor
    */
-  public static ThreadPoolExecutor createFixedThreadPool(int numThreads, long 
timeOut,
-      TimeUnit units, final String name, boolean emitThreadPoolMetrics) {
+  public ThreadPoolExecutor createFixedThreadPool(int numThreads, long 
timeOut, TimeUnit units,
+      final String name, boolean emitThreadPoolMetrics) {
     return createThreadPool(numThreads, numThreads, timeOut, units, name, 
emitThreadPoolMetrics);
   }
 
@@ -381,7 +398,7 @@ public class ThreadPools {
    *          over long time periods.
    * @return ThreadPoolExecutor
    */
-  public static ThreadPoolExecutor createThreadPool(int coreThreads, int 
maxThreads, long timeOut,
+  public ThreadPoolExecutor createThreadPool(int coreThreads, int maxThreads, 
long timeOut,
       TimeUnit units, final String name, boolean emitThreadPoolMetrics) {
     return createThreadPool(coreThreads, maxThreads, timeOut, units, name,
         new LinkedBlockingQueue<>(), emitThreadPoolMetrics);
@@ -411,7 +428,7 @@ public class ThreadPools {
    *          over long time periods.
    * @return ThreadPoolExecutor
    */
-  public static ThreadPoolExecutor createThreadPool(int coreThreads, int 
maxThreads, long timeOut,
+  public ThreadPoolExecutor createThreadPool(int coreThreads, int maxThreads, 
long timeOut,
       TimeUnit units, final String name, BlockingQueue<Runnable> queue,
       boolean emitThreadPoolMetrics) {
     return createThreadPool(coreThreads, maxThreads, timeOut, units, name, 
queue,
@@ -444,11 +461,11 @@ public class ThreadPools {
    *          over long time periods.
    * @return ThreadPoolExecutor
    */
-  public static ThreadPoolExecutor createThreadPool(int coreThreads, int 
maxThreads, long timeOut,
+  public ThreadPoolExecutor createThreadPool(int coreThreads, int maxThreads, 
long timeOut,
       TimeUnit units, final String name, BlockingQueue<Runnable> queue, 
OptionalInt priority,
       boolean emitThreadPoolMetrics) {
     var result = new ThreadPoolExecutor(coreThreads, maxThreads, timeOut, 
units, queue,
-        new NamedThreadFactory(name, priority)) {
+        new NamedThreadFactory(name, priority, handler)) {
 
       @Override
       public void execute(Runnable arg0) {
@@ -488,7 +505,7 @@ public class ThreadPools {
    * If you need the server-side shared ScheduledThreadPoolExecutor, then use
    * ServerContext.getScheduledExecutor()
    */
-  public static ScheduledThreadPoolExecutor
+  public ScheduledThreadPoolExecutor
       createGeneralScheduledExecutorService(AccumuloConfiguration conf) {
     return (ScheduledThreadPoolExecutor) createExecutorService(conf,
         Property.GENERAL_SIMPLETIMER_THREADPOOL_SIZE, true);
@@ -510,58 +527,59 @@ public class ThreadPools {
    *          over long time periods.
    * @return ScheduledThreadPoolExecutor
    */
-  public static ScheduledThreadPoolExecutor createScheduledExecutorService(int 
numThreads,
+  public ScheduledThreadPoolExecutor createScheduledExecutorService(int 
numThreads,
       final String name, boolean emitThreadPoolMetrics) {
-    var result = new ScheduledThreadPoolExecutor(numThreads, new 
NamedThreadFactory(name)) {
+    var result =
+        new ScheduledThreadPoolExecutor(numThreads, new 
NamedThreadFactory(name, handler)) {
 
-      @Override
-      public void execute(Runnable command) {
-        super.execute(TraceUtil.wrap(command));
-      }
+          @Override
+          public void execute(Runnable command) {
+            super.execute(TraceUtil.wrap(command));
+          }
 
-      @Override
-      public <V> ScheduledFuture<V> schedule(Callable<V> callable, long delay, 
TimeUnit unit) {
-        return super.schedule(TraceUtil.wrap(callable), delay, unit);
-      }
+          @Override
+          public <V> ScheduledFuture<V> schedule(Callable<V> callable, long 
delay, TimeUnit unit) {
+            return super.schedule(TraceUtil.wrap(callable), delay, unit);
+          }
 
-      @Override
-      public ScheduledFuture<?> schedule(Runnable command, long delay, 
TimeUnit unit) {
-        return super.schedule(TraceUtil.wrap(command), delay, unit);
-      }
+          @Override
+          public ScheduledFuture<?> schedule(Runnable command, long delay, 
TimeUnit unit) {
+            return super.schedule(TraceUtil.wrap(command), delay, unit);
+          }
 
-      @Override
-      public ScheduledFuture<?> scheduleAtFixedRate(Runnable command, long 
initialDelay,
-          long period, TimeUnit unit) {
-        return super.scheduleAtFixedRate(TraceUtil.wrap(command), 
initialDelay, period, unit);
-      }
+          @Override
+          public ScheduledFuture<?> scheduleAtFixedRate(Runnable command, long 
initialDelay,
+              long period, TimeUnit unit) {
+            return super.scheduleAtFixedRate(TraceUtil.wrap(command), 
initialDelay, period, unit);
+          }
 
-      @Override
-      public ScheduledFuture<?> scheduleWithFixedDelay(Runnable command, long 
initialDelay,
-          long delay, TimeUnit unit) {
-        return super.scheduleWithFixedDelay(TraceUtil.wrap(command), 
initialDelay, delay, unit);
-      }
+          @Override
+          public ScheduledFuture<?> scheduleWithFixedDelay(Runnable command, 
long initialDelay,
+              long delay, TimeUnit unit) {
+            return super.scheduleWithFixedDelay(TraceUtil.wrap(command), 
initialDelay, delay, unit);
+          }
 
-      @Override
-      public <T> Future<T> submit(Callable<T> task) {
-        return super.submit(TraceUtil.wrap(task));
-      }
+          @Override
+          public <T> Future<T> submit(Callable<T> task) {
+            return super.submit(TraceUtil.wrap(task));
+          }
 
-      @Override
-      public <T> Future<T> submit(Runnable task, T result) {
-        return super.submit(TraceUtil.wrap(task), result);
-      }
+          @Override
+          public <T> Future<T> submit(Runnable task, T result) {
+            return super.submit(TraceUtil.wrap(task), result);
+          }
 
-      @Override
-      public Future<?> submit(Runnable task) {
-        return super.submit(TraceUtil.wrap(task));
-      }
+          @Override
+          public Future<?> submit(Runnable task) {
+            return super.submit(TraceUtil.wrap(task));
+          }
 
-      @Override
-      public boolean remove(Runnable task) {
-        return super.remove(TraceUtil.wrap(task));
-      }
+          @Override
+          public boolean remove(Runnable task) {
+            return super.remove(TraceUtil.wrap(task));
+          }
 
-    };
+        };
     if (emitThreadPoolMetrics) {
       MetricsUtil.addExecutorServiceMetrics(result, name);
     }
diff --git 
a/core/src/main/java/org/apache/accumulo/core/util/threads/Threads.java 
b/core/src/main/java/org/apache/accumulo/core/util/threads/Threads.java
index 0d25972..c61c063 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/threads/Threads.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/threads/Threads.java
@@ -25,38 +25,45 @@ import org.apache.accumulo.core.trace.TraceUtil;
 
 public class Threads {
 
+  public static final UncaughtExceptionHandler UEH = new 
AccumuloUncaughtExceptionHandler();
+
   public static class AccumuloDaemonThread extends Thread {
 
-    public AccumuloDaemonThread(Runnable target, String name) {
+    public AccumuloDaemonThread(Runnable target, String name, 
UncaughtExceptionHandler ueh) {
       super(target, name);
-      configureThread(this, name);
+      setDaemon(true);
+      setUncaughtExceptionHandler(ueh);
     }
 
     public AccumuloDaemonThread(String name) {
+      this(name, UEH);
+    }
+
+    private AccumuloDaemonThread(String name, UncaughtExceptionHandler ueh) {
       super(name);
-      configureThread(this, name);
+      setDaemon(true);
+      setUncaughtExceptionHandler(ueh);
     }
+
   }
 
   public static Runnable createNamedRunnable(String name, Runnable r) {
     return new NamedRunnable(name, r);
   }
 
-  private static final UncaughtExceptionHandler UEH = new 
AccumuloUncaughtExceptionHandler();
-
   public static Thread createThread(String name, Runnable r) {
-    return createThread(name, OptionalInt.empty(), r);
+    return createThread(name, OptionalInt.empty(), r, UEH);
   }
 
   public static Thread createThread(String name, OptionalInt priority, 
Runnable r) {
-    Thread thread = new AccumuloDaemonThread(TraceUtil.wrap(r), name);
+    return createThread(name, priority, r, UEH);
+  }
+
+  public static Thread createThread(String name, OptionalInt priority, 
Runnable r,
+      UncaughtExceptionHandler ueh) {
+    Thread thread = new AccumuloDaemonThread(TraceUtil.wrap(r), name, ueh);
     priority.ifPresent(thread::setPriority);
     return thread;
   }
 
-  private static void configureThread(Thread thread, String name) {
-    thread.setName(name);
-    thread.setDaemon(true);
-    thread.setUncaughtExceptionHandler(UEH);
-  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/fate/Fate.java 
b/core/src/main/java/org/apache/accumulo/fate/Fate.java
index 6e51bb3..10678e4 100644
--- a/core/src/main/java/org/apache/accumulo/fate/Fate.java
+++ b/core/src/main/java/org/apache/accumulo/fate/Fate.java
@@ -244,9 +244,10 @@ public class Fate<T> {
    * Launches the specified number of worker threads.
    */
   public void startTransactionRunners(AccumuloConfiguration conf) {
-    final ThreadPoolExecutor pool =
-        ThreadPools.createExecutorService(conf, 
Property.MANAGER_FATE_THREADPOOL_SIZE, true);
-    fatePoolWatcher = ThreadPools.createGeneralScheduledExecutorService(conf);
+    final ThreadPoolExecutor pool = 
ThreadPools.getServerThreadPools().createExecutorService(conf,
+        Property.MANAGER_FATE_THREADPOOL_SIZE, true);
+    fatePoolWatcher =
+        
ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(conf);
     ThreadPools.watchCriticalScheduledTask(fatePoolWatcher.schedule(() -> {
       // resize the pool if the property changed
       ThreadPools.resizePool(pool, conf, 
Property.MANAGER_FATE_THREADPOOL_SIZE);
diff --git 
a/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java
 
b/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java
index fc918b5..06e3550 100644
--- 
a/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java
+++ 
b/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java
@@ -24,6 +24,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.easymock.EasyMock;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
@@ -35,6 +36,8 @@ public class TabletServerBatchReaderTest {
   @BeforeEach
   public void setup() {
     context = EasyMock.createMock(ClientContext.class);
+    
EasyMock.expect(context.threadPools()).andReturn(ThreadPools.getServerThreadPools());
+    EasyMock.replay(context);
   }
 
   @Test
diff --git 
a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java
 
b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java
index 9ce5649..086e171 100644
--- 
a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java
+++ 
b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java
@@ -233,8 +233,8 @@ public class MultiThreadedRFileTest {
       // now start up multiple RFile deepcopies
       int maxThreads = 10;
       String name = "MultiThreadedRFileTestThread";
-      ThreadPoolExecutor pool = ThreadPools.createThreadPool(maxThreads + 1, 
maxThreads + 1, 5 * 60,
-          SECONDS, name, false);
+      ThreadPoolExecutor pool = 
ThreadPools.getServerThreadPools().createThreadPool(maxThreads + 1,
+          maxThreads + 1, 5 * 60, SECONDS, name, false);
       try {
         Runnable runnable = () -> {
           try {
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java 
b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
index 5d2bafd..835430b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
@@ -56,6 +56,7 @@ import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooReader;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
@@ -102,7 +103,7 @@ public class ServerContext extends ClientContext {
   }
 
   private ServerContext(ServerInfo info) {
-    super(SingletonReservation.noop(), info, info.getSiteConfiguration());
+    super(SingletonReservation.noop(), info, info.getSiteConfiguration(), 
Threads.UEH);
     this.info = info;
     zooReaderWriter = new ZooReaderWriter(info.getSiteConfiguration());
     serverDirs = info.getServerDirs();
@@ -455,8 +456,8 @@ public class ServerContext extends ClientContext {
   public synchronized ScheduledThreadPoolExecutor getScheduledExecutor() {
     if (sharedScheduledThreadPool == null) {
       sharedScheduledThreadPool =
-          (ScheduledThreadPoolExecutor) 
ThreadPools.createExecutorService(getConfiguration(),
-              Property.GENERAL_SIMPLETIMER_THREADPOOL_SIZE, true);
+          (ScheduledThreadPoolExecutor) 
ThreadPools.getServerThreadPools().createExecutorService(
+              getConfiguration(), 
Property.GENERAL_SIMPLETIMER_THREADPOOL_SIZE, true);
     }
     return sharedScheduledThreadPool;
   }
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 d68c5ea..f6ecdc7 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
@@ -130,8 +130,8 @@ public class BulkImporter {
           Collections.synchronizedSortedMap(new TreeMap<>());
 
       timer.start(Timers.EXAMINE_MAP_FILES);
-      ExecutorService threadPool =
-          ThreadPools.createFixedThreadPool(numThreads, "findOverlapping", 
false);
+      ExecutorService threadPool = ThreadPools.getServerThreadPools()
+          .createFixedThreadPool(numThreads, "findOverlapping", false);
 
       for (Path path : paths) {
         final Path mapFile = path;
@@ -351,8 +351,8 @@ public class BulkImporter {
 
     final Map<Path,List<AssignmentInfo>> ais = Collections.synchronizedMap(new 
TreeMap<>());
 
-    ExecutorService threadPool =
-        ThreadPools.createFixedThreadPool(numThreads, "estimateSizes", false);
+    ExecutorService threadPool = ThreadPools.getServerThreadPools()
+        .createFixedThreadPool(numThreads, "estimateSizes", false);
 
     for (final Entry<Path,List<TabletLocation>> entry : 
assignments.entrySet()) {
       if (entry.getValue().size() == 1) {
@@ -536,7 +536,8 @@ public class BulkImporter {
       }
     });
 
-    ExecutorService threadPool = ThreadPools.createFixedThreadPool(numThreads, 
"submit", false);
+    ExecutorService threadPool =
+        ThreadPools.getServerThreadPools().createFixedThreadPool(numThreads, 
"submit", false);
 
     for (Entry<String,Map<KeyExtent,List<PathSize>>> entry : 
assignmentsPerTabletServer
         .entrySet()) {
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
 
b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
index 41ba6be..11f8369 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
@@ -304,7 +304,8 @@ public class VolumeManagerImpl implements VolumeManager {
   public void bulkRename(Map<Path,Path> oldToNewPathMap, int poolSize, String 
poolName,
       String transactionId) throws IOException {
     List<Future<Void>> results = new ArrayList<>();
-    ExecutorService workerPool = ThreadPools.createFixedThreadPool(poolSize, 
poolName, false);
+    ExecutorService workerPool =
+        ThreadPools.getServerThreadPools().createFixedThreadPool(poolSize, 
poolName, false);
     oldToNewPathMap.forEach((oldPath, newPath) -> 
results.add(workerPool.submit(() -> {
       boolean success;
       try {
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
 
b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
index 08dc7ab..1b7245e 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
@@ -66,8 +66,8 @@ public class ProblemReports implements 
Iterable<ProblemReport> {
    * processed because the whole system is in a really bad state (like HDFS is 
down) and everything
    * is reporting lots of problems, but problem reports can not be processed
    */
-  private ExecutorService reportExecutor = ThreadPools.createThreadPool(0, 1, 
60, TimeUnit.SECONDS,
-      "acu-problem-reporter", new LinkedBlockingQueue<>(500), false);
+  private ExecutorService reportExecutor = 
ThreadPools.getServerThreadPools().createThreadPool(0, 1,
+      60, TimeUnit.SECONDS, "acu-problem-reporter", new 
LinkedBlockingQueue<>(500), false);
 
   private final ServerContext context;
 
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/rpc/TServerUtils.java 
b/server/base/src/main/java/org/apache/accumulo/server/rpc/TServerUtils.java
index 80e48cf..b176175 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/TServerUtils.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/TServerUtils.java
@@ -311,11 +311,11 @@ public class TServerUtils {
   public static ThreadPoolExecutor createSelfResizingThreadPool(final String 
serverName,
       final int executorThreads, long threadTimeOut, final 
AccumuloConfiguration conf,
       long timeBetweenThreadChecks) {
-    final ThreadPoolExecutor pool = 
ThreadPools.createFixedThreadPool(executorThreads,
-        threadTimeOut, TimeUnit.MILLISECONDS, serverName + "-ClientPool", 
true);
+    final ThreadPoolExecutor pool = 
ThreadPools.getServerThreadPools().createFixedThreadPool(
+        executorThreads, threadTimeOut, TimeUnit.MILLISECONDS, serverName + 
"-ClientPool", true);
     // periodically adjust the number of threads we need by checking how busy 
our threads are
-    ThreadPools.watchCriticalScheduledTask(
-        
ThreadPools.createGeneralScheduledExecutorService(conf).scheduleWithFixedDelay(()
 -> {
+    ThreadPools.watchCriticalScheduledTask(ThreadPools.getServerThreadPools()
+        .createGeneralScheduledExecutorService(conf).scheduleWithFixedDelay(() 
-> {
           // there is a minor race condition between sampling the current 
state of the thread pool
           // and
           // adjusting it
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/FileSystemMonitor.java
 
b/server/base/src/main/java/org/apache/accumulo/server/util/FileSystemMonitor.java
index 11d19cd..255f453 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/util/FileSystemMonitor.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/util/FileSystemMonitor.java
@@ -117,8 +117,8 @@ public class FileSystemMonitor {
 
     // Create a task to check each mount periodically to see if its state has 
changed.
     for (Mount mount : mounts) {
-      ThreadPools.watchCriticalScheduledTask(
-          
ThreadPools.createGeneralScheduledExecutorService(conf).scheduleWithFixedDelay(
+      ThreadPools.watchCriticalScheduledTask(ThreadPools.getServerThreadPools()
+          .createGeneralScheduledExecutorService(conf).scheduleWithFixedDelay(
               Threads.createNamedRunnable(mount.mountPoint + "filesystem 
monitor", () -> {
                 try {
                   checkMount(mount);
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
 
b/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
index ba22df7..9a98036 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
@@ -121,7 +121,8 @@ public class RemoveEntriesForMissingFiles {
 
     Map<Path,Path> cache = new LRUMap<>(100000);
     Set<Path> processing = new HashSet<>();
-    ExecutorService threadPool = ThreadPools.createFixedThreadPool(16, 
"CheckFileTasks", false);
+    ExecutorService threadPool =
+        ThreadPools.getServerThreadPools().createFixedThreadPool(16, 
"CheckFileTasks", false);
 
     System.out.printf("Scanning : %s %s\n", tableName, range);
 
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 0fbd73c..50fe982 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
@@ -124,7 +124,8 @@ public class VerifyTabletAssignments {
       }
     }
 
-    ExecutorService tp = ThreadPools.createFixedThreadPool(20, 
"CheckTabletServer", false);
+    ExecutorService tp =
+        ThreadPools.getServerThreadPools().createFixedThreadPool(20, 
"CheckTabletServer", false);
     for (final Entry<HostAndPort,List<KeyExtent>> entry : 
extentsPerServer.entrySet()) {
       Runnable r = () -> {
         try {
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 77a5c3b..0245f86 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
@@ -127,7 +127,7 @@ public class CompactionCoordinator extends AbstractServer
   protected CompactionCoordinator(ServerOpts opts, String[] args) {
     super("compaction-coordinator", opts, args);
     aconf = getConfiguration();
-    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    schedExecutor = 
ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(aconf);
     compactionFinalizer = createCompactionFinalizer(schedExecutor);
     tserverSet = createLiveTServerSet();
     setupSecurity();
@@ -139,7 +139,7 @@ public class CompactionCoordinator extends AbstractServer
   protected CompactionCoordinator(ServerOpts opts, String[] args, 
AccumuloConfiguration conf) {
     super("compaction-coordinator", opts, args);
     aconf = conf;
-    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    schedExecutor = 
ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(aconf);
     compactionFinalizer = createCompactionFinalizer(schedExecutor);
     tserverSet = createLiveTServerSet();
     setupSecurity();
@@ -319,8 +319,8 @@ public class CompactionCoordinator extends AbstractServer
   }
 
   private void updateSummaries() {
-    ExecutorService executor =
-        ThreadPools.createFixedThreadPool(10, "Compaction Summary Gatherer", 
false);
+    ExecutorService executor = 
ThreadPools.getServerThreadPools().createFixedThreadPool(10,
+        "Compaction Summary Gatherer", false);
     try {
       Set<String> queuesSeen = new ConcurrentSkipListSet<>();
 
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 196ac2a..da0cef0 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
@@ -73,11 +73,11 @@ public class CompactionFinalizer {
     int max = this.context.getConfiguration()
         
.getCount(Property.COMPACTION_COORDINATOR_FINALIZER_TSERVER_NOTIFIER_MAXTHREADS);
 
-    this.ntfyExecutor = ThreadPools.createThreadPool(3, max, 1, 
TimeUnit.MINUTES,
-        "Compaction Finalizer Notifier", true);
+    this.ntfyExecutor = ThreadPools.getServerThreadPools().createThreadPool(3, 
max, 1,
+        TimeUnit.MINUTES, "Compaction Finalizer Notifier", true);
 
-    this.backgroundExecutor =
-        ThreadPools.createFixedThreadPool(1, "Compaction Finalizer Background 
Task", true);
+    this.backgroundExecutor = 
ThreadPools.getServerThreadPools().createFixedThreadPool(1,
+        "Compaction Finalizer Background Task", true);
 
     backgroundExecutor.execute(() -> {
       processPending();
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 d1cc457..ffb3825 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
@@ -166,7 +166,8 @@ public class Compactor extends AbstractServer implements 
MetricsProducer, Compac
     aconf = getConfiguration();
     setupSecurity();
     watcher = new CompactionWatcher(aconf);
-    var schedExecutor = 
ThreadPools.createGeneralScheduledExecutorService(aconf);
+    var schedExecutor =
+        
ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(aconf);
     startGCLogger(schedExecutor);
     startCancelChecker(schedExecutor, TIME_BETWEEN_CANCEL_CHECKS);
     printStartupMsg();
@@ -178,7 +179,8 @@ public class Compactor extends AbstractServer implements 
MetricsProducer, Compac
     aconf = conf;
     setupSecurity();
     watcher = new CompactionWatcher(aconf);
-    var schedExecutor = 
ThreadPools.createGeneralScheduledExecutorService(aconf);
+    var schedExecutor =
+        
ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(aconf);
     startGCLogger(schedExecutor);
     startCancelChecker(schedExecutor, TIME_BETWEEN_CANCEL_CHECKS);
     printStartupMsg();
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 b6174a5..c446c84 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
@@ -286,8 +286,8 @@ public class SimpleGarbageCollector extends AbstractServer 
implements Iface {
 
       minimizeDeletes(confirmedDeletes, processedDeletes, fs);
 
-      ExecutorService deleteThreadPool =
-          ThreadPools.createExecutorService(getConfiguration(), 
Property.GC_DELETE_THREADS, false);
+      ExecutorService deleteThreadPool = ThreadPools.getServerThreadPools()
+          .createExecutorService(getConfiguration(), 
Property.GC_DELETE_THREADS, false);
 
       final List<Pair<Path,Path>> replacements = 
getContext().getVolumeReplacements();
 
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 7f360de..413681a 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
@@ -919,8 +919,8 @@ public class Manager extends AbstractServer
       Set<TServerInstance> currentServers, 
SortedMap<TabletServerId,TServerStatus> balancerMap) {
     final long rpcTimeout = 
getConfiguration().getTimeInMillis(Property.GENERAL_RPC_TIMEOUT);
     int threads = 
getConfiguration().getCount(Property.MANAGER_STATUS_THREAD_POOL_SIZE);
-    ExecutorService tp = ThreadPools.createExecutorService(getConfiguration(),
-        Property.MANAGER_STATUS_THREAD_POOL_SIZE, false);
+    ExecutorService tp = ThreadPools.getServerThreadPools()
+        .createExecutorService(getConfiguration(), 
Property.MANAGER_STATUS_THREAD_POOL_SIZE, false);
     long start = System.currentTimeMillis();
     final SortedMap<TServerInstance,TabletServerStatus> result = new 
ConcurrentSkipListMap<>();
     final RateLimiter shutdownServerRateLimiter = 
RateLimiter.create(MAX_SHUTDOWNS_PER_SEC);
diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java 
b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java
index 237a207..14de767 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java
@@ -64,9 +64,10 @@ public class ManagerTime {
       throw new IOException("Error updating manager time", ex);
     }
 
-    
ThreadPools.watchCriticalScheduledTask(ThreadPools.createGeneralScheduledExecutorService(conf)
-        .scheduleWithFixedDelay(Threads.createNamedRunnable("Manager time 
keeper", () -> run()), 0,
-            SECONDS.toMillis(10), MILLISECONDS));
+    ThreadPools.watchCriticalScheduledTask(
+        
ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(conf)
+            .scheduleWithFixedDelay(Threads.createNamedRunnable("Manager time 
keeper", () -> run()),
+                0, SECONDS.toMillis(10), MILLISECONDS));
   }
 
   /**
diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ReplicationMetrics.java
 
b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ReplicationMetrics.java
index c90337b..419e26f 100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ReplicationMetrics.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ReplicationMetrics.java
@@ -162,8 +162,8 @@ public class ReplicationMetrics implements MetricsProducer {
     maxReplicationThreads = registry.gauge(METRICS_REPLICATION_THREADS, 
MetricsUtil.getCommonTags(),
         new AtomicInteger(0));
 
-    ScheduledExecutorService scheduler =
-        ThreadPools.createScheduledExecutorService(1, 
"replicationMetricsPoller", false);
+    ScheduledExecutorService scheduler = ThreadPools.getServerThreadPools()
+        .createScheduledExecutorService(1, "replicationMetricsPoller", false);
     Runtime.getRuntime().addShutdownHook(new Thread(scheduler::shutdownNow));
     long minimumRefreshDelay = TimeUnit.SECONDS.toMillis(5);
     ScheduledFuture<?> future = scheduler.scheduleAtFixedRate(this::update, 
minimumRefreshDelay,
diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java
 
b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java
index 42e18db..008055a 100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java
@@ -152,8 +152,8 @@ public class FateMetrics implements MetricsProducer {
     update();
 
     // get fate status is read only operation - no reason to be nice on 
shutdown.
-    ScheduledExecutorService scheduler =
-        ThreadPools.createScheduledExecutorService(1, "fateMetricsPoller", 
false);
+    ScheduledExecutorService scheduler = ThreadPools.getServerThreadPools()
+        .createScheduledExecutorService(1, "fateMetricsPoller", false);
     Runtime.getRuntime().addShutdownHook(new Thread(scheduler::shutdownNow));
 
     ScheduledFuture<?> future = scheduler.scheduleAtFixedRate(() -> {
diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java
 
b/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java
index a1ba264..1eed5b4 100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java
@@ -73,7 +73,8 @@ public class RecoveryManager {
         CacheBuilder.newBuilder().expireAfterWrite(timeToCacheExistsInMillis, 
TimeUnit.MILLISECONDS)
             .maximumWeight(10_000_000).weigher((path, exist) -> 
path.toString().length()).build();
 
-    executor = ThreadPools.createScheduledExecutorService(4, "Walog sort 
starter", false);
+    executor = 
ThreadPools.getServerThreadPools().createScheduledExecutorService(4,
+        "Walog sort starter", false);
     zooCache = new ZooCache(manager.getContext().getZooReader(), null);
     try {
       List<String> workIDs =
diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/BulkImport.java
 
b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/BulkImport.java
index 35b58c7..101b449 100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/BulkImport.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/BulkImport.java
@@ -200,7 +200,7 @@ public class BulkImport extends ManagerRepo {
 
     AccumuloConfiguration serverConfig = manager.getConfiguration();
     @SuppressWarnings("deprecation")
-    ExecutorService workers = ThreadPools.createExecutorService(serverConfig,
+    ExecutorService workers = 
ThreadPools.getServerThreadPools().createExecutorService(serverConfig,
         serverConfig.resolve(Property.MANAGER_RENAME_THREADS, 
Property.MANAGER_BULK_RENAME_THREADS),
         false);
     List<Future<Exception>> results = new ArrayList<>();
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 149e9a5..827631e 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
@@ -91,8 +91,8 @@ class LoadFiles extends ManagerRepo {
 
   private static synchronized ExecutorService getThreadPool(Manager manager) {
     if (threadPool == null) {
-      threadPool = 
ThreadPools.createExecutorService(manager.getConfiguration(),
-          Property.MANAGER_BULK_THREADPOOL_SIZE, true);
+      threadPool = ThreadPools.getServerThreadPools().createExecutorService(
+          manager.getConfiguration(), Property.MANAGER_BULK_THREADPOOL_SIZE, 
true);
     }
     return threadPool;
   }
diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java
 
b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java
index de9db32..4b2b3a5 100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java
@@ -174,8 +174,9 @@ public class UpgradeCoordinator {
         "Not currently in a suitable state to do metadata upgrade %s", status);
 
     if (currentVersion < AccumuloDataVersion.get()) {
-      return ThreadPools.createThreadPool(0, Integer.MAX_VALUE, 60L, 
TimeUnit.SECONDS,
-          "UpgradeMetadataThreads", new SynchronousQueue<>(), false).submit(() 
-> {
+      return ThreadPools.getServerThreadPools().createThreadPool(0, 
Integer.MAX_VALUE, 60L,
+          TimeUnit.SECONDS, "UpgradeMetadataThreads", new 
SynchronousQueue<>(), false)
+          .submit(() -> {
             try {
               for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) 
{
                 log.info("Upgrading Root from data version {}", v);
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 c53bb91..f7a7af2 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
@@ -771,8 +771,8 @@ public class TabletServer extends AbstractServer {
       throw new RuntimeException(e);
     }
 
-    ThreadPoolExecutor distWorkQThreadPool =
-        ThreadPools.createExecutorService(getConfiguration(), 
Property.TSERV_WORKQ_THREADS, true);
+    ThreadPoolExecutor distWorkQThreadPool = ThreadPools.getServerThreadPools()
+        .createExecutorService(getConfiguration(), 
Property.TSERV_WORKQ_THREADS, true);
 
     bulkFailedCopyQ =
         new DistributedWorkQueue(getContext().getZooKeeperRoot() + 
Constants.ZBULK_FAILED_COPYQ,
@@ -806,8 +806,8 @@ public class TabletServer extends AbstractServer {
 
     int tabletCheckFrequency = 30 + random.nextInt(31); // random 30-60 minute 
delay
     // Periodically check that metadata of tablets matches what is held in 
memory
-    ThreadPools.watchCriticalScheduledTask(
-        
ThreadPools.createGeneralScheduledExecutorService(aconf).scheduleWithFixedDelay(()
 -> {
+    ThreadPools.watchCriticalScheduledTask(ThreadPools.getServerThreadPools()
+        
.createGeneralScheduledExecutorService(aconf).scheduleWithFixedDelay(() -> {
           final SortedMap<KeyExtent,Tablet> onlineTabletsSnapshot = 
onlineTablets.snapshot();
 
           Map<KeyExtent,Long> updateCounts = new HashMap<>();
@@ -951,7 +951,7 @@ public class TabletServer extends AbstractServer {
     }
 
     // Start the pool to handle outgoing replications
-    final ThreadPoolExecutor replicationThreadPool = ThreadPools
+    final ThreadPoolExecutor replicationThreadPool = 
ThreadPools.getServerThreadPools()
         .createExecutorService(getConfiguration(), 
Property.REPLICATION_WORKER_THREADS, false);
     replWorker.setExecutor(replicationThreadPool);
     replWorker.run();
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
index 23ed47f..74ef81c 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
@@ -187,9 +187,9 @@ public class TabletServerResourceManager {
 
     scanExecQueues.put(sec.name, queue);
 
-    ThreadPoolExecutor es =
-        ThreadPools.createThreadPool(sec.getCurrentMaxThreads(), 
sec.getCurrentMaxThreads(), 0L,
-            TimeUnit.MILLISECONDS, "scan-" + sec.name, queue, sec.priority, 
true);
+    ThreadPoolExecutor es = 
ThreadPools.getServerThreadPools().createThreadPool(
+        sec.getCurrentMaxThreads(), sec.getCurrentMaxThreads(), 0L, 
TimeUnit.MILLISECONDS,
+        "scan-" + sec.name, queue, sec.priority, true);
     modifyThreadPoolSizesAtRuntime(sec::getCurrentMaxThreads, "scan-" + 
sec.name, es);
     return es;
 
@@ -303,22 +303,23 @@ public class TabletServerResourceManager {
       log.warn("In-memory map may not fit into local memory space.");
     }
 
-    minorCompactionThreadPool =
-        ThreadPools.createExecutorService(acuConf, 
Property.TSERV_MINC_MAXCONCURRENT, true);
+    minorCompactionThreadPool = 
ThreadPools.getServerThreadPools().createExecutorService(acuConf,
+        Property.TSERV_MINC_MAXCONCURRENT, true);
     modifyThreadPoolSizesAtRuntime(
         () -> 
context.getConfiguration().getCount(Property.TSERV_MINC_MAXCONCURRENT),
         "minor compactor", minorCompactionThreadPool);
 
-    splitThreadPool = ThreadPools.createThreadPool(0, 1, 1, TimeUnit.SECONDS, 
"splitter", true);
+    splitThreadPool = ThreadPools.getServerThreadPools().createThreadPool(0, 
1, 1, TimeUnit.SECONDS,
+        "splitter", true);
 
-    defaultSplitThreadPool =
-        ThreadPools.createThreadPool(0, 1, 60, TimeUnit.SECONDS, "md 
splitter", true);
+    defaultSplitThreadPool = 
ThreadPools.getServerThreadPools().createThreadPool(0, 1, 60,
+        TimeUnit.SECONDS, "md splitter", true);
 
-    defaultMigrationPool =
-        ThreadPools.createThreadPool(0, 1, 60, TimeUnit.SECONDS, "metadata 
tablet migration", true);
+    defaultMigrationPool = 
ThreadPools.getServerThreadPools().createThreadPool(0, 1, 60,
+        TimeUnit.SECONDS, "metadata tablet migration", true);
 
-    migrationPool =
-        ThreadPools.createExecutorService(acuConf, 
Property.TSERV_MIGRATE_MAXCONCURRENT, true);
+    migrationPool = 
ThreadPools.getServerThreadPools().createExecutorService(acuConf,
+        Property.TSERV_MIGRATE_MAXCONCURRENT, true);
     modifyThreadPoolSizesAtRuntime(
         () -> 
context.getConfiguration().getCount(Property.TSERV_MIGRATE_MAXCONCURRENT),
         "tablet migration", migrationPool);
@@ -328,31 +329,31 @@ public class TabletServerResourceManager {
     // individual tablet servers are already running assignments 
concurrently... having each
     // individual tablet server run
     // concurrent assignments would put more load on the metadata table at 
startup
-    assignmentPool =
-        ThreadPools.createExecutorService(acuConf, 
Property.TSERV_ASSIGNMENT_MAXCONCURRENT, true);
+    assignmentPool = 
ThreadPools.getServerThreadPools().createExecutorService(acuConf,
+        Property.TSERV_ASSIGNMENT_MAXCONCURRENT, true);
     modifyThreadPoolSizesAtRuntime(
         () -> 
context.getConfiguration().getCount(Property.TSERV_ASSIGNMENT_MAXCONCURRENT),
         "tablet assignment", assignmentPool);
 
-    assignMetaDataPool = ThreadPools.createThreadPool(0, 1, 60, 
TimeUnit.SECONDS,
-        "metadata tablet assignment", true);
+    assignMetaDataPool = 
ThreadPools.getServerThreadPools().createThreadPool(0, 1, 60,
+        TimeUnit.SECONDS, "metadata tablet assignment", true);
 
     activeAssignments = new ConcurrentHashMap<>();
 
-    summaryRetrievalPool =
-        ThreadPools.createExecutorService(acuConf, 
Property.TSERV_SUMMARY_RETRIEVAL_THREADS, true);
+    summaryRetrievalPool = 
ThreadPools.getServerThreadPools().createExecutorService(acuConf,
+        Property.TSERV_SUMMARY_RETRIEVAL_THREADS, true);
     modifyThreadPoolSizesAtRuntime(
         () -> 
context.getConfiguration().getCount(Property.TSERV_SUMMARY_RETRIEVAL_THREADS),
         "summary file retriever", summaryRetrievalPool);
 
-    summaryRemotePool =
-        ThreadPools.createExecutorService(acuConf, 
Property.TSERV_SUMMARY_REMOTE_THREADS, true);
+    summaryRemotePool = 
ThreadPools.getServerThreadPools().createExecutorService(acuConf,
+        Property.TSERV_SUMMARY_REMOTE_THREADS, true);
     modifyThreadPoolSizesAtRuntime(
         () -> 
context.getConfiguration().getCount(Property.TSERV_SUMMARY_REMOTE_THREADS),
         "summary remote", summaryRemotePool);
 
-    summaryPartitionPool =
-        ThreadPools.createExecutorService(acuConf, 
Property.TSERV_SUMMARY_PARTITION_THREADS, true);
+    summaryPartitionPool = 
ThreadPools.getServerThreadPools().createExecutorService(acuConf,
+        Property.TSERV_SUMMARY_PARTITION_THREADS, true);
     modifyThreadPoolSizesAtRuntime(
         () -> 
context.getConfiguration().getCount(Property.TSERV_SUMMARY_PARTITION_THREADS),
         "summary partition", summaryPartitionPool);
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionService.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionService.java
index 83ba241..9053164 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionService.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionService.java
@@ -126,8 +126,8 @@ public class CompactionService {
 
     this.executors = Map.copyOf(tmpExecutors);
 
-    this.planningExecutor =
-        ThreadPools.createThreadPool(1, 1, 0L, TimeUnit.MILLISECONDS, 
"CompactionPlanner", false);
+    this.planningExecutor = 
ThreadPools.getServerThreadPools().createThreadPool(1, 1, 0L,
+        TimeUnit.MILLISECONDS, "CompactionPlanner", false);
 
     this.queuedForPlanning = new EnumMap<>(CompactionKind.class);
     for (CompactionKind kind : CompactionKind.values()) {
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/InternalCompactionExecutor.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/InternalCompactionExecutor.java
index 56b0814..7c7cff5 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/InternalCompactionExecutor.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/InternalCompactionExecutor.java
@@ -162,8 +162,8 @@ public class InternalCompactionExecutor implements 
CompactionExecutor {
 
     queue = new PriorityBlockingQueue<>(100, comparator);
 
-    threadPool = ThreadPools.createThreadPool(threads, threads, 60, 
TimeUnit.SECONDS,
-        "compaction." + ceid, queue, false);
+    threadPool = ThreadPools.getServerThreadPools().createThreadPool(threads, 
threads, 60,
+        TimeUnit.SECONDS, "compaction." + ceid, queue, false);
 
     metricCloser =
         ceMetrics.addExecutor(ceid, () -> threadPool.getActiveCount(), () -> 
queuedJob.size());
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LogSorter.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LogSorter.java
index f192372..63eb769 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LogSorter.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LogSorter.java
@@ -214,8 +214,8 @@ public class LogSorter {
     @SuppressWarnings("deprecation")
     int threadPoolSize = 
conf.getCount(conf.resolve(Property.TSERV_WAL_SORT_MAX_CONCURRENT,
         Property.TSERV_RECOVERY_MAX_CONCURRENT));
-    this.threadPool =
-        ThreadPools.createFixedThreadPool(threadPoolSize, 
this.getClass().getName(), true);
+    this.threadPool = 
ThreadPools.getServerThreadPools().createFixedThreadPool(threadPoolSize,
+        this.getClass().getName(), true);
     this.walBlockSize = DfsLogger.getWalBlockSize(conf);
   }
 
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 92089a0..5855bd7 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -262,7 +262,8 @@ public class TabletServerLogger {
     if (nextLogMaker != null) {
       return;
     }
-    nextLogMaker = ThreadPools.createFixedThreadPool(1, "WALog creator", true);
+    nextLogMaker =
+        ThreadPools.getServerThreadPools().createFixedThreadPool(1, "WALog 
creator", true);
     nextLogMaker.execute(new Runnable() {
       @Override
       public void run() {
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/CompactionExecutorsMetrics.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/CompactionExecutorsMetrics.java
index 955d3a2..a9fa3cb 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/CompactionExecutorsMetrics.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/CompactionExecutorsMetrics.java
@@ -65,8 +65,8 @@ public class CompactionExecutorsMetrics implements 
MetricsProducer {
 
   public CompactionExecutorsMetrics() {
 
-    ScheduledExecutorService scheduler =
-        ThreadPools.createScheduledExecutorService(1, 
"compactionExecutorsMetricsPoller", false);
+    ScheduledExecutorService scheduler = ThreadPools.getServerThreadPools()
+        .createScheduledExecutorService(1, "compactionExecutorsMetricsPoller", 
false);
     Runtime.getRuntime().addShutdownHook(new Thread(scheduler::shutdownNow));
     long minimumRefreshDelay = TimeUnit.SECONDS.toMillis(5);
     ScheduledFuture<?> future = scheduler.scheduleAtFixedRate(this::update, 
minimumRefreshDelay,
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
index 3072896..636ffb5 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
@@ -61,6 +61,7 @@ import 
org.apache.accumulo.core.securityImpl.thrift.TCredentials;
 import org.apache.accumulo.core.singletons.SingletonReservation;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.replication.ReplicaSystem;
 import org.apache.accumulo.server.replication.ReplicaSystemHelper;
@@ -571,7 +572,7 @@ public class AccumuloReplicaSystem implements ReplicaSystem 
{
     ClientProperty.setAuthenticationToken(properties, token);
 
     return new ClientContext(SingletonReservation.noop(), 
ClientInfo.from(properties, token),
-        localConf);
+        localConf, Threads.UEH);
   }
 
   protected Set<Integer> consumeWalPrefix(ReplicationTarget target, 
DataInputStream wal,
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java
index ae43495..8c21c1d 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java
@@ -279,8 +279,8 @@ public class SessionManager {
         }
       };
 
-      ScheduledFuture<?> future = 
ThreadPools.createGeneralScheduledExecutorService(aconf)
-          .schedule(r, delay, TimeUnit.MILLISECONDS);
+      ScheduledFuture<?> future = ThreadPools.getServerThreadPools()
+          .createGeneralScheduledExecutorService(aconf).schedule(r, delay, 
TimeUnit.MILLISECONDS);
       ThreadPools.watchNonCriticalScheduledTask(future);
     }
   }
diff --git 
a/test/src/main/java/org/apache/accumulo/test/BalanceWithOfflineTableIT.java 
b/test/src/main/java/org/apache/accumulo/test/BalanceWithOfflineTableIT.java
index 28043b4..26cfcc6 100644
--- a/test/src/main/java/org/apache/accumulo/test/BalanceWithOfflineTableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/BalanceWithOfflineTableIT.java
@@ -77,7 +77,8 @@ public class BalanceWithOfflineTableIT extends 
ConfigurableMacBase {
 
       log.info("Waiting for balance");
 
-      ExecutorService pool = ThreadPools.createFixedThreadPool(1, 
"waitForBalance", false);
+      ExecutorService pool =
+          ThreadPools.getServerThreadPools().createFixedThreadPool(1, 
"waitForBalance", false);
       Future<Boolean> wait = pool.submit(() -> {
         c.instanceOperations().waitForBalance();
         return true;
diff --git 
a/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
 
b/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
index 65af3e2..97022c5 100644
--- 
a/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
+++ 
b/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
@@ -207,8 +207,8 @@ public class BatchWriterFlushIT extends 
AccumuloClusterHarness {
         allMuts.add(muts);
       }
 
-      ThreadPoolExecutor threads =
-          ThreadPools.createFixedThreadPool(NUM_THREADS, "ClientThreads", 
false);
+      ThreadPoolExecutor threads = ThreadPools.getServerThreadPools()
+          .createFixedThreadPool(NUM_THREADS, "ClientThreads", false);
       threads.allowCoreThreadTimeOut(false);
       threads.prestartAllCoreThreads();
 

Reply via email to