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/01/28 15:33:58 UTC

[accumulo] branch main updated: Modified ThreadPools to conditionally register metrics (#2432)

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 36fca0f  Modified ThreadPools to conditionally register metrics (#2432)
36fca0f is described below

commit 36fca0f3c8f7481e2ec400a486e6955026ba7af4
Author: Dave Marion <dl...@apache.org>
AuthorDate: Fri Jan 28 10:33:48 2022 -0500

    Modified ThreadPools to conditionally register metrics (#2432)
    
    When ThreadPools calls ExecutorServiceMetrics, it creates Meters
    that are registered with the MeterRegistry. However, there is no
    easy way to remove the Meters from the MeterRegistry when the
    ThreadPool is shut down. Modified the ThreadPools methods to accept
    a boolean value for registering metrics for the newly created
    ThreadPool. Modified classes that use ThreadPools class to only
    pass true for this boolean value when the ThreadPool is long lived.
    
    Closes #2423
---
 .../core/clientImpl/ConditionalWriterImpl.java     |   4 +-
 .../core/clientImpl/InstanceOperationsImpl.java    |   3 +-
 .../accumulo/core/clientImpl/ScannerIterator.java  |   6 +-
 .../core/clientImpl/TableOperationsImpl.java       |   2 +-
 .../core/clientImpl/TabletServerBatchReader.java   |   2 +-
 .../core/clientImpl/TabletServerBatchWriter.java   |   5 +-
 .../accumulo/core/clientImpl/bulk/BulkImport.java  |   4 +-
 .../accumulo/core/file/BloomFilterLayer.java       |   4 +-
 .../file/blockfile/cache/lru/LruBlockCache.java    |   2 +-
 .../blockfile/cache/tinylfu/TinyLfuBlockCache.java |   2 +-
 .../util/compaction/ExternalCompactionUtil.java    |   4 +-
 .../accumulo/core/util/threads/ThreadPools.java    | 221 +++++++++++++++++----
 .../main/java/org/apache/accumulo/fate/Fate.java   |   2 +-
 .../core/file/rfile/MultiThreadedRFileTest.java    |   2 +-
 .../org/apache/accumulo/server/ServerContext.java  |   5 +-
 .../accumulo/server/client/BulkImporter.java       |   8 +-
 .../accumulo/server/fs/VolumeManagerImpl.java      |   2 +-
 .../accumulo/server/problems/ProblemReports.java   |   2 +-
 .../apache/accumulo/server/rpc/TServerUtils.java   |   2 +-
 .../server/util/RemoveEntriesForMissingFiles.java  |   2 +-
 .../server/util/VerifyTabletAssignments.java       |   2 +-
 .../coordinator/CompactionCoordinator.java         |   3 +-
 .../accumulo/coordinator/CompactionFinalizer.java  |   6 +-
 .../apache/accumulo/gc/SimpleGarbageCollector.java |   2 +-
 .../java/org/apache/accumulo/manager/Manager.java  |   2 +-
 .../manager/metrics/ReplicationMetrics.java        |   2 +-
 .../accumulo/manager/metrics/fate/FateMetrics.java |   2 +-
 .../accumulo/manager/recovery/RecoveryManager.java |   2 +-
 .../manager/tableOps/bulkVer1/BulkImport.java      |   5 +-
 .../manager/tableOps/bulkVer1/LoadFiles.java       |   2 +-
 .../manager/upgrade/UpgradeCoordinator.java        |   2 +-
 .../org/apache/accumulo/tserver/TabletServer.java  |   4 +-
 .../tserver/TabletServerResourceManager.java       |  24 +--
 .../tserver/compactions/CompactionService.java     |   2 +-
 .../compactions/InternalCompactionExecutor.java    |   2 +-
 .../org/apache/accumulo/tserver/log/LogSorter.java |   3 +-
 .../accumulo/tserver/log/TabletServerLogger.java   |   2 +-
 .../metrics/CompactionExecutorsMetrics.java        |   2 +-
 .../accumulo/test/BalanceWithOfflineTableIT.java   |   2 +-
 .../test/functional/BatchWriterFlushIT.java        |   3 +-
 40 files changed, 260 insertions(+), 98 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 57221f7..182d59e 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
@@ -88,7 +88,7 @@ import org.apache.thrift.transport.TTransportException;
 class ConditionalWriterImpl implements ConditionalWriter {
 
   private static ThreadPoolExecutor cleanupThreadPool = ThreadPools.createFixedThreadPool(1, 3,
-      TimeUnit.SECONDS, "Conditional Writer Cleanup Thread");
+      TimeUnit.SECONDS, "Conditional Writer Cleanup Thread", true);
 
   private static final int MAX_SLEEP = 30000;
 
@@ -362,7 +362,7 @@ class ConditionalWriterImpl implements ConditionalWriter {
     this.auths = config.getAuthorizations();
     this.ve = new VisibilityEvaluator(config.getAuthorizations());
     this.threadPool = ThreadPools.createScheduledExecutorService(config.getMaxWriteThreads(),
-        this.getClass().getSimpleName());
+        this.getClass().getSimpleName(), false);
     this.locator = new SyncingTabletLocator(context, tableId);
     this.serverQueues = new HashMap<>();
     this.tableId = tableId;
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 8708749..5075ad7 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
@@ -219,7 +219,8 @@ public class InstanceOperationsImpl implements InstanceOperations {
     List<String> tservers = getTabletServers();
 
     int numThreads = Math.max(4, Math.min((tservers.size() + compactors.size()) / 10, 256));
-    var executorService = ThreadPools.createFixedThreadPool(numThreads, "getactivecompactions");
+    var executorService =
+        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 06f7e51..f8d3912 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
@@ -65,9 +65,9 @@ public class ScannerIterator implements Iterator<Entry<Key,Value>> {
 
   private ScannerImpl.Reporter reporter;
 
-  private static ThreadPoolExecutor readaheadPool =
-      ThreadPools.createThreadPool(0, Integer.MAX_VALUE, 3L, TimeUnit.SECONDS,
-          "Accumulo scanner read ahead thread", new SynchronousQueue<>(), OptionalInt.empty());
+  private static ThreadPoolExecutor readaheadPool = ThreadPools.createThreadPool(0,
+      Integer.MAX_VALUE, 3L, TimeUnit.SECONDS, "Accumulo scanner read ahead thread",
+      new SynchronousQueue<>(), OptionalInt.empty(), true);
 
   private boolean closed = false;
 
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 21ef77d..892805d 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
@@ -487,7 +487,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     CountDownLatch latch = new CountDownLatch(splits.size());
     AtomicReference<Exception> exception = new AtomicReference<>(null);
 
-    ExecutorService executor = ThreadPools.createFixedThreadPool(16, "addSplits");
+    ExecutorService executor = 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 f6b1cab..55e70e3 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
@@ -73,7 +73,7 @@ public class TabletServerBatchReader extends ScannerOptions implements BatchScan
     this.numThreads = numQueryThreads;
 
     queryThreadPool = ThreadPools.createFixedThreadPool(numQueryThreads,
-        "batch scanner " + batchReaderInstance + "-");
+        "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 e9bb6ae..fe25407 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
@@ -645,10 +645,11 @@ public class TabletServerBatchWriter implements AutoCloseable {
     public MutationWriter(int numSendThreads) {
       serversMutations = new HashMap<>();
       queued = new HashSet<>();
-      sendThreadPool = ThreadPools.createFixedThreadPool(numSendThreads, this.getClass().getName());
+      sendThreadPool =
+          ThreadPools.createFixedThreadPool(numSendThreads, this.getClass().getName(), false);
       locators = new HashMap<>();
       binningThreadPool =
-          ThreadPools.createFixedThreadPool(1, "BinMutations", new SynchronousQueue<>());
+          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 4b82ebb..9a97e75 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
@@ -478,11 +478,11 @@ public class BulkImport implements ImportDestinationArguments, ImportMappingOpti
     if (this.executor != null) {
       executor = this.executor;
     } else if (numThreads > 0) {
-      executor = service = ThreadPools.createFixedThreadPool(numThreads, "BulkImportThread");
+      executor = service = ThreadPools.createFixedThreadPool(numThreads, "BulkImportThread", false);
     } else {
       String threads = context.getConfiguration().get(ClientProperty.BULK_LOAD_THREADS.getKey());
       executor = service = ThreadPools.createFixedThreadPool(
-          ConfigurationTypeHelper.getNumThreads(threads), "BulkImportThread");
+          ConfigurationTypeHelper.getNumThreads(threads), "BulkImportThread", false);
     }
 
     try {
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 cc9ca6d..778e7a3 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
@@ -78,8 +78,8 @@ public class BloomFilterLayer {
     }
 
     if (maxLoadThreads > 0) {
-      loadThreadPool =
-          ThreadPools.createThreadPool(0, maxLoadThreads, 60, TimeUnit.SECONDS, "bloom-loader");
+      loadThreadPool = ThreadPools.createThreadPool(0, maxLoadThreads, 60, TimeUnit.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 26ed14f..cadc4e0 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
@@ -101,7 +101,7 @@ public class LruBlockCache extends SynchronousLoadingBlockCache implements Block
 
   /** Statistics thread schedule pool (for heavy debugging, could remove) */
   private final ScheduledExecutorService scheduleThreadPool =
-      ThreadPools.createScheduledExecutorService(1, "LRUBlockCacheStats");
+      ThreadPools.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 38da111..dba920f 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
@@ -59,7 +59,7 @@ public final class TinyLfuBlockCache implements BlockCache {
   private final Policy.Eviction<String,Block> policy;
   private final int maxSize;
   private final ScheduledExecutorService statsExecutor =
-      ThreadPools.createScheduledExecutorService(1, "TinyLfuBlockCacheStatsExecutor");
+      ThreadPools.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 04d470e..d034d4e 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
@@ -233,7 +233,7 @@ public class ExternalCompactionUtil {
   public static List<RunningCompaction> getCompactionsRunningOnCompactors(ClientContext context) {
     final List<RunningCompactionFuture> rcFutures = new ArrayList<>();
     final ExecutorService executor =
-        ThreadPools.createFixedThreadPool(16, "CompactorRunningCompactions");
+        ThreadPools.createFixedThreadPool(16, "CompactorRunningCompactions", false);
 
     getCompactorAddrs(context).forEach((q, hp) -> {
       hp.forEach(hostAndPort -> {
@@ -261,7 +261,7 @@ public class ExternalCompactionUtil {
   public static Collection<ExternalCompactionId>
       getCompactionIdsRunningOnCompactors(ClientContext context) {
     final ExecutorService executor =
-        ThreadPools.createFixedThreadPool(16, "CompactorRunningCompactions");
+        ThreadPools.createFixedThreadPool(16, "CompactorRunningCompactions", false);
 
     List<Future<ExternalCompactionId>> futures = new ArrayList<>();
 
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 25c3c7c..25807f6 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
@@ -98,87 +98,201 @@ public class ThreadPools {
    *          accumulo configuration
    * @param p
    *          thread pool related property
+   * @param emitThreadPoolMetrics
+   *          When set to true will emit metrics and register the metrics in a static registry.
+   *          After the thread pool is deleted, there will still be metrics objects related to it in
+   *          the static registry. There is no way to clean these left over objects up therefore its
+   *          recommended that this option only be set true for long lived thread pools. Creating
+   *          lots of short lived thread pools and registering them can lead to out of memory errors
+   *          over long time periods.
    * @return ExecutorService impl
    * @throws RuntimeException
    *           if property is not handled
    */
   @SuppressWarnings("deprecation")
   public static ExecutorService createExecutorService(final AccumuloConfiguration conf,
-      final Property p) {
+      final Property p, boolean emitThreadPoolMetrics) {
 
     switch (p) {
       case GENERAL_SIMPLETIMER_THREADPOOL_SIZE:
-        return createScheduledExecutorService(conf.getCount(p), "SimpleTimer");
+        return createScheduledExecutorService(conf.getCount(p), "SimpleTimer",
+            emitThreadPoolMetrics);
       case MANAGER_BULK_THREADPOOL_SIZE:
         return createFixedThreadPool(conf.getCount(p),
             conf.getTimeInMillis(Property.MANAGER_BULK_THREADPOOL_TIMEOUT), TimeUnit.MILLISECONDS,
-            "bulk import");
+            "bulk import", emitThreadPoolMetrics);
       case MANAGER_RENAME_THREADS:
-        return createFixedThreadPool(conf.getCount(p), "bulk move");
+        return createFixedThreadPool(conf.getCount(p), "bulk move", emitThreadPoolMetrics);
       case MANAGER_FATE_THREADPOOL_SIZE:
-        return createFixedThreadPool(conf.getCount(p), "Repo Runner");
+        return createFixedThreadPool(conf.getCount(p), "Repo Runner", emitThreadPoolMetrics);
       case MANAGER_STATUS_THREAD_POOL_SIZE:
         int threads = conf.getCount(p);
         if (threads == 0) {
           return createThreadPool(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS,
-              "GatherTableInformation", new SynchronousQueue<Runnable>(), OptionalInt.empty());
+              "GatherTableInformation", new SynchronousQueue<Runnable>(), OptionalInt.empty(),
+              emitThreadPoolMetrics);
         } else {
-          return createFixedThreadPool(threads, "GatherTableInformation");
+          return createFixedThreadPool(threads, "GatherTableInformation", emitThreadPoolMetrics);
         }
       case TSERV_WORKQ_THREADS:
-        return createFixedThreadPool(conf.getCount(p), "distributed work queue");
+        return createFixedThreadPool(conf.getCount(p), "distributed work queue",
+            emitThreadPoolMetrics);
       case TSERV_MINC_MAXCONCURRENT:
-        return createFixedThreadPool(conf.getCount(p), 0L, TimeUnit.MILLISECONDS,
-            "minor compactor");
+        return createFixedThreadPool(conf.getCount(p), 0L, TimeUnit.MILLISECONDS, "minor compactor",
+            emitThreadPoolMetrics);
       case TSERV_MIGRATE_MAXCONCURRENT:
         return createFixedThreadPool(conf.getCount(p), 0L, TimeUnit.MILLISECONDS,
-            "tablet migration");
+            "tablet migration", emitThreadPoolMetrics);
       case TSERV_ASSIGNMENT_MAXCONCURRENT:
         return createFixedThreadPool(conf.getCount(p), 0L, TimeUnit.MILLISECONDS,
-            "tablet assignment");
+            "tablet assignment", emitThreadPoolMetrics);
       case TSERV_SUMMARY_RETRIEVAL_THREADS:
         return createThreadPool(conf.getCount(p), conf.getCount(p), 60, TimeUnit.SECONDS,
-            "summary file retriever");
+            "summary file retriever", emitThreadPoolMetrics);
       case TSERV_SUMMARY_REMOTE_THREADS:
         return createThreadPool(conf.getCount(p), conf.getCount(p), 60, TimeUnit.SECONDS,
-            "summary remote");
+            "summary remote", emitThreadPoolMetrics);
       case TSERV_SUMMARY_PARTITION_THREADS:
         return createThreadPool(conf.getCount(p), conf.getCount(p), 60, TimeUnit.SECONDS,
-            "summary partition");
+            "summary partition", emitThreadPoolMetrics);
       case GC_DELETE_THREADS:
-        return createFixedThreadPool(conf.getCount(p), "deleting");
+        return createFixedThreadPool(conf.getCount(p), "deleting", emitThreadPoolMetrics);
       case REPLICATION_WORKER_THREADS:
-        return createFixedThreadPool(conf.getCount(p), "replication task");
+        return createFixedThreadPool(conf.getCount(p), "replication task", emitThreadPoolMetrics);
       default:
         throw new RuntimeException("Unhandled thread pool property: " + p);
     }
   }
 
-  public static ThreadPoolExecutor createFixedThreadPool(int numThreads, final String name) {
-    return createFixedThreadPool(numThreads, DEFAULT_TIMEOUT_MILLISECS, TimeUnit.MILLISECONDS,
-        name);
+  /**
+   * Create a named thread pool
+   *
+   * @param numThreads
+   *          number of threads
+   * @param name
+   *          thread pool name
+   * @param emitThreadPoolMetrics
+   *          When set to true will emit metrics and register the metrics in a static registry.
+   *          After the thread pool is deleted, there will still be metrics objects related to it in
+   *          the static registry. There is no way to clean these left over objects up therefore its
+   *          recommended that this option only be set true for long lived thread pools. Creating
+   *          lots of short lived thread pools and registering them can lead to out of memory errors
+   *          over long time periods.
+   * @return ThreadPoolExecutor
+   */
+  public static ThreadPoolExecutor createFixedThreadPool(int numThreads, final String name,
+      boolean emitThreadPoolMetrics) {
+    return createFixedThreadPool(numThreads, DEFAULT_TIMEOUT_MILLISECS, TimeUnit.MILLISECONDS, name,
+        emitThreadPoolMetrics);
   }
 
+  /**
+   * Create a named thread pool
+   *
+   * @param numThreads
+   *          number of threads
+   * @param name
+   *          thread pool name
+   * @param queue
+   *          queue to use for tasks
+   * @param emitThreadPoolMetrics
+   *          When set to true will emit metrics and register the metrics in a static registry.
+   *          After the thread pool is deleted, there will still be metrics objects related to it in
+   *          the static registry. There is no way to clean these left over objects up therefore its
+   *          recommended that this option only be set true for long lived thread pools. Creating
+   *          lots of short lived thread pools and registering them can lead to out of memory errors
+   *          over long time periods.
+   * @return ThreadPoolExecutor
+   */
   public static ThreadPoolExecutor createFixedThreadPool(int numThreads, final String name,
-      BlockingQueue<Runnable> queue) {
+      BlockingQueue<Runnable> queue, boolean emitThreadPoolMetrics) {
     return createThreadPool(numThreads, numThreads, DEFAULT_TIMEOUT_MILLISECS,
-        TimeUnit.MILLISECONDS, name, queue, OptionalInt.empty());
+        TimeUnit.MILLISECONDS, name, queue, OptionalInt.empty(), emitThreadPoolMetrics);
   }
 
+  /**
+   * Create a named thread pool
+   *
+   * @param numThreads
+   *          number of threads
+   * @param timeOut
+   *          core thread time out
+   * @param units
+   *          core thread time out units
+   * @param name
+   *          thread pool name
+   * @param emitThreadPoolMetrics
+   *          When set to true will emit metrics and register the metrics in a static registry.
+   *          After the thread pool is deleted, there will still be metrics objects related to it in
+   *          the static registry. There is no way to clean these left over objects up therefore its
+   *          recommended that this option only be set true for long lived thread pools. Creating
+   *          lots of short lived thread pools and registering them can lead to out of memory errors
+   *          over long time periods.
+   * @return ThreadPoolExecutor
+   */
   public static ThreadPoolExecutor createFixedThreadPool(int numThreads, long timeOut,
-      TimeUnit units, final String name) {
+      TimeUnit units, final String name, boolean emitThreadPoolMetrics) {
     return createThreadPool(numThreads, numThreads, timeOut, units, name,
-        new LinkedBlockingQueue<Runnable>(), OptionalInt.empty());
+        new LinkedBlockingQueue<Runnable>(), OptionalInt.empty(), emitThreadPoolMetrics);
   }
 
+  /**
+   * Create a named thread pool
+   *
+   * @param coreThreads
+   *          number of threads
+   * @param maxThreads
+   *          max number of threads
+   * @param timeOut
+   *          core thread time out
+   * @param units
+   *          core thread time out units
+   * @param name
+   *          thread pool name
+   * @param emitThreadPoolMetrics
+   *          When set to true will emit metrics and register the metrics in a static registry.
+   *          After the thread pool is deleted, there will still be metrics objects related to it in
+   *          the static registry. There is no way to clean these left over objects up therefore its
+   *          recommended that this option only be set true for long lived thread pools. Creating
+   *          lots of short lived thread pools and registering them can lead to out of memory errors
+   *          over long time periods.
+   * @return ThreadPoolExecutor
+   */
   public static ThreadPoolExecutor createThreadPool(int coreThreads, int maxThreads, long timeOut,
-      TimeUnit units, final String name) {
+      TimeUnit units, final String name, boolean emitThreadPoolMetrics) {
     return createThreadPool(coreThreads, maxThreads, timeOut, units, name,
-        new LinkedBlockingQueue<Runnable>(), OptionalInt.empty());
+        new LinkedBlockingQueue<Runnable>(), OptionalInt.empty(), emitThreadPoolMetrics);
   }
 
+  /**
+   * Create a named thread pool
+   *
+   * @param coreThreads
+   *          number of threads
+   * @param maxThreads
+   *          max number of threads
+   * @param timeOut
+   *          core thread time out
+   * @param units
+   *          core thread time out units
+   * @param name
+   *          thread pool name
+   * @param queue
+   *          queue to use for tasks
+   * @param priority
+   *          thread priority
+   * @param emitThreadPoolMetrics
+   *          When set to true will emit metrics and register the metrics in a static registry.
+   *          After the thread pool is deleted, there will still be metrics objects related to it in
+   *          the static registry. There is no way to clean these left over objects up therefore its
+   *          recommended that this option only be set true for long lived thread pools. Creating
+   *          lots of short lived thread pools and registering them can lead to out of memory errors
+   *          over long time periods.
+   * @return ThreadPoolExecutor
+   */
   public static ThreadPoolExecutor createThreadPool(int coreThreads, int maxThreads, long timeOut,
-      TimeUnit units, final String name, BlockingQueue<Runnable> queue, OptionalInt priority) {
+      TimeUnit units, final String name, BlockingQueue<Runnable> queue, OptionalInt priority,
+      boolean emitThreadPoolMetrics) {
     ThreadPoolExecutor result = new ThreadPoolExecutor(coreThreads, maxThreads, timeOut, units,
         queue, new NamedThreadFactory(name, priority)) {
 
@@ -210,7 +324,9 @@ public class ThreadPools {
     if (timeOut > 0) {
       result.allowCoreThreadTimeOut(true);
     }
-    MetricsUtil.addExecutorServiceMetrics(result, name);
+    if (emitThreadPoolMetrics) {
+      MetricsUtil.addExecutorServiceMetrics(result, name);
+    }
     return result;
   }
 
@@ -221,16 +337,51 @@ public class ThreadPools {
   public static ScheduledThreadPoolExecutor
       createGeneralScheduledExecutorService(AccumuloConfiguration conf) {
     return (ScheduledThreadPoolExecutor) createExecutorService(conf,
-        Property.GENERAL_SIMPLETIMER_THREADPOOL_SIZE);
+        Property.GENERAL_SIMPLETIMER_THREADPOOL_SIZE, true);
   }
 
+  /**
+   * Create a named ScheduledThreadPool
+   *
+   * @param numThreads
+   *          number of threads
+   * @param name
+   *          thread pool name
+   * @param emitThreadPoolMetrics
+   *          When set to true will emit metrics and register the metrics in a static registry.
+   *          After the thread pool is deleted, there will still be metrics objects related to it in
+   *          the static registry. There is no way to clean these left over objects up therefore its
+   *          recommended that this option only be set true for long lived thread pools. Creating
+   *          lots of short lived thread pools and registering them can lead to out of memory errors
+   *          over long time periods.
+   * @return ScheduledThreadPoolExecutor
+   */
   public static ScheduledThreadPoolExecutor createScheduledExecutorService(int numThreads,
-      final String name) {
-    return createScheduledExecutorService(numThreads, name, OptionalInt.empty());
+      final String name, boolean emitThreadPoolMetrics) {
+    return createScheduledExecutorService(numThreads, name, OptionalInt.empty(),
+        emitThreadPoolMetrics);
   }
 
-  public static ScheduledThreadPoolExecutor createScheduledExecutorService(int numThreads,
-      final String name, OptionalInt priority) {
+  /**
+   * Create a named ScheduledThreadPool
+   *
+   * @param numThreads
+   *          number of threads
+   * @param name
+   *          thread pool name
+   * @param priority
+   *          thread priority
+   * @param emitThreadPoolMetrics
+   *          When set to true will emit metrics and register the metrics in a static registry.
+   *          After the thread pool is deleted, there will still be metrics objects related to it in
+   *          the static registry. There is no way to clean these left over objects up therefore its
+   *          recommended that this option only be set true for long lived thread pools. Creating
+   *          lots of short lived thread pools and registering them can lead to out of memory errors
+   *          over long time periods.
+   * @return ScheduledThreadPoolExecutor
+   */
+  private static ScheduledThreadPoolExecutor createScheduledExecutorService(int numThreads,
+      final String name, OptionalInt priority, boolean emitThreadPoolMetrics) {
     ScheduledThreadPoolExecutor result =
         new ScheduledThreadPoolExecutor(numThreads, new NamedThreadFactory(name, priority)) {
 
@@ -284,7 +435,9 @@ public class ThreadPools {
           }
 
         };
-    MetricsUtil.addExecutorServiceMetrics(result, name);
+    if (emitThreadPoolMetrics) {
+      MetricsUtil.addExecutorServiceMetrics(result, name);
+    }
     return result;
   }
 
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 cd53546..6f0fde2 100644
--- a/core/src/main/java/org/apache/accumulo/fate/Fate.java
+++ b/core/src/main/java/org/apache/accumulo/fate/Fate.java
@@ -230,7 +230,7 @@ public class Fate<T> {
    */
   public void startTransactionRunners(AccumuloConfiguration conf) {
     final ThreadPoolExecutor pool = (ThreadPoolExecutor) ThreadPools.createExecutorService(conf,
-        Property.MANAGER_FATE_THREADPOOL_SIZE);
+        Property.MANAGER_FATE_THREADPOOL_SIZE, true);
     fatePoolWatcher = ThreadPools.createGeneralScheduledExecutorService(conf);
     fatePoolWatcher.schedule(() -> {
       // resize the pool if the property changed
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 906f7ff..87dcf0e 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
@@ -241,7 +241,7 @@ public class MultiThreadedRFileTest {
       int maxThreads = 10;
       String name = "MultiThreadedRFileTestThread";
       ThreadPoolExecutor pool = ThreadPools.createThreadPool(maxThreads + 1, maxThreads + 1, 5 * 60,
-          TimeUnit.SECONDS, name, new LinkedBlockingQueue<>(), OptionalInt.empty());
+          TimeUnit.SECONDS, name, new LinkedBlockingQueue<>(), OptionalInt.empty(), 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 6e9a7dc..4e6b6b4 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
@@ -441,8 +441,9 @@ public class ServerContext extends ClientContext {
    */
   public synchronized ScheduledThreadPoolExecutor getScheduledExecutor() {
     if (sharedScheduledThreadPool == null) {
-      sharedScheduledThreadPool = (ScheduledThreadPoolExecutor) ThreadPools
-          .createExecutorService(getConfiguration(), Property.GENERAL_SIMPLETIMER_THREADPOOL_SIZE);
+      sharedScheduledThreadPool =
+          (ScheduledThreadPoolExecutor) ThreadPools.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 57d5474..131160a 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
@@ -129,7 +129,8 @@ public class BulkImporter {
           Collections.synchronizedSortedMap(new TreeMap<>());
 
       timer.start(Timers.EXAMINE_MAP_FILES);
-      ExecutorService threadPool = ThreadPools.createFixedThreadPool(numThreads, "findOverlapping");
+      ExecutorService threadPool =
+          ThreadPools.createFixedThreadPool(numThreads, "findOverlapping", false);
 
       for (Path path : paths) {
         final Path mapFile = path;
@@ -349,7 +350,8 @@ public class BulkImporter {
 
     final Map<Path,List<AssignmentInfo>> ais = Collections.synchronizedMap(new TreeMap<>());
 
-    ExecutorService threadPool = ThreadPools.createFixedThreadPool(numThreads, "estimateSizes");
+    ExecutorService threadPool =
+        ThreadPools.createFixedThreadPool(numThreads, "estimateSizes", false);
 
     for (final Entry<Path,List<TabletLocation>> entry : assignments.entrySet()) {
       if (entry.getValue().size() == 1) {
@@ -533,7 +535,7 @@ public class BulkImporter {
       }
     });
 
-    ExecutorService threadPool = ThreadPools.createFixedThreadPool(numThreads, "submit");
+    ExecutorService threadPool = ThreadPools.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 4e5533e..41ba6be 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,7 @@ 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);
+    ExecutorService workerPool = ThreadPools.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 bc97a2b..906fc00 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
@@ -68,7 +68,7 @@ public class ProblemReports implements Iterable<ProblemReport> {
    * 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), OptionalInt.empty());
+      "acu-problem-reporter", new LinkedBlockingQueue<>(500), OptionalInt.empty(), 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 48fee3b..429746c 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
@@ -313,7 +313,7 @@ public class TServerUtils {
       final int executorThreads, long threadTimeOut, final AccumuloConfiguration conf,
       long timeBetweenThreadChecks) {
     final ThreadPoolExecutor pool = ThreadPools.createFixedThreadPool(executorThreads,
-        threadTimeOut, TimeUnit.MILLISECONDS, serverName + "-ClientPool");
+        threadTimeOut, TimeUnit.MILLISECONDS, serverName + "-ClientPool", true);
     // periodically adjust the number of threads we need by checking how busy our threads are
     ThreadPools.createGeneralScheduledExecutorService(conf).scheduleWithFixedDelay(() -> {
       // there is a minor race condition between sampling the current state of the thread pool and
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 6928728..a7ee48c 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
@@ -126,7 +126,7 @@ public class RemoveEntriesForMissingFiles {
     @SuppressWarnings({"rawtypes"})
     Map cache = new LRUMap(100000);
     Set<Path> processing = new HashSet<>();
-    ExecutorService threadPool = ThreadPools.createFixedThreadPool(16, "CheckFileTasks");
+    ExecutorService threadPool = ThreadPools.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 7fb5dd8..f285426 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
@@ -125,7 +125,7 @@ public class VerifyTabletAssignments {
       }
     }
 
-    ExecutorService tp = ThreadPools.createFixedThreadPool(20, "CheckTabletServer");
+    ExecutorService tp = ThreadPools.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 74f6dab..fef096f 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
@@ -317,7 +317,8 @@ public class CompactionCoordinator extends AbstractServer
   }
 
   private void updateSummaries() {
-    ExecutorService executor = ThreadPools.createFixedThreadPool(10, "Compaction Summary Gatherer");
+    ExecutorService executor =
+        ThreadPools.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 d196481..424deb7 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");
+    this.ntfyExecutor = ThreadPools.createThreadPool(3, max, 1, TimeUnit.MINUTES,
+        "Compaction Finalizer Notifier", true);
 
     this.backgroundExecutor =
-        ThreadPools.createFixedThreadPool(1, "Compaction Finalizer Background Task");
+        ThreadPools.createFixedThreadPool(1, "Compaction Finalizer Background Task", true);
 
     backgroundExecutor.execute(() -> {
       processPending();
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 3c10c2b..d18d8bd 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
@@ -288,7 +288,7 @@ public class SimpleGarbageCollector extends AbstractServer implements Iface {
       minimizeDeletes(confirmedDeletes, processedDeletes, fs);
 
       ExecutorService deleteThreadPool =
-          ThreadPools.createExecutorService(getConfiguration(), Property.GC_DELETE_THREADS);
+          ThreadPools.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 66561ae..6e6b5ae 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
@@ -918,7 +918,7 @@ public class Manager extends AbstractServer
     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);
+        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/metrics/ReplicationMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ReplicationMetrics.java
index 9fa5565..35665cd 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
@@ -163,7 +163,7 @@ public class ReplicationMetrics implements MetricsProducer {
         new AtomicInteger(0));
 
     ScheduledExecutorService scheduler =
-        ThreadPools.createScheduledExecutorService(1, "replicationMetricsPoller");
+        ThreadPools.createScheduledExecutorService(1, "replicationMetricsPoller", false);
     Runtime.getRuntime().addShutdownHook(new Thread(scheduler::shutdownNow));
     long minimumRefreshDelay = TimeUnit.SECONDS.toMillis(5);
     scheduler.scheduleAtFixedRate(this::update, minimumRefreshDelay, 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 0a1fc3a..d87156c 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
@@ -146,7 +146,7 @@ public class FateMetrics implements MetricsProducer {
 
     // get fate status is read only operation - no reason to be nice on shutdown.
     ScheduledExecutorService scheduler =
-        ThreadPools.createScheduledExecutorService(1, "fateMetricsPoller");
+        ThreadPools.createScheduledExecutorService(1, "fateMetricsPoller", false);
     Runtime.getRuntime().addShutdownHook(new Thread(scheduler::shutdownNow));
 
     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 7c5500d..a617e93 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
@@ -72,7 +72,7 @@ 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 ");
+    executor = ThreadPools.createScheduledExecutorService(4, "Walog sort starter", false);
     zooCache = new ZooCache(manager.getContext().getZooReaderWriter(), 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 ca628f2..e433915 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
@@ -201,8 +201,9 @@ public class BulkImport extends ManagerRepo {
 
     AccumuloConfiguration serverConfig = manager.getConfiguration();
     @SuppressWarnings("deprecation")
-    ExecutorService workers = ThreadPools.createExecutorService(serverConfig, serverConfig
-        .resolve(Property.MANAGER_RENAME_THREADS, Property.MANAGER_BULK_RENAME_THREADS));
+    ExecutorService workers = ThreadPools.createExecutorService(serverConfig,
+        serverConfig.resolve(Property.MANAGER_RENAME_THREADS, Property.MANAGER_BULK_RENAME_THREADS),
+        false);
     List<Future<Exception>> results = new ArrayList<>();
 
     for (FileStatus file : mapFiles) {
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 a64fe8a..149e9a5 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
@@ -92,7 +92,7 @@ class LoadFiles extends ManagerRepo {
   private static synchronized ExecutorService getThreadPool(Manager manager) {
     if (threadPool == null) {
       threadPool = ThreadPools.createExecutorService(manager.getConfiguration(),
-          Property.MANAGER_BULK_THREADPOOL_SIZE);
+          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 c0e4f4e..d464b1e 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
@@ -176,7 +176,7 @@ public class UpgradeCoordinator {
 
     if (currentVersion < AccumuloDataVersion.get()) {
       return ThreadPools.createThreadPool(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS,
-          "UpgradeMetadataThreads", new SynchronousQueue<Runnable>(), OptionalInt.empty())
+          "UpgradeMetadataThreads", new SynchronousQueue<Runnable>(), OptionalInt.empty(), false)
           .submit(() -> {
             try {
               for (int v = currentVersion; v < AccumuloDataVersion.get(); 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 d454cce..84d106e 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
@@ -768,7 +768,7 @@ public class TabletServer extends AbstractServer {
     }
 
     ThreadPoolExecutor distWorkQThreadPool = (ThreadPoolExecutor) ThreadPools
-        .createExecutorService(getConfiguration(), Property.TSERV_WORKQ_THREADS);
+        .createExecutorService(getConfiguration(), Property.TSERV_WORKQ_THREADS, true);
 
     bulkFailedCopyQ =
         new DistributedWorkQueue(getContext().getZooKeeperRoot() + Constants.ZBULK_FAILED_COPYQ,
@@ -940,7 +940,7 @@ public class TabletServer extends AbstractServer {
 
     // Start the pool to handle outgoing replications
     final ThreadPoolExecutor replicationThreadPool = (ThreadPoolExecutor) ThreadPools
-        .createExecutorService(getConfiguration(), Property.REPLICATION_WORKER_THREADS);
+        .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 69054df..7f63210 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
@@ -188,7 +188,7 @@ public class TabletServerResourceManager {
 
     ExecutorService es =
         ThreadPools.createThreadPool(sec.getCurrentMaxThreads(), sec.getCurrentMaxThreads(), 0L,
-            TimeUnit.MILLISECONDS, "scan-" + sec.name, queue, sec.priority);
+            TimeUnit.MILLISECONDS, "scan-" + sec.name, queue, sec.priority, true);
     modifyThreadPoolSizesAtRuntime(sec::getCurrentMaxThreads, "scan-" + sec.name,
         (ThreadPoolExecutor) es);
     return es;
@@ -304,21 +304,21 @@ public class TabletServerResourceManager {
     }
 
     minorCompactionThreadPool =
-        ThreadPools.createExecutorService(acuConf, Property.TSERV_MINC_MAXCONCURRENT);
+        ThreadPools.createExecutorService(acuConf, Property.TSERV_MINC_MAXCONCURRENT, true);
     modifyThreadPoolSizesAtRuntime(
         () -> context.getConfiguration().getCount(Property.TSERV_MINC_MAXCONCURRENT),
         "minor compactor", (ThreadPoolExecutor) minorCompactionThreadPool);
 
-    splitThreadPool = ThreadPools.createThreadPool(0, 1, 1, TimeUnit.SECONDS, "splitter");
+    splitThreadPool = ThreadPools.createThreadPool(0, 1, 1, TimeUnit.SECONDS, "splitter", true);
 
     defaultSplitThreadPool =
-        ThreadPools.createThreadPool(0, 1, 60, TimeUnit.SECONDS, "md splitter");
+        ThreadPools.createThreadPool(0, 1, 60, TimeUnit.SECONDS, "md splitter", true);
 
     defaultMigrationPool =
-        ThreadPools.createThreadPool(0, 1, 60, TimeUnit.SECONDS, "metadata tablet migration");
+        ThreadPools.createThreadPool(0, 1, 60, TimeUnit.SECONDS, "metadata tablet migration", true);
 
     migrationPool =
-        ThreadPools.createExecutorService(acuConf, Property.TSERV_MIGRATE_MAXCONCURRENT);
+        ThreadPools.createExecutorService(acuConf, Property.TSERV_MIGRATE_MAXCONCURRENT, true);
     modifyThreadPoolSizesAtRuntime(
         () -> context.getConfiguration().getCount(Property.TSERV_MIGRATE_MAXCONCURRENT),
         "tablet migration", (ThreadPoolExecutor) migrationPool);
@@ -329,30 +329,30 @@ public class TabletServerResourceManager {
     // individual tablet server run
     // concurrent assignments would put more load on the metadata table at startup
     assignmentPool =
-        ThreadPools.createExecutorService(acuConf, Property.TSERV_ASSIGNMENT_MAXCONCURRENT);
+        ThreadPools.createExecutorService(acuConf, Property.TSERV_ASSIGNMENT_MAXCONCURRENT, true);
     modifyThreadPoolSizesAtRuntime(
         () -> context.getConfiguration().getCount(Property.TSERV_ASSIGNMENT_MAXCONCURRENT),
         "tablet assignment", (ThreadPoolExecutor) assignmentPool);
 
-    assignMetaDataPool =
-        ThreadPools.createThreadPool(0, 1, 60, TimeUnit.SECONDS, "metadata tablet assignment");
+    assignMetaDataPool = ThreadPools.createThreadPool(0, 1, 60, TimeUnit.SECONDS,
+        "metadata tablet assignment", true);
 
     activeAssignments = new ConcurrentHashMap<>();
 
     summaryRetrievalPool =
-        ThreadPools.createExecutorService(acuConf, Property.TSERV_SUMMARY_RETRIEVAL_THREADS);
+        ThreadPools.createExecutorService(acuConf, Property.TSERV_SUMMARY_RETRIEVAL_THREADS, true);
     modifyThreadPoolSizesAtRuntime(
         () -> context.getConfiguration().getCount(Property.TSERV_SUMMARY_RETRIEVAL_THREADS),
         "summary file retriever", (ThreadPoolExecutor) summaryRetrievalPool);
 
     summaryRemotePool =
-        ThreadPools.createExecutorService(acuConf, Property.TSERV_SUMMARY_REMOTE_THREADS);
+        ThreadPools.createExecutorService(acuConf, Property.TSERV_SUMMARY_REMOTE_THREADS, true);
     modifyThreadPoolSizesAtRuntime(
         () -> context.getConfiguration().getCount(Property.TSERV_SUMMARY_REMOTE_THREADS),
         "summary remote", (ThreadPoolExecutor) summaryRemotePool);
 
     summaryPartitionPool =
-        ThreadPools.createExecutorService(acuConf, Property.TSERV_SUMMARY_PARTITION_THREADS);
+        ThreadPools.createExecutorService(acuConf, Property.TSERV_SUMMARY_PARTITION_THREADS, true);
     modifyThreadPoolSizesAtRuntime(
         () -> context.getConfiguration().getCount(Property.TSERV_SUMMARY_PARTITION_THREADS),
         "summary partition", (ThreadPoolExecutor) 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 0829f26..83ba241 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
@@ -127,7 +127,7 @@ public class CompactionService {
     this.executors = Map.copyOf(tmpExecutors);
 
     this.planningExecutor =
-        ThreadPools.createThreadPool(1, 1, 0L, TimeUnit.MILLISECONDS, "CompactionPlanner");
+        ThreadPools.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 f60c41e..9e64cff 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
@@ -161,7 +161,7 @@ public class InternalCompactionExecutor implements CompactionExecutor {
     queue = new PriorityBlockingQueue<Runnable>(100, comparator);
 
     threadPool = ThreadPools.createThreadPool(threads, threads, 60, TimeUnit.SECONDS,
-        "compaction." + ceid, queue, OptionalInt.empty());
+        "compaction." + ceid, queue, OptionalInt.empty(), 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 766184f..f192372 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,7 +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());
+    this.threadPool =
+        ThreadPools.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 1f4eaf9..b884e61 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,7 @@ public class TabletServerLogger {
     if (nextLogMaker != null) {
       return;
     }
-    nextLogMaker = ThreadPools.createFixedThreadPool(1, "WALog creator");
+    nextLogMaker = ThreadPools.createFixedThreadPool(1, "WALog creator", true);
     nextLogMaker.submit(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 4b386bd..e5d79ad 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,7 +65,7 @@ public class CompactionExecutorsMetrics implements MetricsProducer {
   public CompactionExecutorsMetrics() {
 
     ScheduledExecutorService scheduler =
-        ThreadPools.createScheduledExecutorService(1, "compactionExecutorsMetricsPoller");
+        ThreadPools.createScheduledExecutorService(1, "compactionExecutorsMetricsPoller", false);
     Runtime.getRuntime().addShutdownHook(new Thread(scheduler::shutdownNow));
     long minimumRefreshDelay = TimeUnit.SECONDS.toMillis(5);
     scheduler.scheduleAtFixedRate(this::update, minimumRefreshDelay, minimumRefreshDelay,
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 52e8240..28043b4 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,7 @@ public class BalanceWithOfflineTableIT extends ConfigurableMacBase {
 
       log.info("Waiting for balance");
 
-      ExecutorService pool = ThreadPools.createFixedThreadPool(1, "waitForBalance");
+      ExecutorService pool = ThreadPools.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 c008ef7..65af3e2 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,7 +207,8 @@ public class BatchWriterFlushIT extends AccumuloClusterHarness {
         allMuts.add(muts);
       }
 
-      ThreadPoolExecutor threads = ThreadPools.createFixedThreadPool(NUM_THREADS, "ClientThreads");
+      ThreadPoolExecutor threads =
+          ThreadPools.createFixedThreadPool(NUM_THREADS, "ClientThreads", false);
       threads.allowCoreThreadTimeOut(false);
       threads.prestartAllCoreThreads();