You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by be...@apache.org on 2021/10/07 11:25:44 UTC

[cassandra] 02/06: [CASSANDRA-16925] CEP-10 Phase 1: Mockable Task Execution

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

benedict pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit e215d2a5b25991b47cec66cc1a970d835b89005b
Author: Benedict Elliott Smith <be...@apache.org>
AuthorDate: Wed Jul 28 19:47:45 2021 +0100

    [CASSANDRA-16925] CEP-10 Phase 1: Mockable Task Execution
    
    Co-authored-by: Benedict Elliott Smith <be...@apache.org>
    Co-authored-by: Sam Tunnicliffe <sa...@apache.org>
---
 build.xml                                          |  21 +-
 checkstyle.xml                                     |  42 +-
 checkstyle_suppressions.xml                        |  32 +-
 src/java/org/apache/cassandra/auth/AuthCache.java  |  47 +-
 .../apache/cassandra/auth/AuthenticatedUser.java   |   2 -
 .../apache/cassandra/batchlog/BatchlogManager.java |  13 +-
 .../apache/cassandra/cache/AutoSavingCache.java    |  25 +-
 .../AbstractLocalAwareExecutorService.java         | 229 ----------
 .../DebuggableScheduledThreadPoolExecutor.java     | 127 ------
 .../concurrent/DebuggableThreadPoolExecutor.java   | 426 ------------------
 .../cassandra/concurrent/ExecutionFailure.java     | 179 ++++++++
 .../cassandra/concurrent/ExecutorBuilder.java      |  92 ++++
 .../concurrent/ExecutorBuilderFactory.java         |  81 ++++
 .../cassandra/concurrent/ExecutorFactory.java      | 266 +++++++++++
 .../cassandra/concurrent/ExecutorLocals.java       |  85 ++--
 .../apache/cassandra/concurrent/ExecutorPlus.java  | 183 ++++++++
 .../apache/cassandra/concurrent/FutureTask.java    | 149 +++++++
 .../concurrent/FutureTaskWithResources.java        |  57 +++
 .../cassandra/concurrent/ImmediateExecutor.java    | 113 ++++-
 .../cassandra/concurrent/InfiniteLoopExecutor.java |  95 +++-
 ...ResizableThreadPool.java => Interruptible.java} |  37 +-
 .../concurrent/JMXEnabledSingleThreadExecutor.java |  82 ----
 .../concurrent/JMXEnabledThreadPoolExecutor.java   | 191 --------
 ...ThreadPool.java => LocalAwareExecutorPlus.java} |  24 +-
 .../concurrent/LocalAwareExecutorService.java      |  77 ----
 ....java => LocalAwareSequentialExecutorPlus.java} |  24 +-
 ...ava => LocalAwareSingleThreadExecutorPlus.java} |  26 +-
 ....java => LocalAwareThreadPoolExecutorPlus.java} |  26 +-
 .../cassandra/concurrent/NamedThreadFactory.java   | 110 ++++-
 .../cassandra/concurrent/ResizableThreadPool.java  |  39 +-
 ...orMBean.java => ResizableThreadPoolMXBean.java} |   2 +-
 .../apache/cassandra/concurrent/SEPExecutor.java   |  76 +++-
 .../org/apache/cassandra/concurrent/SEPWorker.java |  10 +-
 ...eThreadPool.java => ScheduledExecutorPlus.java} |  23 +-
 .../cassandra/concurrent/ScheduledExecutors.java   |  14 +-
 .../ScheduledThreadPoolExecutorPlus.java           | 240 ++++++++++
 .../concurrent/SequentialExecutorPlus.java         |  53 +++
 .../cassandra/concurrent/SharedExecutorPool.java   |  19 +-
 ...{ResizableThreadPool.java => Shutdownable.java} |  21 +-
 .../concurrent/SingleThreadExecutorPlus.java       | 100 +++++
 .../org/apache/cassandra/concurrent/Stage.java     | 121 ++---
 .../cassandra/concurrent/SyncFutureTask.java       |  70 +++
 .../apache/cassandra/concurrent/TaskFactory.java   | 178 ++++++++
 .../concurrent/ThreadPoolExecutorBase.java         | 186 ++++++++
 .../concurrent/ThreadPoolExecutorBuilder.java      | 204 +++++++++
 .../concurrent/ThreadPoolExecutorJMXAdapter.java   | 246 +++++++++++
 .../concurrent/ThreadPoolExecutorPlus.java         | 125 ++++++
 .../cassandra/concurrent/WrappedExecutorPlus.java  | 178 ++++++++
 .../cql3/functions/JavaBasedUDFunction.java        |   2 +-
 .../cql3/functions/ScriptBasedUDFunction.java      |   2 +-
 .../cql3/functions/UDFExecutorService.java         |  40 +-
 .../cassandra/cql3/functions/UDFunction.java       |   4 +-
 .../org/apache/cassandra/db/ColumnFamilyStore.java | 115 +++--
 src/java/org/apache/cassandra/db/Keyspace.java     |  25 +-
 src/java/org/apache/cassandra/db/Mutation.java     |   4 +-
 .../apache/cassandra/db/MutationVerbHandler.java   |   5 +-
 .../org/apache/cassandra/db/SystemKeyspace.java    |   4 +-
 .../commitlog/AbstractCommitLogSegmentManager.java | 138 +++---
 .../db/commitlog/AbstractCommitLogService.java     |  99 ++---
 .../cassandra/db/commitlog/CommitLogArchiver.java  |   9 +-
 .../cassandra/db/commitlog/CommitLogReplayer.java  |   3 +-
 .../cassandra/db/commitlog/CommitLogSegment.java   |   2 +-
 .../db/commitlog/CommitLogSegmentManagerCDC.java   |   8 +-
 .../cassandra/db/compaction/CompactionLogger.java  |  13 +-
 .../cassandra/db/compaction/CompactionManager.java | 163 +++----
 .../apache/cassandra/db/compaction/Scrubber.java   |   1 +
 .../cassandra/db/context/CounterContext.java       |   4 +-
 .../cassandra/db/monitoring/MonitoringTask.java    |   1 -
 .../db/repair/CassandraKeyspaceRepairManager.java  |  14 +-
 .../cassandra/db/repair/PendingAntiCompaction.java |  36 +-
 .../org/apache/cassandra/db/view/ViewBuilder.java  |  47 +-
 .../apache/cassandra/db/view/ViewBuilderTask.java  |   1 -
 .../org/apache/cassandra/dht/BootStrapper.java     |   5 +-
 src/java/org/apache/cassandra/gms/Gossiper.java    |  24 +-
 .../cassandra/hadoop/cql3/CqlInputFormat.java      |   3 +-
 src/java/org/apache/cassandra/hints/Hint.java      |   7 +-
 .../apache/cassandra/hints/HintVerbHandler.java    |   2 +-
 .../cassandra/hints/HintsDispatchExecutor.java     |  24 +-
 .../org/apache/cassandra/hints/HintsService.java   |   5 +-
 .../apache/cassandra/hints/HintsWriteExecutor.java |  12 +-
 .../cassandra/index/SecondaryIndexManager.java     |  98 ++---
 .../apache/cassandra/index/sasi/TermIterator.java  |  16 +-
 .../index/sasi/disk/PerSSTableIndexWriter.java     |  30 +-
 .../cassandra/io/sstable/IndexSummaryManager.java  |   9 +-
 .../io/sstable/SSTableSimpleUnsortedWriter.java    |  12 +-
 .../cassandra/io/sstable/format/SSTableReader.java |  68 +--
 .../cassandra/metrics/CompactionMetrics.java       |   6 +-
 src/java/org/apache/cassandra/metrics/Sampler.java |  33 +-
 .../cassandra/metrics/ThreadPoolMetrics.java       |   4 +-
 .../apache/cassandra/net/AsyncChannelPromise.java  |   2 +-
 .../org/apache/cassandra/net/FutureCombiner.java   |  86 ----
 .../org/apache/cassandra/net/FutureDelegate.java   |  46 +-
 .../org/apache/cassandra/net/FutureResult.java     |   7 +-
 .../cassandra/net/InboundMessageHandler.java       |   2 +-
 .../org/apache/cassandra/net/InboundSockets.java   |   9 +-
 .../org/apache/cassandra/net/MessagingService.java |   5 +-
 .../apache/cassandra/net/OutboundConnection.java   |   4 +-
 .../cassandra/net/OutboundConnectionInitiator.java |  12 +-
 .../apache/cassandra/net/OutboundConnections.java  |   7 +-
 .../org/apache/cassandra/net/RequestCallbacks.java |   6 +-
 .../org/apache/cassandra/net/SocketFactory.java    |   4 +-
 .../cassandra/repair/KeyspaceRepairManager.java    |  14 +-
 .../org/apache/cassandra/repair/RepairJob.java     | 108 +++--
 .../apache/cassandra/repair/RepairRunnable.java    | 141 +++---
 .../org/apache/cassandra/repair/RepairSession.java |  20 +-
 .../repair/asymmetric/RangeDenormalizer.java       |   1 -
 .../repair/consistent/CoordinatorSession.java      |  99 ++---
 .../cassandra/repair/consistent/LocalSessions.java |  30 +-
 .../cassandra/schema/MigrationCoordinator.java     |  10 +-
 .../apache/cassandra/schema/MigrationManager.java  |   5 +-
 .../cassandra/service/ActiveRepairService.java     |  69 +--
 .../org/apache/cassandra/service/CacheService.java |   7 +-
 .../apache/cassandra/service/CassandraDaemon.java  |  34 +-
 .../org/apache/cassandra/service/ClientWarn.java   |  21 +-
 .../service/PendingRangeCalculatorService.java     | 108 ++---
 .../PendingRangeCalculatorServiceDiagnostics.java  |  21 +-
 .../PendingRangeCalculatorServiceEvent.java        |  14 +-
 .../apache/cassandra/service/StorageService.java   |  50 +--
 .../service/snapshot/SnapshotManager.java          |   5 +-
 .../cassandra/streaming/StreamReceiveTask.java     |   5 +-
 .../cassandra/streaming/StreamResultFuture.java    |   4 +-
 .../apache/cassandra/streaming/StreamSession.java  |  12 +-
 .../cassandra/streaming/StreamTransferTask.java    |   5 +-
 .../async/NettyStreamingMessageSender.java         |  16 +-
 .../org/apache/cassandra/tools/GenerateTokens.java |   1 -
 .../apache/cassandra/tracing/TraceStateImpl.java   |  19 +-
 src/java/org/apache/cassandra/tracing/Tracing.java |  21 +-
 .../org/apache/cassandra/transport/Dispatcher.java |  10 +-
 .../java/org/apache/cassandra/utils/Closeable.java |  21 +-
 .../cassandra/utils/DiagnosticSnapshotService.java |   4 +-
 .../org/apache/cassandra/utils/ExecutorUtils.java  |  13 +-
 .../org/apache/cassandra/utils/FBUtilities.java    | 103 ++---
 .../cassandra/utils/JVMStabilityInspector.java     |  16 +
 .../cassandra/utils/RecomputingSupplier.java       |  19 +-
 .../org/apache/cassandra/utils/WithResources.java  |  88 ++++
 .../cassandra/utils/binlog/ExternalArchiver.java   |   4 +-
 .../cassandra/utils/concurrent/AbstractFuture.java | 489 +++++++++++++++++++++
 .../cassandra/utils/concurrent/AsyncFuture.java    | 374 ++--------------
 .../cassandra/utils/concurrent/AsyncPromise.java   |  96 ++--
 .../cassandra/utils/concurrent/Awaitable.java      |   2 -
 .../apache/cassandra/utils/concurrent/Future.java  |  58 ++-
 .../cassandra/utils/concurrent/FutureCombiner.java | 262 +++++++++++
 .../utils/concurrent/ImmediateFuture.java          |  43 +-
 .../cassandra/utils/concurrent/IntrusiveStack.java | 156 +++++++
 .../cassandra/utils/concurrent/ListenerList.java   | 360 +++++++++++++++
 .../apache/cassandra/utils/concurrent/Promise.java |  12 +-
 .../org/apache/cassandra/utils/concurrent/Ref.java |   9 +-
 .../cassandra/utils/concurrent/RunnableFuture.java |  20 +-
 .../cassandra/utils/concurrent/SyncFuture.java     | 160 +++++++
 .../cassandra/utils/concurrent/WaitQueue.java      |   2 +
 .../apache/cassandra/utils/memory/BufferPool.java  |   7 +-
 .../cassandra/utils/memory/MemtableCleaner.java    |   4 +-
 .../utils/memory/MemtableCleanerThread.java        |  49 ++-
 .../cassandra/utils/memory/MemtablePool.java       |   1 -
 .../utils/progress/jmx/JMXBroadcastExecutor.java   |   5 +-
 .../apache/cassandra/net/ConnectionBurnTest.java   |   3 +-
 .../cassandra/distributed/impl/Instance.java       |  15 +-
 .../distributed/test/IPMembershipTest.java         |   1 -
 .../cassandra/distributed/test/JVMDTestTest.java   |   4 +-
 ...bilityInspectorCorruptSSTableExceptionTest.java |   4 +-
 .../distributed/test/SecondaryIndexTest.java       | 108 +++++
 .../test/metrics/HintsServiceMetricsTest.java      |   3 +-
 .../org/apache/cassandra/cql3/ViewLongTest.java    |   2 +-
 .../apache/cassandra/cache/CacheProviderTest.java  |   2 +-
 .../concurrent/AbstractExecutorPlusTest.java       | 200 +++++++++
 .../DebuggableScheduledThreadPoolExecutorTest.java |   4 +-
 .../DebuggableThreadPoolExecutorTest.java          |  93 ++--
 .../ExecutorPlusTest.java}                         |  23 +-
 .../concurrent/InfiniteLoopExecutorTest.java       |  80 ++++
 .../concurrent/LocalAwareExecutorPlusTest.java     |  45 +-
 .../cassandra/concurrent/SEPExecutorTest.java      |  11 +-
 .../apache/cassandra/concurrent/WaitQueueTest.java |   4 +-
 test/unit/org/apache/cassandra/cql3/ViewTest.java  |   2 +-
 .../miscellaneous/CrcCheckChanceTest.java          |   2 +-
 .../unit/org/apache/cassandra/db/CellSpecTest.java |   4 +-
 .../cassandra/db/ClusteringHeapSizeTest.java       |   4 +-
 .../org/apache/cassandra/db/NativeCellTest.java    |   3 +-
 .../apache/cassandra/db/RecoveryManagerTest.java   |  46 +-
 .../db/commitlog/AbstractCommitLogServiceTest.java |  14 +-
 .../commitlog/CommitLogInitWithExceptionTest.java  |   2 +-
 .../CommitLogSegmentBackpressureTest.java          |   4 +-
 .../db/compaction/CompactionExecutorTest.java      |  25 +-
 .../CorruptedSSTablesCompactionsTest.java          |   2 +-
 .../db/repair/PendingAntiCompactionTest.java       |  20 +-
 .../db/streaming/CassandraStreamManagerTest.java   |   2 +-
 .../apache/cassandra/hints/HintsBufferTest.java    |   2 +-
 .../io/sstable/IndexSummaryManagerTest.java        |   2 +-
 .../cassandra/io/sstable/SSTableRewriterTest.java  |   2 +-
 .../org/apache/cassandra/metrics/SamplerTest.java  |   2 +-
 .../cassandra/metrics/ThreadPoolMetricsTest.java   |  32 +-
 .../cassandra/net/AsyncChannelPromiseTest.java     |  57 ++-
 .../org/apache/cassandra/net/HandshakeTest.java    |   2 +-
 .../org/apache/cassandra/repair/RepairJobTest.java |  13 +-
 .../consistent/CoordinatorMessagingTest.java       |  25 +-
 .../repair/consistent/CoordinatorSessionTest.java  |  35 +-
 .../repair/consistent/LocalSessionTest.java        |  43 +-
 .../cassandra/schema/MigrationCoordinatorTest.java |   2 +-
 .../cassandra/service/ActiveRepairServiceTest.java |  10 +-
 .../org/apache/cassandra/service/RemoveTest.java   |   2 +-
 ...ntireSSTableStreamingCorrectFilesCountTest.java |   4 +-
 .../streaming/StreamTransferTaskTest.java          |   2 +-
 .../org/apache/cassandra/tools/BulkLoaderTest.java |   8 +-
 .../org/apache/cassandra/tools/GetVersionTest.java |   2 +-
 .../apache/cassandra/tools/OfflineToolUtils.java   |  32 +-
 .../tools/SSTableExpiredBlockersTest.java          |   2 +-
 .../tools/SSTableExportSchemaLoadingTest.java      |   2 +-
 .../apache/cassandra/tools/SSTableExportTest.java  |   2 +-
 .../cassandra/tools/SSTableLevelResetterTest.java  |   2 +-
 .../cassandra/tools/SSTableMetadataViewerTest.java |   4 +-
 .../cassandra/tools/SSTableOfflineRelevelTest.java |   2 +-
 .../tools/SSTableRepairedAtSetterTest.java         |   8 +-
 .../cassandra/tools/ToolsSchemaLoadingTest.java    |  10 +-
 .../utils/concurrent/AbstractTestAsyncPromise.java | 299 +++++++++++--
 .../utils/concurrent/AsyncPromiseTest.java         | 206 ++++++++-
 .../utils/concurrent/ImmediateFutureTest.java      | 106 +++++
 .../utils/memory/MemtableCleanerThreadTest.java    |  47 +-
 .../utils/memory/NativeAllocatorTest.java          |   3 +-
 .../apache/cassandra/fqltool/QueryReplayer.java    |   5 +-
 .../apache/cassandra/stress/CompactionStress.java  |   6 +-
 219 files changed, 7403 insertions(+), 3591 deletions(-)

diff --git a/build.xml b/build.xml
index e16e936..f5acaff 100644
--- a/build.xml
+++ b/build.xml
@@ -716,17 +716,18 @@
                 artifactId="cassandra-parent"
                 version="${version}"
                 relativePath="${final.name}-parent.pom"/>
-        <dependency groupId="junit" artifactId="junit"/>
-        <dependency groupId="commons-io" artifactId="commons-io"/>
-        <dependency groupId="org.mockito" artifactId="mockito-core"/>
+        <dependency groupId="junit" artifactId="junit" scope="test"/>
+        <dependency groupId="commons-io" artifactId="commons-io" scope="test"/>
+        <dependency groupId="org.mockito" artifactId="mockito-core" scope="test"/>
         <dependency groupId="com.puppycrawl.tools" artifactId="checkstyle" scope="test"/>
-        <dependency groupId="org.quicktheories" artifactId="quicktheories"/>
-        <dependency groupId="org.reflections" artifactId="reflections"/>
-        <dependency groupId="com.google.code.java-allocation-instrumenter" artifactId="java-allocation-instrumenter" version="${allocation-instrumenter.version}"/>
-        <dependency groupId="org.apache.cassandra" artifactId="dtest-api"/>
-        <dependency groupId="org.openjdk.jmh" artifactId="jmh-core"/>
-        <dependency groupId="org.openjdk.jmh" artifactId="jmh-generator-annprocess"/>
-        <dependency groupId="org.apache.ant" artifactId="ant-junit"/>
+        <dependency groupId="org.quicktheories" artifactId="quicktheories" scope="test"/>
+        <dependency groupId="org.reflections" artifactId="reflections" scope="test"/>
+        <dependency groupId="com.google.code.java-allocation-instrumenter" artifactId="java-allocation-instrumenter" version="${allocation-instrumenter.version}" scope="test"/>
+        <dependency groupId="org.apache.cassandra" artifactId="dtest-api" scope="test"/>
+        <dependency groupId="org.openjdk.jmh" artifactId="jmh-core" scope="test"/>
+        <dependency groupId="org.openjdk.jmh" artifactId="jmh-generator-annprocess" scope="test"/>
+        <dependency groupId="net.ju-n.compile-command-annotations" artifactId="compile-command-annotations" scope="test"/>
+        <dependency groupId="org.apache.ant" artifactId="ant-junit" version="1.9.7" scope="test"/>
         <!-- adding this dependency is necessary for assertj. When updating assertj, need to also update the version of
              this that the new assertj's `assertj-parent-pom` depends on. -->
         <dependency groupId="org.junit" artifactId="junit-bom" type="pom"/>
diff --git a/checkstyle.xml b/checkstyle.xml
index 8a90cc7..c745c2e 100644
--- a/checkstyle.xml
+++ b/checkstyle.xml
@@ -1,21 +1,20 @@
 <?xml version="1.0"?>
 <!--
-  ~ Licensed to the Apache Software Foundation (ASF) under one
-  ~ or more contributor license agreements.  See the NOTICE file
-  ~ distributed with this work for additional information
-  ~ regarding copyright ownership.  The ASF licenses this file
-  ~ to you under the Apache License, Version 2.0 (the
-  ~ "License"); you may not use this file except in compliance
-  ~ with the License.  You may obtain a copy of the License at
-  ~
-  ~     http://www.apache.org/licenses/LICENSE-2.0
-  ~
-  ~ Unless required by applicable law or agreed to in writing, software
-  ~ distributed under the License is distributed on an "AS IS" BASIS,
-  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  ~ See the License for the specific language governing permissions and
-  ~ limitations under the License.
-  -->
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
 <!DOCTYPE module PUBLIC
           "-//Checkstyle//DTD Checkstyle Configuration 1.3//EN"
           "https://checkstyle.org/dtds/configuration_1_3.dtd">
@@ -39,10 +38,17 @@
   <module name="TreeWalker">
 
     <module name="RegexpSinglelineJava">
-      <!-- To prevent static imports -->
-      <property name="format" value="System\.(currentTimeMillis|nanoTime)"/>
+      <!-- To prevent static imports and System.nanoTime or System.currentTimeMillis -->
+      <property name="format" value="(newSingleThreadExecutor|newFixedThreadPool|newCachedThreadPool|newSingleThreadScheduledExecutor|newWorkStealingPool|newScheduledThreadPool|defaultThreadFactory)\(|System\.(currentTimeMillis|nanoTime)"/>
       <property name="ignoreComments" value="true"/>
     </module>
+    <module name="IllegalImport">
+      <property name="illegalPkgs" value=""/>
+      <property name="illegalClasses" value="java.util.concurrent.Semaphore,java.util.concurrent.CountDownLatch,java.util.concurrent.Executors,java.util.concurrent.LinkedBlockingQueue,java.util.concurrent.SynchronousQueue,java.util.concurrent.ArrayBlockingQueue,com.google.common.util.concurrent.Futures"/>
+    </module>
+    <module name="IllegalInstantiation">
+      <property name="classes" value="java.lang.Thread,java.util.concurrent.FutureTask,java.util.concurrent.Semaphore,java.util.concurrent.CountDownLatch,java.util.concurrent.ScheduledThreadPoolExecutor,java.util.concurrent.ThreadPoolExecutor,java.util.concurrent.ForkJoinPool))"/>
+    </module>
   </module>
 
 </module>
diff --git a/checkstyle_suppressions.xml b/checkstyle_suppressions.xml
index 3a2ae0c..860efc5 100644
--- a/checkstyle_suppressions.xml
+++ b/checkstyle_suppressions.xml
@@ -1,21 +1,20 @@
 <?xml version="1.0"?>
 <!--
-  ~ Licensed to the Apache Software Foundation (ASF) under one
-  ~ or more contributor license agreements.  See the NOTICE file
-  ~ distributed with this work for additional information
-  ~ regarding copyright ownership.  The ASF licenses this file
-  ~ to you under the Apache License, Version 2.0 (the
-  ~ "License"); you may not use this file except in compliance
-  ~ with the License.  You may obtain a copy of the License at
-  ~
-  ~     http://www.apache.org/licenses/LICENSE-2.0
-  ~
-  ~ Unless required by applicable law or agreed to in writing, software
-  ~ distributed under the License is distributed on an "AS IS" BASIS,
-  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  ~ See the License for the specific language governing permissions and
-  ~ limitations under the License.
-  -->
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
 
 <!DOCTYPE suppressions PUBLIC
         "-//Checkstyle//DTD SuppressionFilter Configuration 1.1//EN"
@@ -23,4 +22,5 @@
 
 <suppressions>
   <suppress checks="RegexpSinglelineJava" files="Clock\.java|Semaphore\.java"/>
+  <suppress checks="IllegalImport|IllegalInstantiation" files="File\.java|[\\/]binlog[\\/]|Semaphore\.java|DataStructureFactory\.java|PathUtils\.java|UDFunction\.java|BlockingQueues\.java"/>
 </suppressions>
diff --git a/src/java/org/apache/cassandra/auth/AuthCache.java b/src/java/org/apache/cassandra/auth/AuthCache.java
index 32e9f0f..e69cd6c 100644
--- a/src/java/org/apache/cassandra/auth/AuthCache.java
+++ b/src/java/org/apache/cassandra/auth/AuthCache.java
@@ -18,7 +18,10 @@
 
 package org.apache.cassandra.auth;
 
+import java.util.HashSet;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.function.BooleanSupplier;
 import java.util.function.Function;
 import java.util.function.IntConsumer;
@@ -29,22 +32,33 @@ import org.slf4j.LoggerFactory;
 
 import com.github.benmanes.caffeine.cache.Caffeine;
 import com.github.benmanes.caffeine.cache.LoadingCache;
-import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
+import org.apache.cassandra.concurrent.ExecutorPlus;
+import org.apache.cassandra.concurrent.Shutdownable;
+import org.apache.cassandra.utils.ExecutorUtils;
 import org.apache.cassandra.utils.MBeanWrapper;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 
-public class AuthCache<K, V> implements AuthCacheMBean
+public class AuthCache<K, V> implements AuthCacheMBean, Shutdownable
 {
     private static final Logger logger = LoggerFactory.getLogger(AuthCache.class);
 
     public static final String MBEAN_NAME_BASE = "org.apache.cassandra.auth:type=";
 
+    // Keep a handle on created instances so their executors can be terminated cleanly
+    private static final Set<Shutdownable> REGISTRY = new HashSet<>(4);
+
+    public static void shutdownAllAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
+    {
+        ExecutorUtils.shutdownNowAndWait(timeout, unit, REGISTRY);
+    }
+
     /**
      * Underlying cache. LoadingCache will call underlying load function on {@link #get} if key is not present
      */
     protected volatile LoadingCache<K, V> cache;
-    private DebuggableThreadPoolExecutor cacheRefreshExecutor;
+    private ExecutorPlus cacheRefreshExecutor;
 
     private String name;
     private IntConsumer setValidityDelegate;
@@ -94,9 +108,10 @@ public class AuthCache<K, V> implements AuthCacheMBean
      */
     protected void init()
     {
-        this.cacheRefreshExecutor = new DebuggableThreadPoolExecutor(name + "Refresh", Thread.NORM_PRIORITY);
+        this.cacheRefreshExecutor = executorFactory().sequential(name + "Refresh");
         cache = initCache(null);
         MBeanWrapper.instance.registerMBean(this, getObjectName());
+        REGISTRY.add(this);
     }
 
     protected void unregisterMBean()
@@ -233,4 +248,28 @@ public class AuthCache<K, V> implements AuthCacheMBean
             policy.setMaximum(getMaxEntries()));
         return cache;
     }
+
+    @Override
+    public boolean isTerminated()
+    {
+        return cacheRefreshExecutor.isTerminated();
+    }
+
+    @Override
+    public void shutdown()
+    {
+        cacheRefreshExecutor.shutdown();
+    }
+
+    @Override
+    public Object shutdownNow()
+    {
+        return cacheRefreshExecutor.shutdownNow();
+    }
+
+    @Override
+    public boolean awaitTermination(long timeout, TimeUnit units) throws InterruptedException
+    {
+        return cacheRefreshExecutor.awaitTermination(timeout, units);
+    }
 }
diff --git a/src/java/org/apache/cassandra/auth/AuthenticatedUser.java b/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
index c2d93ca..678295c 100644
--- a/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
+++ b/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.auth;
 
 import java.util.Set;
-
 import com.google.common.base.Objects;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -164,5 +163,4 @@ public class AuthenticatedUser
     {
         return Objects.hashCode(name);
     }
-
 }
diff --git a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
index 799acbc..432858c 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
@@ -28,9 +28,6 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
@@ -38,10 +35,11 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Collections2;
 import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.RateLimiter;
+import org.apache.cassandra.concurrent.ScheduledExecutorPlus;
+import org.apache.cassandra.utils.concurrent.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -80,6 +78,7 @@ import org.apache.cassandra.utils.UUIDGen;
 
 import static com.google.common.collect.Iterables.transform;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternalWithPaging;
 import static org.apache.cassandra.net.Verb.MUTATION_REQ;
@@ -100,15 +99,13 @@ public class BatchlogManager implements BatchlogManagerMBean
     private volatile UUID lastReplayedUuid = UUIDGen.minTimeUUID(0);
 
     // Single-thread executor service for scheduling and serializing log replay.
-    private final ScheduledExecutorService batchlogTasks;
+    private final ScheduledExecutorPlus batchlogTasks;
 
     private final RateLimiter rateLimiter = RateLimiter.create(Double.MAX_VALUE);
 
     public BatchlogManager()
     {
-        ScheduledThreadPoolExecutor executor = new DebuggableScheduledThreadPoolExecutor("BatchlogTasks");
-        executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
-        batchlogTasks = executor;
+        batchlogTasks = executorFactory().scheduled(false, "BatchlogTasks");
     }
 
     public void start()
diff --git a/src/java/org/apache/cassandra/cache/AutoSavingCache.java b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
index c7a015c..a9a09fe 100644
--- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java
+++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
@@ -17,10 +17,13 @@
  */
 package org.apache.cassandra.cache;
 
-import java.io.*;
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.util.*;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 
@@ -28,10 +31,7 @@ import org.cliffc.high_scale_lib.NonBlockingHashSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-
+import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
@@ -50,6 +50,9 @@ import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.UUIDGen;
+import org.apache.cassandra.utils.concurrent.Future;
+
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
 
@@ -155,12 +158,12 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
         }
     }
 
-    public ListenableFuture<Integer> loadSavedAsync()
+    public Future<Integer> loadSavedAsync()
     {
-        final ListeningExecutorService es = MoreExecutors.listeningDecorator(Executors.newSingleThreadExecutor());
+        final ExecutorPlus es = executorFactory().sequential("loadSavedCache");
         final long start = nanoTime();
 
-        ListenableFuture<Integer> cacheLoad = es.submit(this::loadSaved);
+        Future<Integer> cacheLoad = es.submit(this::loadSaved);
         cacheLoad.addListener(() -> {
             if (size() > 0)
                 logger.info("Completed loading ({} ms; {} keys) {} cache",
@@ -168,7 +171,7 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
                         CacheService.instance.keyCache.size(),
                         cacheType);
             es.shutdown();
-        }, MoreExecutors.directExecutor());
+        });
 
         return cacheLoad;
     }
diff --git a/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
deleted file mode 100644
index 6053b83..0000000
--- a/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.concurrent;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.cassandra.utils.concurrent.Condition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-import org.apache.cassandra.utils.JVMStabilityInspector;
-
-import static org.apache.cassandra.tracing.Tracing.isTracing;
-
-public abstract class AbstractLocalAwareExecutorService implements LocalAwareExecutorService
-{
-    private static final Logger logger = LoggerFactory.getLogger(AbstractLocalAwareExecutorService.class);
-
-    protected abstract void addTask(FutureTask<?> futureTask);
-    protected abstract void onCompletion();
-
-    /** Task Submission / Creation / Objects **/
-
-    public <T> FutureTask<T> submit(Callable<T> task)
-    {
-        return submit(newTaskFor(task));
-    }
-
-    public FutureTask<?> submit(Runnable task)
-    {
-        return submit(newTaskFor(task, null));
-    }
-
-    public <T> FutureTask<T> submit(Runnable task, T result)
-    {
-        return submit(newTaskFor(task, result));
-    }
-
-    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result)
-    {
-        return newTaskFor(runnable, result, ExecutorLocals.create());
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result, ExecutorLocals locals)
-    {
-        if (locals != null)
-        {
-            if (runnable instanceof LocalSessionFutureTask)
-                return (LocalSessionFutureTask<T>) runnable;
-            return new LocalSessionFutureTask<T>(runnable, result, locals);
-        }
-        if (runnable instanceof FutureTask)
-            return (FutureTask<T>) runnable;
-        return new FutureTask<>(runnable, result);
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Callable<T> callable)
-    {
-        if (isTracing())
-        {
-            if (callable instanceof LocalSessionFutureTask)
-                return (LocalSessionFutureTask<T>) callable;
-            return new LocalSessionFutureTask<T>(callable, ExecutorLocals.create());
-        }
-        if (callable instanceof FutureTask)
-            return (FutureTask<T>) callable;
-        return new FutureTask<>(callable);
-    }
-
-    private class LocalSessionFutureTask<T> extends FutureTask<T>
-    {
-        private final ExecutorLocals locals;
-
-        public LocalSessionFutureTask(Callable<T> callable, ExecutorLocals locals)
-        {
-            super(callable);
-            this.locals = locals;
-        }
-
-        public LocalSessionFutureTask(Runnable runnable, T result, ExecutorLocals locals)
-        {
-            super(runnable, result);
-            this.locals = locals;
-        }
-
-        public void run()
-        {
-            ExecutorLocals old = ExecutorLocals.create();
-            ExecutorLocals.set(locals);
-            try
-            {
-                super.run();
-            }
-            finally
-            {
-                ExecutorLocals.set(old);
-            }
-        }
-    }
-
-    class FutureTask<T> extends Condition.Async implements Future<T>, Runnable
-    {
-        private boolean failure;
-        private Object result = this;
-        private final Callable<T> callable;
-
-        public FutureTask(Callable<T> callable)
-        {
-            this.callable = callable;
-        }
-        public FutureTask(Runnable runnable, T result)
-        {
-            this(Executors.callable(runnable, result));
-        }
-
-        public void run()
-        {
-            try
-            {
-                result = callable.call();
-            }
-            catch (Throwable t)
-            {
-                logger.error("Uncaught exception on thread {}", Thread.currentThread(), t);
-                result = t;
-                failure = true;
-                JVMStabilityInspector.inspectThrowable(t);
-            }
-            finally
-            {
-                signalAll();
-                onCompletion();
-            }
-        }
-
-        public boolean cancel(boolean mayInterruptIfRunning)
-        {
-            return false;
-        }
-
-        public boolean isCancelled()
-        {
-            return false;
-        }
-
-        public boolean isDone()
-        {
-            return isSignalled();
-        }
-
-        public T get() throws InterruptedException, ExecutionException
-        {
-            await();
-            Object result = this.result;
-            if (failure)
-                throw new ExecutionException((Throwable) result);
-            return (T) result;
-        }
-
-        public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-        {
-            if (!await(timeout, unit))
-                throw new TimeoutException();
-            Object result = this.result;
-            if (failure)
-                throw new ExecutionException((Throwable) result);
-            return (T) result;
-        }
-    }
-
-    private <T> FutureTask<T> submit(FutureTask<T> task)
-    {
-        addTask(task);
-        return task;
-    }
-
-    public void execute(Runnable command)
-    {
-        addTask(newTaskFor(command, ExecutorLocals.create()));
-    }
-
-    public void execute(Runnable command, ExecutorLocals locals)
-    {
-        addTask(newTaskFor(command, null, locals));
-    }
-}
diff --git a/src/java/org/apache/cassandra/concurrent/DebuggableScheduledThreadPoolExecutor.java b/src/java/org/apache/cassandra/concurrent/DebuggableScheduledThreadPoolExecutor.java
deleted file mode 100644
index 3b9d2ff..0000000
--- a/src/java/org/apache/cassandra/concurrent/DebuggableScheduledThreadPoolExecutor.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.concurrent;
-
-import java.util.concurrent.*;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-
-/**
- * Like DebuggableThreadPoolExecutor, DebuggableScheduledThreadPoolExecutor always
- * logs exceptions from the tasks it is given, even if Future.get is never called elsewhere.
- *
- * DebuggableScheduledThreadPoolExecutor also catches exceptions during Task execution
- * so that they don't supress subsequent invocations of the task.
- *
- * Finally, there is a special rejected execution handler for tasks rejected during the shutdown hook.
- *
- * For fire and forget tasks (like ref tidy) we can safely ignore the exceptions.
- * For any callers that care to know their task was rejected we cancel passed task.
- */
-public class DebuggableScheduledThreadPoolExecutor extends ScheduledThreadPoolExecutor
-{
-    private static final Logger logger = LoggerFactory.getLogger(DebuggableScheduledThreadPoolExecutor.class);
-
-    public static final RejectedExecutionHandler rejectedExecutionHandler = new RejectedExecutionHandler()
-    {
-        public void rejectedExecution(Runnable task, ThreadPoolExecutor executor)
-        {
-            if (executor.isShutdown())
-            {
-                if (!StorageService.instance.isShutdown())
-                    throw new RejectedExecutionException("ScheduledThreadPoolExecutor has shut down.");
-
-                //Give some notification to the caller the task isn't going to run
-                if (task instanceof Future)
-                    ((Future) task).cancel(false);
-
-                logger.debug("ScheduledThreadPoolExecutor has shut down as part of C* shutdown");
-            }
-            else
-            {
-                throw new AssertionError("Unknown rejection of ScheduledThreadPoolExecutor task");
-            }
-        }
-    };
-
-    public DebuggableScheduledThreadPoolExecutor(int corePoolSize, String threadPoolName, int priority)
-    {
-        super(corePoolSize, new NamedThreadFactory(threadPoolName, priority));
-        setRejectedExecutionHandler(rejectedExecutionHandler);
-    }
-
-    public DebuggableScheduledThreadPoolExecutor(int corePoolSize, ThreadFactory threadFactory)
-    {
-        super(corePoolSize, threadFactory);
-        setRejectedExecutionHandler(rejectedExecutionHandler);
-    }
-
-    public DebuggableScheduledThreadPoolExecutor(String threadPoolName)
-    {
-        this(1, threadPoolName, Thread.NORM_PRIORITY);
-        setRejectedExecutionHandler(rejectedExecutionHandler);
-    }
-
-    // We need this as well as the wrapper for the benefit of non-repeating tasks
-    @Override
-    public void afterExecute(Runnable r, Throwable t)
-    {
-        super.afterExecute(r,t);
-        DebuggableThreadPoolExecutor.logExceptionsAfterExecute(r, t);
-    }
-
-    // override scheduling to supress exceptions that would cancel future executions
-    @Override
-    public ScheduledFuture<?> scheduleAtFixedRate(Runnable command, long initialDelay, long period, TimeUnit unit)
-    {
-        return super.scheduleAtFixedRate(new UncomplainingRunnable(command), initialDelay, period, unit);
-    }
-
-    @Override
-    public ScheduledFuture<?> scheduleWithFixedDelay(Runnable command, long initialDelay, long delay, TimeUnit unit)
-    {
-        return super.scheduleWithFixedDelay(new UncomplainingRunnable(command), initialDelay, delay, unit);
-    }
-
-    private static class UncomplainingRunnable implements Runnable
-    {
-        private final Runnable runnable;
-
-        public UncomplainingRunnable(Runnable runnable)
-        {
-            this.runnable = runnable;
-        }
-
-        public void run()
-        {
-            try
-            {
-                runnable.run();
-            }
-            catch (Throwable t)
-            {
-                JVMStabilityInspector.inspectThrowable(t);
-                DebuggableThreadPoolExecutor.handleOrLog(t);
-            }
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
deleted file mode 100644
index a2de775..0000000
--- a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
+++ /dev/null
@@ -1,426 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.concurrent;
-
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.RejectedExecutionHandler;
-import java.util.concurrent.RunnableFuture;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.cassandra.tracing.Tracing.isTracing;
-
-/**
- * This class encorporates some Executor best practices for Cassandra.  Most of the executors in the system
- * should use or extend this.  There are two main improvements over a vanilla TPE:
- *
- * - If a task throws an exception, the default uncaught exception handler will be invoked; if there is
- *   no such handler, the exception will be logged.
- * - MaximumPoolSize is not supported.  Here is what that means (quoting TPE javadoc):
- *
- *     If fewer than corePoolSize threads are running, the Executor always prefers adding a new thread rather than queuing.
- *     If corePoolSize or more threads are running, the Executor always prefers queuing a request rather than adding a new thread.
- *     If a request cannot be queued, a new thread is created unless this would exceed maximumPoolSize, in which case, the task will be rejected.
- *
- *   We don't want this last stage of creating new threads if the queue is full; it makes it needlessly difficult to
- *   reason about the system's behavior.  In other words, if DebuggableTPE has allocated our maximum number of (core)
- *   threads and the queue is full, we want the enqueuer to block.  But to allow the number of threads to drop if a
- *   stage is less busy, core thread timeout is enabled.
- */
-public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements LocalAwareExecutorService
-{
-    protected static final Logger logger = LoggerFactory.getLogger(DebuggableThreadPoolExecutor.class);
-    public static final RejectedExecutionHandler blockingExecutionHandler = new RejectedExecutionHandler()
-    {
-        public void rejectedExecution(Runnable task, ThreadPoolExecutor executor)
-        {
-            ((DebuggableThreadPoolExecutor) executor).onInitialRejection(task);
-            BlockingQueue<Runnable> queue = executor.getQueue();
-            while (true)
-            {
-                if (executor.isShutdown())
-                {
-                    ((DebuggableThreadPoolExecutor) executor).onFinalRejection(task);
-                    throw new RejectedExecutionException("ThreadPoolExecutor has shut down");
-                }
-                try
-                {
-                    if (queue.offer(task, 1000, TimeUnit.MILLISECONDS))
-                    {
-                        ((DebuggableThreadPoolExecutor) executor).onFinalAccept(task);
-                        break;
-                    }
-                }
-                catch (InterruptedException e)
-                {
-                    throw new AssertionError(e);
-                }
-            }
-        }
-    };
-
-    public DebuggableThreadPoolExecutor(String threadPoolName, int priority)
-    {
-        this(1, Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), new NamedThreadFactory(threadPoolName, priority));
-    }
-
-    public DebuggableThreadPoolExecutor(int corePoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> queue, ThreadFactory factory)
-    {
-        this(corePoolSize, corePoolSize, keepAliveTime, unit, queue, factory);
-    }
-
-    public DebuggableThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory)
-    {
-        super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory);
-        allowCoreThreadTimeOut(true);
-
-        // block task submissions until queue has room.
-        // this is fighting TPE's design a bit because TPE rejects if queue.offer reports a full queue.
-        // we'll just override this with a handler that retries until it gets in.  ugly, but effective.
-        // (there is an extensive analysis of the options here at
-        //  http://today.java.net/pub/a/today/2008/10/23/creating-a-notifying-blocking-thread-pool-executor.html)
-        this.setRejectedExecutionHandler(blockingExecutionHandler);
-    }
-
-    /**
-     * Creates a thread pool that creates new threads as needed, but
-     * will reuse previously constructed threads when they are
-     * available.
-     * @param threadPoolName the name of the threads created by this executor
-     * @return The new DebuggableThreadPoolExecutor
-     */
-    public static DebuggableThreadPoolExecutor createCachedThreadpoolWithMaxSize(String threadPoolName)
-    {
-        return new DebuggableThreadPoolExecutor(0, Integer.MAX_VALUE,
-                                                60L, TimeUnit.SECONDS,
-                                                new SynchronousQueue<Runnable>(),
-                                                new NamedThreadFactory(threadPoolName));
-    }
-
-    /**
-     * Returns a ThreadPoolExecutor with a fixed number of threads.
-     * When all threads are actively executing tasks, new tasks are queued.
-     * If (most) threads are expected to be idle most of the time, prefer createWithMaxSize() instead.
-     * @param threadPoolName the name of the threads created by this executor
-     * @param size the fixed number of threads for this executor
-     * @return the new DebuggableThreadPoolExecutor
-     */
-    public static DebuggableThreadPoolExecutor createWithFixedPoolSize(String threadPoolName, int size)
-    {
-        return createWithMaximumPoolSize(threadPoolName, size, Integer.MAX_VALUE, TimeUnit.SECONDS);
-    }
-
-    /**
-     * Returns a ThreadPoolExecutor with a fixed maximum number of threads, but whose
-     * threads are terminated when idle for too long.
-     * When all threads are actively executing tasks, new tasks are queued.
-     * @param threadPoolName the name of the threads created by this executor
-     * @param size the maximum number of threads for this executor
-     * @param keepAliveTime the time an idle thread is kept alive before being terminated
-     * @param unit tht time unit for {@code keepAliveTime}
-     * @return the new DebuggableThreadPoolExecutor
-     */
-    public static DebuggableThreadPoolExecutor createWithMaximumPoolSize(String threadPoolName, int size, int keepAliveTime, TimeUnit unit)
-    {
-        return new DebuggableThreadPoolExecutor(size, Integer.MAX_VALUE, keepAliveTime, unit, new LinkedBlockingQueue<Runnable>(), new NamedThreadFactory(threadPoolName));
-    }
-
-    protected void onInitialRejection(Runnable task) {}
-    protected void onFinalAccept(Runnable task) {}
-    protected void onFinalRejection(Runnable task) {}
-
-    public void execute(Runnable command, ExecutorLocals locals)
-    {
-        super.execute(locals == null || command instanceof LocalSessionWrapper
-                      ? command
-                      : LocalSessionWrapper.create(command, null, locals));
-    }
-
-    public void maybeExecuteImmediately(Runnable command)
-    {
-        execute(command);
-    }
-
-    // execute does not call newTaskFor
-    @Override
-    public void execute(Runnable command)
-    {
-        super.execute(isTracing() && !(command instanceof LocalSessionWrapper)
-                      ? LocalSessionWrapper.create(command)
-                      : command);
-    }
-
-    @Override
-    protected <T> RunnableFuture<T> newTaskFor(Runnable runnable, T result)
-    {
-        if (isTracing() && !(runnable instanceof LocalSessionWrapper))
-            return LocalSessionWrapper.create(runnable, result);
-        if (runnable instanceof RunnableFuture)
-            return new ForwardingRunnableFuture<>((RunnableFuture) runnable, result);
-        return super.newTaskFor(runnable, result);
-    }
-
-    @Override
-    protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable)
-    {
-        if (isTracing() && !(callable instanceof LocalSessionWrapper))
-            return LocalSessionWrapper.create(callable);
-        return super.newTaskFor(callable);
-    }
-
-    @Override
-    protected void afterExecute(Runnable r, Throwable t)
-    {
-        super.afterExecute(r, t);
-
-        maybeResetTraceSessionWrapper(r);
-        logExceptionsAfterExecute(r, t);
-    }
-
-    protected static void maybeResetTraceSessionWrapper(Runnable r)
-    {
-        if (r instanceof LocalSessionWrapper)
-        {
-            LocalSessionWrapper tsw = (LocalSessionWrapper) r;
-            // we have to reset trace state as its presence is what denotes the current thread is tracing
-            // and if left this thread might start tracing unrelated tasks
-            tsw.reset();
-        }
-    }
-
-    @Override
-    protected void beforeExecute(Thread t, Runnable r)
-    {
-        if (r instanceof LocalSessionWrapper)
-            ((LocalSessionWrapper) r).setupContext();
-
-        super.beforeExecute(t, r);
-    }
-
-    @Override
-    public int getActiveTaskCount()
-    {
-        return getActiveCount();
-    }
-
-    @Override
-    public int getPendingTaskCount()
-    {
-        return getQueue().size();
-    }
-
-    /**
-     * Send @param t and any exception wrapped by @param r to the default uncaught exception handler,
-     * or log them if none such is set up
-     */
-    public static void logExceptionsAfterExecute(Runnable r, Throwable t)
-    {
-        Throwable hiddenThrowable = extractThrowable(r);
-        if (hiddenThrowable != null)
-            handleOrLog(hiddenThrowable);
-
-        // ThreadPoolExecutor will re-throw exceptions thrown by its Task (which will be seen by
-        // the default uncaught exception handler) so we only need to do anything if that handler
-        // isn't set up yet.
-        if (t != null && Thread.getDefaultUncaughtExceptionHandler() == null)
-            handleOrLog(t);
-    }
-
-    /**
-     * Send @param t to the default uncaught exception handler, or log it if none such is set up
-     */
-    public static void handleOrLog(Throwable t)
-    {
-        if (Thread.getDefaultUncaughtExceptionHandler() == null)
-            logger.error("Error in ThreadPoolExecutor", t);
-        else
-            Thread.getDefaultUncaughtExceptionHandler().uncaughtException(Thread.currentThread(), t);
-    }
-
-    /**
-     * @return any exception wrapped by @param runnable, i.e., if it is a FutureTask
-     */
-    public static Throwable extractThrowable(Runnable runnable)
-    {
-        // Check for exceptions wrapped by FutureTask or tasks which wrap FutureTask (HasDelegateFuture interface)
-        Throwable throwable = null;
-        if (runnable instanceof Future<?>)
-        {
-            throwable = extractThrowable(((Future<?>) runnable));
-        }
-        if (throwable == null && runnable instanceof HasDelegateFuture)
-        {
-            throwable =  extractThrowable(((HasDelegateFuture) runnable).getDelegate());
-        }
-
-        return throwable;
-    }
-
-    private static Throwable extractThrowable(Future<?> future)
-    {
-        // Check for exceptions wrapped by Future.  We do this by calling get(), which will
-        // cause it to throw any saved exception.
-        //
-        // Complicating things, calling get() on a ScheduledFutureTask will block until the task
-        // is cancelled.  Hence, the extra isDone check beforehand.
-        if (future.isDone())
-        {
-            try
-            {
-                future.get();
-            }
-            catch (InterruptedException e)
-            {
-                throw new AssertionError(e);
-            }
-            catch (CancellationException e)
-            {
-                logger.trace("Task cancelled", e);
-            }
-            catch (ExecutionException e)
-            {
-                return e.getCause();
-            }
-        }
-        return null;
-    }
-
-    /**
-     * If a task wraps a {@link Future} then it should implement this interface to expose the underlining future for
-     * {@link #extractThrowable(Runnable)} to handle.
-     */
-    private interface HasDelegateFuture
-    {
-        Future<?> getDelegate();
-    }
-
-    /**
-     * Used to wrap a Runnable or Callable passed to submit or execute so we can clone the ExecutorLocals and move
-     * them into the worker thread.
-     *
-     * The {@link DebuggableThreadPoolExecutor#afterExecute(java.lang.Runnable, java.lang.Throwable)}
-     * method is called after the runnable completes, which will then call {@link #extractThrowable(Runnable)} to
-     * attempt to get the "hidden" throwable from a task which implements {@link Future}.  The problem is that {@link LocalSessionWrapper}
-     * expects that the {@link Callable} provided to it will throw; which is not true for {@link RunnableFuture} tasks;
-     * the expected semantic in this case is to have the LocalSessionWrapper future be successful and a new implementation
-     * {@link FutureLocalSessionWrapper} is created to expose the underline {@link Future} for {@link #extractThrowable(Runnable)}.
-     *
-     * If a task is a {@link Runnable} the create family of methods should be called rather than {@link Executors#callable(Runnable)}
-     * since they will handle the case where the task is also a future, and will make sure the {@link #extractThrowable(Runnable)}
-     * is able to detect the task's underline exception.
-     *
-     * @param <T>
-     */
-    private static class LocalSessionWrapper<T> extends FutureTask<T>
-    {
-        private final ExecutorLocals locals;
-
-        private LocalSessionWrapper(Callable<T> callable, ExecutorLocals locals)
-        {
-            super(callable);
-            this.locals = locals;
-        }
-
-        static LocalSessionWrapper<Object> create(Runnable command)
-        {
-            return create(command, null, ExecutorLocals.create());
-        }
-
-        static <T> LocalSessionWrapper<T> create(Runnable command, T result)
-        {
-            return create(command, result, ExecutorLocals.create());
-        }
-
-        static <T> LocalSessionWrapper<T> create(Runnable command, T result, ExecutorLocals locals)
-        {
-            if (command instanceof RunnableFuture)
-                return new FutureLocalSessionWrapper<>((RunnableFuture) command, result, locals);
-            return new LocalSessionWrapper<>(Executors.callable(command, result), locals);
-        }
-
-        static <T> LocalSessionWrapper<T> create(Callable<T> command)
-        {
-            return new LocalSessionWrapper<>(command, ExecutorLocals.create());
-        }
-
-        private void setupContext()
-        {
-            ExecutorLocals.set(locals);
-        }
-
-        private void reset()
-        {
-            ExecutorLocals.set(null);
-        }
-    }
-
-    private static class FutureLocalSessionWrapper<T> extends LocalSessionWrapper<T> implements HasDelegateFuture
-    {
-        private final RunnableFuture<T> delegate;
-
-        private FutureLocalSessionWrapper(RunnableFuture command, T result, ExecutorLocals locals)
-        {
-            super(() -> {
-                command.run();
-                return result;
-            }, locals);
-            this.delegate = command;
-        }
-
-        public Future<T> getDelegate()
-        {
-            return delegate;
-        }
-    }
-
-    /**
-     * Similar to {@link FutureLocalSessionWrapper}, this class wraps a {@link Future} and will be success
-     * if the underline future is marked as failed; the main difference is that this class does not setup
-     * {@link ExecutorLocals}.
-     *
-     * @param <T>
-     */
-    private static class ForwardingRunnableFuture<T> extends FutureTask<T> implements HasDelegateFuture
-    {
-        private final RunnableFuture<T> delegate;
-
-        public ForwardingRunnableFuture(RunnableFuture<T> delegate, T result)
-        {
-            super(delegate, result);
-            this.delegate = delegate;
-        }
-
-        public Future<T> getDelegate()
-        {
-            return delegate;
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutionFailure.java b/src/java/org/apache/cassandra/concurrent/ExecutionFailure.java
new file mode 100644
index 0000000..61bdae0
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutionFailure.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.concurrent;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.utils.Closeable;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.WithResources;
+
+/**
+ * Standardised handling of failures during execution - mostly this involves invoking a thread's
+ * {@link java.lang.Thread.UncaughtExceptionHandler} or
+ * {@link JVMStabilityInspector#uncaughtException(Thread, Throwable)},
+ * with special handling for {@link CompactionInterruptedException}.
+ * This class also provides wrappers for {@link WithResources} with {@link Runnable} and {@link Callable}.
+ */
+public class ExecutionFailure
+{
+    private static final Logger logger = LoggerFactory.getLogger(ExecutionFailure.class);
+
+    /**
+     * Invoke the relevant {@link java.lang.Thread.UncaughtExceptionHandler},
+     * ignoring (except for logging) any {@link CompactionInterruptedException}
+     */
+    public static void handle(Throwable t)
+    {
+        try
+        {
+            if (t instanceof CompactionInterruptedException)
+            {
+                // TODO: should we check to see there aren't nested CompactionInterruptedException?
+                logger.info(t.getMessage());
+                if (t.getSuppressed() != null && t.getSuppressed().length > 0)
+                    logger.warn("Interruption of compaction encountered exceptions:", t);
+                else
+                    logger.trace("Full interruption stack trace:", t);
+            }
+            else
+            {
+                Thread thread = Thread.currentThread();
+                Thread.UncaughtExceptionHandler handler = thread.getUncaughtExceptionHandler();
+                if (handler == null || handler.equals(thread.getThreadGroup()))
+                    handler = JVMStabilityInspector::uncaughtException;
+                handler.uncaughtException(thread, t);
+            }
+        }
+        catch (Throwable shouldNeverHappen)
+        {
+            logger.error("Unexpected error while handling unexpected error", shouldNeverHappen);
+        }
+    }
+
+    /**
+     * See {@link #propagating(WithResources, Runnable)}
+     */
+    static Runnable propagating(Runnable wrap)
+    {
+        return wrap instanceof FutureTask<?> ? wrap : propagating(WithResources.none(), wrap);
+    }
+
+    /**
+     * In the case of plain executions, we want to handle exceptions without the full {@link FutureTask} machinery
+     * while still propagating the exception to the encapsulating Future
+     */
+    static Runnable propagating(WithResources withResources, Runnable wrap)
+    {
+        return enforceOptions(withResources, wrap, true);
+    }
+
+    /**
+     * See {@link #suppressing(WithResources, Runnable)}
+     */
+    static Runnable suppressing(Runnable wrap)
+    {
+        return wrap instanceof FutureTask<?> ? wrap : suppressing(WithResources.none(), wrap);
+    }
+
+    /**
+     * In the case of scheduled periodic tasks, we don't want exceptions propagating to cancel the recurring execution.
+     */
+    static Runnable suppressing(WithResources withResources, Runnable wrap)
+    {
+        return enforceOptions(withResources, wrap, false);
+    }
+
+    /**
+     * Encapsulate the execution, propagating or suppressing any exceptions as requested.
+     *
+     * note that if {@code wrap} is a {@link java.util.concurrent.Future} its exceptions may not be captured,
+     * however the codebase should be using our internal {@link Future} variants which handle exceptions in the
+     * desired way.
+     */
+    private static Runnable enforceOptions(WithResources withResources, Runnable wrap, boolean propagate)
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                try (Closeable close = withResources.get())
+                {
+                    wrap.run();
+                }
+                catch (Throwable t)
+                {
+                    handle(t);
+                    if (propagate)
+                        throw t;
+                }
+            }
+
+            @Override
+            public String toString()
+            {
+                return wrap.toString();
+            }
+        };
+    }
+
+    /**
+     * See {@link #enforceOptions(WithResources, Callable)}
+     */
+    static <V> Callable<V> propagating(Callable<V> wrap)
+    {
+        return enforceOptions(WithResources.none(), wrap);
+    }
+
+    /**
+     * In the case of non-recurring scheduled tasks, we want to handle exceptions without the full {@link FutureTask}
+     * machinery, while still propagating the exception to the encapsulating Future
+     */
+    static <V> Callable<V> enforceOptions(WithResources withResources, Callable<V> wrap)
+    {
+        return new Callable<V>()
+        {
+            @Override
+            public V call() throws Exception
+            {
+                try (Closeable close = withResources.get())
+                {
+                    return wrap.call();
+                }
+                catch (Throwable t)
+                {
+                    handle(t);
+                    throw t;
+                }
+            }
+
+            @Override
+            public String toString()
+            {
+                return wrap.toString();
+            }
+        };
+    }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorBuilder.java b/src/java/org/apache/cassandra/concurrent/ExecutorBuilder.java
new file mode 100644
index 0000000..89ca28a
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorBuilder.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.concurrent;
+
+import java.lang.Thread.UncaughtExceptionHandler;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.RejectedExecutionHandler;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+/**
+ * Configure an executor before creating it.
+ * See {@link ThreadPoolExecutorBuilder}
+ */
+public interface ExecutorBuilder<E extends ExecutorService>
+{
+    /**
+     * Threads for the executor built by this factory will timeout (terminate) after the specified period.
+     */
+    ExecutorBuilder<E> withKeepAlive(long keepAlive, TimeUnit keepAliveUnits);
+
+    /**
+     * Core threads for the executor built by this factory will never timeout (default for single threaded builders).
+     * Note that there is ordinarily no difference between core and non-core threads; only when the queue limit is zero
+     * do we create non-core threads.
+     */
+    ExecutorBuilder<E> withKeepAlive();
+
+    /**
+     * Specify the priority of threads that service the executor built by this factory (default to {@link Thread#NORM_PRIORITY})
+     */
+    ExecutorBuilder<E> withThreadPriority(int threadPriority);
+
+    /**
+     * Threads for the executor built by this factory will all be (transitively) members of {@code threadGroup},
+     * but may directly reside in a child thread group.
+     */
+    ExecutorBuilder<E> withThreadGroup(ThreadGroup threadGroup);
+
+    /**
+     * Use the system default thread group for the threads we create.
+     * This is used only for testing, so that we do not hold onto a transitive global reference to all threads.
+     */
+    @VisibleForTesting
+    ExecutorBuilder<E> withDefaultThreadGroup();
+
+    /**
+     * The executor built by this factory will limit the number of queued tasks; default is unlimited.
+     * Once the queue limit is reached and all threads are executing tasks will be rejected
+     * (see {@link #withRejectedExecutionHandler(RejectedExecutionHandler)})
+     */
+    ExecutorBuilder<E> withQueueLimit(int queueLimit);
+
+    /**
+     * Set the {@link RejectedExecutionHandler} for the executor built by this factory.
+     * By default this is executor-specific, either:
+     * <li> {@link ThreadPoolExecutorBase#blockingExecutionHandler}
+     * <li> {@link ScheduledThreadPoolExecutorPlus#rejectedExecutionHandler}
+     * <li> and maybe wrapped by {@link ThreadPoolExecutorJMXAdapter#rejectedExecutionHandler}
+     */
+    ExecutorBuilder<E> withRejectedExecutionHandler(RejectedExecutionHandler rejectedExecutionHandler);
+
+    /**
+     * Set the {@link UncaughtExceptionHandler} for threads that service executors built by this factory.
+     * By default {@link JVMStabilityInspector#uncaughtException(Thread, Throwable)}
+     */
+    ExecutorBuilder<E> withUncaughtExceptionHandler(UncaughtExceptionHandler uncaughtExceptionHandler);
+
+    /**
+     * Build the configured executor
+     */
+    E build();
+}
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorBuilderFactory.java b/src/java/org/apache/cassandra/concurrent/ExecutorBuilderFactory.java
new file mode 100644
index 0000000..f96def8
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorBuilderFactory.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.concurrent;
+
+/**
+ * Entry point for configuring and creating new executors.
+ *
+ * Supports quick and easy construction of default-configured executors via
+ * <li>{@link #sequential(String)}
+ * <li>{@link #pooled(String, int)}
+ *
+ * Supports custom configuration of executors via
+ * <li>{@link #configureSequential(String)}
+ * <li>{@link #configurePooled(String, int)}
+ */
+public interface ExecutorBuilderFactory<E extends ExecutorPlus, S extends SequentialExecutorPlus>
+{
+    /**
+     * Configure a sequential (single threaded) executor
+     */
+    ExecutorBuilder<? extends S> configureSequential(String name);
+
+    /**
+     * Configure a pooled executor with the requested number of threads
+     */
+    ExecutorBuilder<? extends E> configurePooled(String name, int threads);
+
+    /**
+     * Return a default configuration of sequential executor
+     */
+    default S sequential(String name) { return configureSequential(name).build(); }
+
+    /**
+     * Return a default configuration of pooled executor
+     */
+    default E pooled(String name, int threads) { return configurePooled(name, threads).build(); }
+
+    /**
+     * Entry point for configuring and creating new executors.
+     *
+     * Supports quick and easy construction of default-configured executors via
+     * <li>{@link #sequential(String)}
+     * <li>{@link #pooled(String, int)}
+     *
+     * Supports custom configuration of executors via
+     * <li>{@link #configureSequential(String)}
+     * <li>{@link #configurePooled(String, int)}
+     *
+     * Supports any of the above with added JMX registration via sub-factories
+     * <li>{@link #withJmx(String)}
+     * <li>{@link #withJmxInternal()}
+     */
+    interface Jmxable<E extends ExecutorPlus, S extends SequentialExecutorPlus> extends ExecutorBuilderFactory<E, S>
+    {
+        /**
+         * @return a factory that configures executors that register against JMX using the provided jmx path
+         */
+        ExecutorBuilderFactory<E, S> withJmx(String jmxPath);
+
+        /**
+         * @return a factory that configures executors that register against JMX using the "internal" jmx path
+         */
+        default ExecutorBuilderFactory<E, S> withJmxInternal() { return withJmx("internal"); }
+    }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorFactory.java b/src/java/org/apache/cassandra/concurrent/ExecutorFactory.java
new file mode 100644
index 0000000..9c7a2cf
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorFactory.java
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.concurrent;
+
+import java.util.function.Consumer;
+
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static java.lang.Thread.*;
+import static org.apache.cassandra.concurrent.NamedThreadFactory.createThread;
+import static org.apache.cassandra.concurrent.NamedThreadFactory.setupThread;
+import static org.apache.cassandra.concurrent.ThreadPoolExecutorBuilder.pooledJmx;
+import static org.apache.cassandra.concurrent.ThreadPoolExecutorBuilder.sequentialJmx;
+
+/**
+ * Entry point for configuring and creating new executors.
+ *
+ * Supports quick and easy construction of default-configured executors via
+ * <li>{@link #sequential(String)}
+ * <li>{@link #pooled(String, int)}
+ * <li>{@link #scheduled(String)}
+ * <li>{@link #scheduled(boolean, String)}
+ * <li>{@link #scheduled(boolean, String, int)}
+ *
+ * Supports custom configuration of executors via
+ * <li>{@link #configureSequential(String)}
+ * <li>{@link #configurePooled(String, int)}
+ *
+ * Supports any of the above with added JMX registration via sub-factories
+ * <li>{@link #withJmx(String)}
+ * <li>{@link #withJmxInternal()}
+ *
+ * Supports any of the above with the resultant executor propagating {@link ExecutorLocals} via sub-factory
+ * <li>{@link #localAware()}
+ *
+ * Supports shared executors via sub-factory {@code localAware().withJMX()}
+ * using {@link LocalAwareSubFactoryWithJMX#shared(String, int, ExecutorPlus.MaximumPoolSizeListener)}
+ */
+public interface ExecutorFactory extends ExecutorBuilderFactory.Jmxable<ExecutorPlus, SequentialExecutorPlus>
+{
+    public interface LocalAwareSubFactoryWithJMX extends ExecutorBuilderFactory<LocalAwareExecutorPlus, LocalAwareSequentialExecutorPlus>
+    {
+        LocalAwareExecutorPlus shared(String name, int threads, ExecutorPlus.MaximumPoolSizeListener onSetMaxSize);
+    }
+
+    public interface LocalAwareSubFactory extends ExecutorBuilderFactory<LocalAwareExecutorPlus, LocalAwareSequentialExecutorPlus>
+    {
+        LocalAwareSubFactoryWithJMX withJmx(String jmxPath);
+        default LocalAwareSubFactoryWithJMX withJmxInternal() { return withJmx("internal"); }
+    }
+
+    /**
+     * @return a factory that configures executors that propagate {@link ExecutorLocals} to the executing thread
+     */
+    LocalAwareSubFactory localAware();
+
+    /**
+     * @param name the name of the executor, the executor's thread group, and of any worker threads
+     * @return a default-configured {@link ScheduledExecutorPlus}
+     */
+    default ScheduledExecutorPlus scheduled(String name) { return scheduled(true, name, NORM_PRIORITY); }
+
+    /**
+     * @param executeOnShutdown if false, waiting tasks will be cancelled on shutdown
+     * @param name the name of the executor, the executor's thread group, and of any worker threads
+     * @return a {@link ScheduledExecutorPlus} with normal thread priority
+     */
+    default ScheduledExecutorPlus scheduled(boolean executeOnShutdown, String name) { return scheduled(executeOnShutdown, name, NORM_PRIORITY); }
+
+    /**
+     * @param executeOnShutdown if false, waiting tasks will be cancelled on shutdown
+     * @param name the name of the executor, the executor's thread group, and of any worker threads
+     * @param priority the thread priority of workers
+     * @return a {@link ScheduledExecutorPlus}
+     */
+    ScheduledExecutorPlus scheduled(boolean executeOnShutdown, String name, int priority);
+
+    /**
+     * Create and start a new thread to execute {@code runnable}
+     * @param name the name of the thread
+     * @param runnable the task to execute
+     * @return the new thread
+     */
+    Thread startThread(String name, Runnable runnable);
+
+    /**
+     * Create and start a new InfiniteLoopExecutor to repeatedly invoke {@code runnable}.
+     * On shutdown, the executing thread will be interrupted; to support clean shutdown
+     * {@code runnable} should propagate {@link InterruptedException}
+     *
+     * @param name the name of the thread used to invoke the task repeatedly
+     * @param task the task to execute repeatedly
+     * @return the new thread
+     */
+    Interruptible infiniteLoop(String name, Interruptible.Task task, boolean simulatorSafe);
+
+    /**
+     * Create and start a new InfiniteLoopExecutor to repeatedly invoke {@code runnable}.
+     * On shutdown, the executing thread will be interrupted; to support clean shutdown
+     * {@code runnable} should propagate {@link InterruptedException}
+     *
+     * @param name the name of the thread used to invoke the task repeatedly
+     * @param task the task to execute repeatedly
+     * @param interruptHandler perform specific processing of interrupts of the task execution thread
+     * @return the new thread
+     */
+    Interruptible infiniteLoop(String name, Interruptible.Task task, boolean simulatorSafe, Consumer<Thread> interruptHandler);
+
+    /**
+     * Create and start a new InfiniteLoopExecutor to repeatedly invoke {@code runnable}.
+     * On shutdown, the executing thread will be interrupted; to support clean shutdown
+     * {@code runnable} should propagate {@link InterruptedException}
+     *
+     * @param name the name of the thread used to invoke the task repeatedly
+     * @param task the task to execute repeatedly
+     * @return the new thread
+     */
+    default Interruptible infiniteLoop(String name, Interruptible.SimpleTask task, boolean simulatorSafe)
+    {
+        return infiniteLoop(name, Interruptible.Task.from(task), simulatorSafe);
+    }
+
+    /**
+     * Create a new thread group for use with builders - this thread group will be situated within
+     * this factory's parent thread group, and may be supplied to multiple executor builders.
+     */
+    ThreadGroup newThreadGroup(String name);
+
+    public static final class Global
+    {
+        // deliberately not volatile to ensure zero overhead outside of testing;
+        // depend on other memory visibility primitives to ensure visibility
+        private static ExecutorFactory FACTORY = new ExecutorFactory.Default(null, null, JVMStabilityInspector::uncaughtException);
+        public static ExecutorFactory executorFactory()
+        {
+            return FACTORY;
+        }
+
+        public static void unsafeSet(ExecutorFactory executorFactory)
+        {
+            FACTORY = executorFactory;
+        }
+    }
+
+    public static final class Default extends NamedThreadFactory.MetaFactory implements ExecutorFactory
+    {
+        public Default(ClassLoader contextClassLoader, ThreadGroup threadGroup, UncaughtExceptionHandler uncaughtExceptionHandler)
+        {
+            super(contextClassLoader, threadGroup, uncaughtExceptionHandler);
+        }
+
+        public LocalAwareSubFactory localAware()
+        {
+            return new LocalAwareSubFactory()
+            {
+                public ExecutorBuilder<? extends LocalAwareSequentialExecutorPlus> configureSequential(String name)
+                {
+                    return ThreadPoolExecutorBuilder.sequential(LocalAwareSingleThreadExecutorPlus::new, contextClassLoader, threadGroup, uncaughtExceptionHandler, name);
+                }
+
+                public ExecutorBuilder<LocalAwareThreadPoolExecutorPlus> configurePooled(String name, int threads)
+                {
+                    return ThreadPoolExecutorBuilder.pooled(LocalAwareThreadPoolExecutorPlus::new, contextClassLoader, threadGroup, uncaughtExceptionHandler, name, threads);
+                }
+
+                public LocalAwareSubFactoryWithJMX withJmx(String jmxPath)
+                {
+                    return new LocalAwareSubFactoryWithJMX()
+                    {
+                        public ExecutorBuilder<LocalAwareSingleThreadExecutorPlus> configureSequential(String name)
+                        {
+                            return sequentialJmx(LocalAwareSingleThreadExecutorPlus::new, contextClassLoader, threadGroup, uncaughtExceptionHandler, name, jmxPath);
+                        }
+
+                        public ExecutorBuilder<LocalAwareThreadPoolExecutorPlus> configurePooled(String name, int threads)
+                        {
+                            return pooledJmx(LocalAwareThreadPoolExecutorPlus::new, contextClassLoader, threadGroup, uncaughtExceptionHandler, name, threads, jmxPath);
+                        }
+
+                        public LocalAwareExecutorPlus shared(String name, int threads, ExecutorPlus.MaximumPoolSizeListener onSetMaxSize)
+                        {
+                            return SharedExecutorPool.SHARED.newExecutor(threads, onSetMaxSize, jmxPath, name);
+                        }
+                    };
+                }
+            };
+        }
+
+        @Override
+        public ExecutorBuilderFactory<ExecutorPlus, SequentialExecutorPlus> withJmx(String jmxPath)
+        {
+            return new ExecutorBuilderFactory<ExecutorPlus, SequentialExecutorPlus>()
+            {
+                @Override
+                public ExecutorBuilder<? extends SequentialExecutorPlus> configureSequential(String name)
+                {
+                    return ThreadPoolExecutorBuilder.sequentialJmx(SingleThreadExecutorPlus::new, contextClassLoader, threadGroup, uncaughtExceptionHandler, name, jmxPath);
+                }
+
+                @Override
+                public ExecutorBuilder<? extends ExecutorPlus> configurePooled(String name, int threads)
+                {
+                    return ThreadPoolExecutorBuilder.pooledJmx(ThreadPoolExecutorPlus::new, contextClassLoader, threadGroup, uncaughtExceptionHandler, name, threads, jmxPath);
+                }
+            };
+        }
+
+        public ExecutorBuilder<SingleThreadExecutorPlus> configureSequential(String name)
+        {
+            return ThreadPoolExecutorBuilder.sequential(SingleThreadExecutorPlus::new, contextClassLoader, threadGroup, uncaughtExceptionHandler, name);
+        }
+
+        public ExecutorBuilder<ThreadPoolExecutorPlus> configurePooled(String name, int threads)
+        {
+            return ThreadPoolExecutorBuilder.pooled(ThreadPoolExecutorPlus::new, contextClassLoader, threadGroup, uncaughtExceptionHandler, name, threads);
+        }
+
+        public ScheduledExecutorPlus scheduled(boolean executeOnShutdown, String name, int priority)
+        {
+            ScheduledThreadPoolExecutorPlus executor = new ScheduledThreadPoolExecutorPlus(newThreadFactory(name, priority));
+            if (!executeOnShutdown)
+                executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
+            return executor;
+        }
+
+        public Thread startThread(String name, Runnable runnable)
+        {
+            Thread thread = setupThread(createThread(threadGroup, runnable, name, true), Thread.NORM_PRIORITY, contextClassLoader, uncaughtExceptionHandler);
+            thread.start();
+            return thread;
+        }
+
+        public Interruptible infiniteLoop(String name, Interruptible.Task task, boolean simulatorSafe)
+        {
+            return new InfiniteLoopExecutor(this, name, task);
+        }
+
+        @Override
+        public Interruptible infiniteLoop(String name, Interruptible.Task task, boolean simulatorSafe, Consumer<Thread> interruptHandler)
+        {
+            return new InfiniteLoopExecutor(this, name, task, interruptHandler);
+        }
+
+        @Override
+        public ThreadGroup newThreadGroup(String name)
+        {
+            return threadGroup == null ? null : new ThreadGroup(threadGroup, name);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java b/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
index 8e6d6ea..4eeb2e5 100644
--- a/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
@@ -18,11 +18,11 @@
 
 package org.apache.cassandra.concurrent;
 
-import java.util.Arrays;
-
+import io.netty.util.concurrent.FastThreadLocal;
 import org.apache.cassandra.service.ClientWarn;
 import org.apache.cassandra.tracing.TraceState;
-import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.Closeable;
+import org.apache.cassandra.utils.WithResources;
 
 /*
  * This class only knows about Tracing and ClientWarn, so if any different executor locals are added, it must be
@@ -30,55 +30,80 @@ import org.apache.cassandra.tracing.Tracing;
  *
  * We don't enumerate the ExecutorLocal.all array each time because it would be much slower.
  */
-public class ExecutorLocals
+public class ExecutorLocals implements WithResources, Closeable
 {
-    private static final ExecutorLocal<TraceState> tracing = Tracing.instance;
-    private static final ExecutorLocal<ClientWarn.State> clientWarn = ClientWarn.instance;
+    private static final ExecutorLocals none = new ExecutorLocals(null, null);
+    private static final FastThreadLocal<ExecutorLocals> locals = new FastThreadLocal<ExecutorLocals>()
+    {
+        @Override
+        protected ExecutorLocals initialValue()
+        {
+            return none;
+        }
+    };
+
+    public static class Impl
+    {
+        @SuppressWarnings("resource")
+        protected static void set(TraceState traceState, ClientWarn.State clientWarnState)
+        {
+            if (traceState == null && clientWarnState == null) locals.set(none);
+            else locals.set(new ExecutorLocals(traceState, clientWarnState));
+        }
+    }
 
     public final TraceState traceState;
     public final ClientWarn.State clientWarnState;
 
-    private ExecutorLocals(TraceState traceState, ClientWarn.State clientWarnState)
+    protected ExecutorLocals(TraceState traceState, ClientWarn.State clientWarnState)
     {
         this.traceState = traceState;
         this.clientWarnState = clientWarnState;
     }
 
-    static
+    /**
+     * @return an ExecutorLocals object which has the current trace state and client warn state.
+     */
+    public static ExecutorLocals current()
     {
-        assert Arrays.equals(ExecutorLocal.all, new ExecutorLocal[]{ tracing, clientWarn })
-        : "ExecutorLocals has not been updated to reflect new ExecutorLocal.all";
+        return locals.get();
     }
 
     /**
-     * This creates a new ExecutorLocals object based on what is already set.
-     *
-     * @return an ExecutorLocals object which has the trace state and client warn state captured if either has been set,
-     *         or null if both are unset. The null result short-circuits logic in
-     *         {@link AbstractLocalAwareExecutorService#newTaskFor(Runnable, Object, ExecutorLocals)}, preventing
-     *         unnecessarily calling {@link ExecutorLocals#set(ExecutorLocals)}.
+     * The {@link #current}Locals, if any; otherwise {@link WithResources#none()}.
+     * Used to propagate current to other executors as a {@link WithResources}.
      */
-    public static ExecutorLocals create()
+    public static WithResources propagate()
     {
-        TraceState traceState = tracing.get();
-        ClientWarn.State clientWarnState = clientWarn.get();
-        if (traceState == null && clientWarnState == null)
-            return null;
-        else
-            return new ExecutorLocals(traceState, clientWarnState);
+        ExecutorLocals locals = current();
+        return locals == none ? WithResources.none() : locals;
     }
 
+    @SuppressWarnings("resource")
     public static ExecutorLocals create(TraceState traceState)
     {
-        ClientWarn.State clientWarnState = clientWarn.get();
-        return new ExecutorLocals(traceState, clientWarnState);
+        ExecutorLocals current = locals.get();
+        return current.traceState == traceState ? current : new ExecutorLocals(traceState, current.clientWarnState);
+    }
+
+    public static void clear()
+    {
+        locals.set(none);
+    }
+
+    /**
+     * Overwrite current locals, and return the previous ones
+     */
+    public Closeable get()
+    {
+        ExecutorLocals old = current();
+        if (old != this)
+            locals.set(this);
+        return old;
     }
 
-    public static void set(ExecutorLocals locals)
+    public void close()
     {
-        TraceState traceState = locals == null ? null : locals.traceState;
-        ClientWarn.State clientWarnState = locals == null ? null : locals.clientWarnState;
-        tracing.set(traceState);
-        clientWarn.set(clientWarnState);
+        locals.set(this);
     }
 }
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorPlus.java b/src/java/org/apache/cassandra/concurrent/ExecutorPlus.java
new file mode 100644
index 0000000..c42a475
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorPlus.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.concurrent;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.cassandra.utils.WithResources;
+import org.apache.cassandra.utils.concurrent.Future;
+
+/**
+ * Cassandra's extension of {@link ExecutorService}, using our own {@link Future}, supporting
+ * {@link #inExecutor()}, and execution with associated resources {@link #execute(WithResources, Runnable)}
+ * (which is primarily used for encapsulating {@link ExecutorLocals} without leaking implementing classes).
+ */
+public interface ExecutorPlus extends ExecutorService, ResizableThreadPool
+{
+    interface MaximumPoolSizeListener
+    {
+        /**
+         * Listener to follow changes to the maximum pool size
+         */
+        void onUpdateMaximumPoolSize(int maximumPoolSize);
+    }
+
+    /**
+     * MAY execute {@code task} immediately, if the calling thread is permitted to do so.
+     */
+    default void maybeExecuteImmediately(Runnable task)
+    {
+        execute(task);
+    }
+
+    /**
+     * Overrides {@link ExecutorService#submit(Callable)} to return a Cassandra {@link Future}
+     */
+    @Override
+    <T> Future<T> submit(Callable<T> task);
+
+    /**
+     * Overrides {@link ExecutorService#submit(Runnable, Object)} to return a Cassandra {@link Future}
+     */
+    @Override
+    <T> Future<T> submit(Runnable task, T result);
+
+    /**
+     * Overrides {@link ExecutorService#submit(Runnable)} to return a Cassandra {@link Future}
+     */
+    @Override
+    Future<?> submit(Runnable task);
+
+    /*
+     * ==============================================
+     * WithResources variants of submit and execute.
+     *
+     * (We need a way to inject a TraceState directly into the Executor context without going through
+     * the global Tracing sessions; see CASSANDRA-5668)
+     * ==============================================
+     */
+
+    /**
+     * Invoke {@code task}. The invoking thread will first instantiate the resources provided before
+     * invoking {@code task}, so that thread state may be modified and cleaned up.
+     *
+     * The invoking thread will execute something semantically equivlent to:
+     *
+     * <code>
+     *     try (Closeable close = withResources.get())
+     *     {
+     *         task.run();
+     *     }
+     * </code>
+     *
+     * @param withResources the resources to create and hold while executing {@code task}
+     * @param task the task to execute
+     */
+    void execute(WithResources withResources, Runnable task);
+
+    /**
+     * Invoke {@code task}, returning a future representing this computation.
+     * The invoking thread will first instantiate the resources provided before
+     * invoking {@code task}, so that thread state may be modified and cleaned up.
+     *
+     * The invoking thread will execute something semantically equivlent to:
+     *
+     * <code>
+     *     try (Closeable close = withResources.get())
+     *     {
+     *         return task.call();
+     *     }
+     * </code>
+     *
+     * @param withResources the resources to create and hold while executing {@code task}
+     * @param task the task to execute
+     */
+    <T> Future<T> submit(WithResources withResources, Callable<T> task);
+
+    /**
+     * Invoke {@code task}, returning a future yielding {@code null} if successful,
+     * or the abnormal termination of {@code task} otherwise.
+     *
+     * The invoking thread will first instantiate the resources provided before
+     * invoking {@code task}, so that thread state may be modified and cleaned up
+     *
+     * <code>
+     *     try (Closeable close = withResources.get())
+     *     {
+     *         task.run();
+     *         return null;
+     *     }
+     * </code>
+     *
+     * @param withResources the resources to create and hold while executing {@code task}
+     * @param task the task to execute
+     */
+    Future<?> submit(WithResources withResources, Runnable task);
+
+    /**
+     * Invoke {@code task}, returning a future yielding {@code result} if successful,
+     * or the abnormal termination of {@code task} otherwise.
+     *
+     * The invoking thread will first instantiate the resources provided before
+     * invoking {@code task}, so that thread state may be modified and cleaned up.
+     *
+     * The invoking thread will execute something semantically equivlent to:
+     *
+     * <code>
+     *     try (Closeable close = withResources.get())
+     *     {
+     *         task.run();
+     *         return result;
+     *     }
+     * </code>
+     *
+     * @param withResources the resources to create and hold while executing {@code task}
+     * @param task the task to execute
+     * @param result the result if successful
+     */
+    <T> Future<T> submit(WithResources withResources, Runnable task, T result);
+
+    /**
+     * @return true iff the caller is a worker thread actively serving this executor
+     */
+    boolean inExecutor();
+
+    default <T> List<java.util.concurrent.Future<T>> invokeAll(Collection<? extends Callable<T>> tasks) throws InterruptedException
+    {
+        throw new UnsupportedOperationException();
+    }
+    default <T> List<java.util.concurrent.Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
+    {
+        throw new UnsupportedOperationException();
+    }
+    default <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
+    {
+        throw new UnsupportedOperationException();
+    }
+    default <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+    {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/FutureTask.java b/src/java/org/apache/cassandra/concurrent/FutureTask.java
new file mode 100644
index 0000000..e2fc5cf
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/FutureTask.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.concurrent;
+
+import java.util.concurrent.Callable;
+
+import org.apache.cassandra.utils.concurrent.RunnableFuture;
+
+import org.apache.cassandra.utils.concurrent.AsyncFuture;
+
+/**
+ * A FutureTask that utilises Cassandra's {@link AsyncFuture}, making it compatible with {@link ExecutorPlus}.
+ * Propagates exceptions to the uncaught exception handler.
+ */
+public class FutureTask<V> extends AsyncFuture<V> implements RunnableFuture<V>
+{
+    private Callable<V> call;
+
+    public FutureTask(Callable<V> call)
+    {
+        this.call = call;
+    }
+
+    public FutureTask(Runnable run)
+    {
+        this.call = callable(run);
+    }
+
+    V call() throws Exception
+    {
+        return call.call();
+    }
+
+    public void run()
+    {
+        try
+        {
+            if (!setUncancellable())
+                return;
+
+            trySuccess(call());
+        }
+        catch (Throwable t)
+        {
+            tryFailure(t);
+        }
+        finally
+        {
+            call = null;
+        }
+    }
+
+    protected boolean tryFailure(Throwable t)
+    {
+        ExecutionFailure.handle(t);
+        return super.tryFailure(t);
+    }
+
+    public static <T> Callable<T> callable(Runnable run)
+    {
+        return new Callable<T>()
+        {
+            public T call()
+            {
+                run.run();
+                return null;
+            }
+
+            public String toString()
+            {
+                return run.toString();
+            }
+        };
+    }
+
+    public static <T> Callable<T> callable(Object id, Runnable run)
+    {
+        return new Callable<T>()
+        {
+            public T call()
+            {
+                run.run();
+                return null;
+            }
+
+            public String toString()
+            {
+                return id.toString();
+            }
+        };
+    }
+
+    public static <T> Callable<T> callable(Runnable run, T result)
+    {
+        return new Callable<T>()
+        {
+            public T call()
+            {
+                run.run();
+                return result;
+            }
+
+            public String toString()
+            {
+                return run + "->" + result;
+            }
+        };
+    }
+
+    public static <T> Callable<T> callable(Object id, Runnable run, T result)
+    {
+        return new Callable<T>()
+        {
+            public T call()
+            {
+                run.run();
+                return result;
+            }
+
+            public String toString()
+            {
+                return id.toString();
+            }
+        };
+    }
+
+    @Override
+    protected String description()
+    {
+        Object desc = call;
+        return desc == null ? null : call.toString();
+    }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/FutureTaskWithResources.java b/src/java/org/apache/cassandra/concurrent/FutureTaskWithResources.java
new file mode 100644
index 0000000..78c4987
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/FutureTaskWithResources.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.concurrent;
+
+import java.util.concurrent.Callable;
+
+import org.apache.cassandra.utils.Closeable;
+import org.apache.cassandra.utils.WithResources;
+import org.apache.cassandra.utils.concurrent.AsyncFuture;
+
+/**
+ * A FutureTask that utilises Cassandra's {@link AsyncFuture}, making it compatible with {@link ExecutorPlus}.
+ * Encapsulates a {@link WithResources}; the call will instantiate the resources before executing,
+ * and close them after executing but before completing the task.
+ *
+ * Propagates exceptions to the uncaught exception handler.
+ */
+public class FutureTaskWithResources<V> extends FutureTask<V>
+{
+    private final WithResources withResources;
+
+    public FutureTaskWithResources(WithResources withResources, Callable<V> call)
+    {
+        super(call);
+        this.withResources = withResources;
+    }
+
+    public FutureTaskWithResources(WithResources withResources, Runnable task)
+    {
+        super(task);
+        this.withResources = withResources;
+    }
+
+    V call() throws Exception
+    {
+        try (Closeable ignore = withResources.get())
+        {
+            return super.call();
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java b/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java
index 10c369c..14777ff 100644
--- a/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java
@@ -18,30 +18,127 @@
 
 package org.apache.cassandra.concurrent;
 
+import org.apache.cassandra.utils.Closeable;
+import org.apache.cassandra.utils.WithResources;
+import org.apache.cassandra.utils.concurrent.Future;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
+
 import java.util.Collections;
 import java.util.List;
-import java.util.concurrent.AbstractExecutorService;
+import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
-public class ImmediateExecutor extends AbstractExecutorService implements LocalAwareExecutorService
+public class ImmediateExecutor implements LocalAwareExecutorPlus
 {
     public static final ImmediateExecutor INSTANCE = new ImmediateExecutor();
 
     private ImmediateExecutor() {}
 
-    public void execute(Runnable command, ExecutorLocals locals)
+    public <T> Future<T> submit(Callable<T> task)
+    {
+        try
+        {
+            return ImmediateFuture.success(task.call());
+        }
+        catch (Throwable t)
+        {
+            ExecutionFailure.handle(t);
+            return ImmediateFuture.failure(t);
+        }
+    }
+
+    public <T> Future<T> submit(Runnable task, T result)
+    {
+        try
+        {
+            task.run();
+            return ImmediateFuture.success(result);
+        }
+        catch (Throwable t)
+        {
+            ExecutionFailure.handle(t);
+            return ImmediateFuture.failure(t);
+        }
+    }
+
+    public Future<?> submit(Runnable task)
+    {
+        try
+        {
+            task.run();
+            return ImmediateFuture.success(null);
+        }
+        catch (Throwable t)
+        {
+            ExecutionFailure.handle(t);
+            return ImmediateFuture.failure(t);
+        }
+    }
+
+    @Override
+    public void execute(WithResources withResources, Runnable task)
+    {
+        try (Closeable ignored = withResources.get())
+        {
+            task.run();
+        }
+        catch (Throwable t)
+        {
+            ExecutionFailure.handle(t);
+        }
+    }
+
+    @Override
+    public <T> Future<T> submit(WithResources withResources, Callable<T> task)
+    {
+        try (Closeable ignored = withResources.get())
+        {
+            return ImmediateFuture.success(task.call());
+        }
+        catch (Throwable t)
+        {
+            ExecutionFailure.handle(t);
+            return ImmediateFuture.failure(t);
+        }
+    }
+
+    @Override
+    public Future<?> submit(WithResources withResources, Runnable task)
+    {
+        return submit(withResources, task, null);
+    }
+
+    @Override
+    public <T> Future<T> submit(WithResources withResources, Runnable task, T result)
     {
-        command.run();
+        try (Closeable ignored = withResources.get())
+        {
+            task.run();
+            return ImmediateFuture.success(result);
+        }
+        catch (Throwable t)
+        {
+            ExecutionFailure.handle(t);
+            return ImmediateFuture.failure(t);
+        }
     }
 
-    public void maybeExecuteImmediately(Runnable command)
+    @Override
+    public boolean inExecutor()
     {
-        command.run();
+        return true;
     }
 
-    public void execute(Runnable command)
+    public void execute(Runnable task)
     {
-        command.run();
+        try
+        {
+            task.run();
+        }
+        catch (Throwable t)
+        {
+            ExecutionFailure.handle(t);
+        }
     }
 
     public int  getActiveTaskCount()    { return 0; }
diff --git a/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java b/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java
index 8e72d91..4012970 100644
--- a/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java
@@ -24,40 +24,77 @@ import org.slf4j.LoggerFactory;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.annotations.VisibleForTesting;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
 
-public class InfiniteLoopExecutor
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
+
+import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.InternalState.TERMINATED;
+import static org.apache.cassandra.concurrent.Interruptible.State.INTERRUPTED;
+import static org.apache.cassandra.concurrent.Interruptible.State.NORMAL;
+import static org.apache.cassandra.concurrent.Interruptible.State.SHUTTING_DOWN;
+
+public class InfiniteLoopExecutor implements Interruptible
 {
     private static final Logger logger = LoggerFactory.getLogger(InfiniteLoopExecutor.class);
 
-    public interface InterruptibleRunnable
+    public enum InternalState { TERMINATED }
+
+    private static final AtomicReferenceFieldUpdater<InfiniteLoopExecutor, Object> stateUpdater = AtomicReferenceFieldUpdater.newUpdater(InfiniteLoopExecutor.class, Object.class, "state");
+    private final Thread thread;
+    private final Task task;
+    private volatile Object state = NORMAL;
+    private final Consumer<Thread> interruptHandler;
+
+    public InfiniteLoopExecutor(String name, Task task)
     {
-        void run() throws InterruptedException;
+        this(ExecutorFactory.Global.executorFactory(), name, task, Thread::interrupt);
     }
 
-    private final Thread thread;
-    private final InterruptibleRunnable runnable;
-    private volatile boolean isShutdown = false;
+    public InfiniteLoopExecutor(ExecutorFactory factory, String name, Task task)
+    {
+        this(factory, name, task, Thread::interrupt);
+    }
+
+    public InfiniteLoopExecutor(ExecutorFactory factory, String name, Task task, Consumer<Thread> interruptHandler)
+    {
+        this.task = task;
+        this.thread = factory.startThread(name, this::loop);
+        this.interruptHandler = interruptHandler;
+    }
 
-    public InfiniteLoopExecutor(String name, InterruptibleRunnable runnable)
+    public InfiniteLoopExecutor(BiFunction<String, Runnable, Thread> threadStarter, String name, Task task, Consumer<Thread> interruptHandler)
     {
-        this.runnable = runnable;
-        this.thread = new Thread(this::loop, name);
-        this.thread.setDaemon(true);
+        this.task = task;
+        this.thread = threadStarter.apply(name, this::loop);
+        this.interruptHandler = interruptHandler;
     }
 
     private void loop()
     {
-        while (!isShutdown)
+        boolean interrupted = false;
+        while (true)
         {
             try
             {
-                runnable.run();
+                Object cur = state;
+                if (cur == TERMINATED) break;
+
+                interrupted |= Thread.interrupted();
+                if (cur == NORMAL && interrupted) cur = INTERRUPTED;
+                task.run((State) cur);
+
+                interrupted = false;
+                if (cur == SHUTTING_DOWN) state = TERMINATED;
             }
-            catch (InterruptedException ie)
+            catch (TerminateException ignore)
             {
-                if (isShutdown)
-                    return;
-                logger.error("Interrupted while executing {}, but not shutdown; continuing with loop", runnable, ie);
+                state = TERMINATED;
+            }
+            catch (UncheckedInterruptedException | InterruptedException ignore)
+            {
+                interrupted = true;
             }
             catch (Throwable t)
             {
@@ -66,22 +103,34 @@ public class InfiniteLoopExecutor
         }
     }
 
-    public InfiniteLoopExecutor start()
+    public void interrupt()
+    {
+        interruptHandler.accept(thread);
+    }
+
+    public void shutdown()
+    {
+        stateUpdater.updateAndGet(this, cur -> cur != TERMINATED ? SHUTTING_DOWN : TERMINATED);
+        interruptHandler.accept(thread);
+    }
+
+    public Object shutdownNow()
     {
-        thread.start();
-        return this;
+        state = TERMINATED;
+        interruptHandler.accept(thread);
+        return null;
     }
 
-    public void shutdownNow()
+    @Override
+    public boolean isTerminated()
     {
-        isShutdown = true;
-        thread.interrupt();
+        return state == TERMINATED && !thread.isAlive();
     }
 
     public boolean awaitTermination(long time, TimeUnit unit) throws InterruptedException
     {
         thread.join(unit.toMillis(time));
-        return !thread.isAlive();
+        return isTerminated();
     }
 
     @VisibleForTesting
diff --git a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java b/src/java/org/apache/cassandra/concurrent/Interruptible.java
similarity index 56%
copy from src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
copy to src/java/org/apache/cassandra/concurrent/Interruptible.java
index bd3b8ea..cc13a63 100644
--- a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
+++ b/src/java/org/apache/cassandra/concurrent/Interruptible.java
@@ -18,25 +18,32 @@
 
 package org.apache.cassandra.concurrent;
 
-public interface ResizableThreadPool
+import static org.apache.cassandra.concurrent.Interruptible.State.*;
+
+public interface Interruptible extends Shutdownable
 {
-    /**
-     * Returns maximum pool size of thread pool.
-     */
-    public int getCorePoolSize();
+    public enum State { NORMAL, INTERRUPTED, SHUTTING_DOWN }
 
-    /**
-     * Allows user to resize maximum size of the thread pool.
-     */
-    public void setCorePoolSize(int newCorePoolSize);
+    public static class TerminateException extends InterruptedException {}
 
-    /**
-     * Returns maximum pool size of thread pool.
-     */
-    public int getMaximumPoolSize();
+    public interface Task
+    {
+        void run(State state) throws InterruptedException;
+
+        static Task from(SimpleTask simpleTask)
+        {
+            return state -> { if (state == NORMAL) simpleTask.run(); };
+        }
+    }
 
     /**
-     * Allows user to resize maximum size of the thread pool.
+     * A Task that only runs on NORMAL states
      */
-    public void setMaximumPoolSize(int newMaximumPoolSize);
+    public interface SimpleTask
+    {
+        void run() throws InterruptedException;
+    }
+
+    void interrupt();
 }
+
diff --git a/src/java/org/apache/cassandra/concurrent/JMXEnabledSingleThreadExecutor.java b/src/java/org/apache/cassandra/concurrent/JMXEnabledSingleThreadExecutor.java
deleted file mode 100644
index 1e61aa1..0000000
--- a/src/java/org/apache/cassandra/concurrent/JMXEnabledSingleThreadExecutor.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.cassandra.concurrent;
-
-import java.util.concurrent.ThreadFactory;
-
-import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
-
-public class JMXEnabledSingleThreadExecutor extends JMXEnabledThreadPoolExecutor
-{
-    public JMXEnabledSingleThreadExecutor(String threadPoolName, String jmxPath)
-    {
-        super(1, Integer.MAX_VALUE, SECONDS, newBlockingQueue(), new SingleThreadFactory(threadPoolName), jmxPath);
-    }
-
-    @Override
-    public void setCoreThreads(int number)
-    {
-        throw new UnsupportedOperationException("Cannot change core pool size for single threaded executor.");
-    }
-
-    @Override
-    public void setMaximumThreads(int number)
-    {
-        throw new UnsupportedOperationException("Cannot change max threads for single threaded executor.");
-    }
-
-    @Override
-    public void setMaximumPoolSize(int newMaximumPoolSize)
-    {
-        setMaximumThreads(newMaximumPoolSize);
-    }
-
-    public boolean isExecutedBy(Thread test)
-    {
-        return getThreadFactory().thread == test;
-    }
-
-    public SingleThreadFactory getThreadFactory()
-    {
-        return (SingleThreadFactory) super.getThreadFactory();
-    }
-
-    public void setThreadFactory(ThreadFactory threadFactory)
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    private static class SingleThreadFactory extends NamedThreadFactory
-    {
-        private volatile Thread thread;
-        SingleThreadFactory(String id)
-        {
-            super(id);
-        }
-
-        @Override
-        public Thread newThread(Runnable r)
-        {
-            Thread thread = super.newThread(r);
-            this.thread = thread;
-            return thread;
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutor.java b/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutor.java
deleted file mode 100644
index 4283d4f..0000000
--- a/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutor.java
+++ /dev/null
@@ -1,191 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.concurrent;
-
-import java.util.List;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.RejectedExecutionHandler;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.cassandra.metrics.ThreadPoolMetrics;
-import org.apache.cassandra.utils.MBeanWrapper;
-
-/**
- * This is a wrapper class for the <i>ScheduledThreadPoolExecutor</i>. It provides an implementation
- * for the <i>afterExecute()</i> found in the <i>ThreadPoolExecutor</i> class to log any unexpected
- * Runtime Exceptions.
- */
-
-public class JMXEnabledThreadPoolExecutor extends DebuggableThreadPoolExecutor implements JMXEnabledThreadPoolExecutorMBean
-{
-    private final String mbeanName;
-    public final ThreadPoolMetrics metrics;
-
-    public JMXEnabledThreadPoolExecutor(String threadPoolName)
-    {
-        this(1, Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), new NamedThreadFactory(threadPoolName), "internal");
-    }
-
-    public JMXEnabledThreadPoolExecutor(String threadPoolName, String jmxPath)
-    {
-        this(1, Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), new NamedThreadFactory(threadPoolName), jmxPath);
-    }
-
-    public JMXEnabledThreadPoolExecutor(String threadPoolName, int priority)
-    {
-        this(1, Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), new NamedThreadFactory(threadPoolName, priority), "internal");
-    }
-
-    public JMXEnabledThreadPoolExecutor(NamedThreadFactory threadFactory, String jmxPath)
-    {
-        this(1, Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), threadFactory, jmxPath);
-    }
-
-    public JMXEnabledThreadPoolExecutor(int corePoolSize,
-            long keepAliveTime,
-            TimeUnit unit,
-            BlockingQueue<Runnable> workQueue,
-            NamedThreadFactory threadFactory,
-            String jmxPath)
-    {
-        this(corePoolSize, corePoolSize, keepAliveTime, unit, workQueue, threadFactory, jmxPath);
-    }
-
-    public JMXEnabledThreadPoolExecutor(int corePoolSize,
-                                        int maxPoolSize,
-                                        long keepAliveTime,
-                                        TimeUnit unit,
-                                        BlockingQueue<Runnable> workQueue,
-                                        NamedThreadFactory threadFactory,
-                                        String jmxPath)
-    {
-        super(corePoolSize, maxPoolSize, keepAliveTime, unit, workQueue, threadFactory);
-        super.prestartAllCoreThreads();
-        metrics = new ThreadPoolMetrics(this, jmxPath, threadFactory.id).register();
-
-        mbeanName = "org.apache.cassandra." + jmxPath + ":type=" + threadFactory.id;
-        MBeanWrapper.instance.registerMBean(this, mbeanName);
-    }
-
-    public JMXEnabledThreadPoolExecutor(int corePoolSize,
-                                        int maxPoolSize,
-                                        long keepAliveTime,
-                                        TimeUnit unit,
-                                        BlockingQueue<Runnable> workQueue,
-                                        NamedThreadFactory threadFactory,
-                                        String jmxPath,
-                                        RejectedExecutionHandler rejectedExecutionHandler)
-    {
-        this(corePoolSize, maxPoolSize, keepAliveTime, unit, workQueue, threadFactory, jmxPath);
-        setRejectedExecutionHandler(rejectedExecutionHandler);
-    }
-
-    private void unregisterMBean()
-    {
-        MBeanWrapper.instance.unregisterMBean(mbeanName);
-
-        // release metrics
-        metrics.release();
-    }
-
-    @Override
-    public synchronized void shutdown()
-    {
-        // synchronized, because there is no way to access super.mainLock, which would be
-        // the preferred way to make this threadsafe
-        if (!isShutdown())
-        {
-            unregisterMBean();
-        }
-        super.shutdown();
-    }
-
-    @Override
-    public synchronized List<Runnable> shutdownNow()
-    {
-        // synchronized, because there is no way to access super.mainLock, which would be
-        // the preferred way to make this threadsafe
-        if (!isShutdown())
-        {
-            unregisterMBean();
-        }
-        return super.shutdownNow();
-    }
-
-    public int getTotalBlockedTasks()
-    {
-        return (int) metrics.totalBlocked.getCount();
-    }
-
-    public int getCurrentlyBlockedTasks()
-    {
-        return (int) metrics.currentBlocked.getCount();
-    }
-
-    @Deprecated
-    public int getCoreThreads()
-    {
-        return getCorePoolSize();
-    }
-
-    @Deprecated
-    public void setCoreThreads(int number)
-    {
-        setCorePoolSize(number);
-    }
-
-    @Deprecated
-    public int getMaximumThreads()
-    {
-        return getMaximumPoolSize();
-    }
-
-    @Deprecated
-    public void setMaximumThreads(int number)
-    {
-        setMaximumPoolSize(number);
-    }
-
-    @Override
-    public void setMaximumPoolSize(int newMaximumPoolSize)
-    {
-        if (newMaximumPoolSize < getCorePoolSize())
-            throw new IllegalArgumentException("maximum pool size cannot be less than core pool size");
-        super.setMaximumPoolSize(newMaximumPoolSize);
-    }
-
-    @Override
-    protected void onInitialRejection(Runnable task)
-    {
-        metrics.totalBlocked.inc();
-        metrics.currentBlocked.inc();
-    }
-
-    @Override
-    protected void onFinalAccept(Runnable task)
-    {
-        metrics.currentBlocked.dec();
-    }
-
-    @Override
-    protected void onFinalRejection(Runnable task)
-    {
-        metrics.currentBlocked.dec();
-    }
-}
diff --git a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java b/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorPlus.java
similarity index 62%
copy from src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
copy to src/java/org/apache/cassandra/concurrent/LocalAwareExecutorPlus.java
index bd3b8ea..7509619 100644
--- a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
+++ b/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorPlus.java
@@ -18,25 +18,9 @@
 
 package org.apache.cassandra.concurrent;
 
-public interface ResizableThreadPool
+/**
+ * An {@link ExecutorPlus} that is aware of, and propagates to execution, any ExecutorLocals
+ */
+public interface LocalAwareExecutorPlus extends ExecutorPlus
 {
-    /**
-     * Returns maximum pool size of thread pool.
-     */
-    public int getCorePoolSize();
-
-    /**
-     * Allows user to resize maximum size of the thread pool.
-     */
-    public void setCorePoolSize(int newCorePoolSize);
-
-    /**
-     * Returns maximum pool size of thread pool.
-     */
-    public int getMaximumPoolSize();
-
-    /**
-     * Allows user to resize maximum size of the thread pool.
-     */
-    public void setMaximumPoolSize(int newMaximumPoolSize);
 }
diff --git a/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
deleted file mode 100644
index d6ac8e4..0000000
--- a/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- *
- */
-
-package org.apache.cassandra.concurrent;
-
-import java.util.concurrent.ExecutorService;
-
-public interface LocalAwareExecutorService extends ExecutorService, ResizableThreadPool
-{
-    // we need a way to inject a TraceState directly into the Executor context without going through
-    // the global Tracing sessions; see CASSANDRA-5668
-    void execute(Runnable command, ExecutorLocals locals);
-
-    // permits executing in the context of the submitting thread
-    void maybeExecuteImmediately(Runnable command);
-
-    /**
-     * Returns the approximate number of threads that are actively
-     * executing tasks.
-     *
-     * @return the number of threads
-     */
-    int getActiveTaskCount();
-
-    /**
-     * Returns the approximate total number of tasks that have
-     * completed execution. Because the states of tasks and threads
-     * may change dynamically during computation, the returned value
-     * is only an approximation, but one that does not ever decrease
-     * across successive calls.
-     *
-     * @return the number of tasks
-     */
-    long getCompletedTaskCount();
-
-    /**
-     * Returns the approximate total of tasks waiting to be executed.
-     * Because the states of tasks and threads
-     * may change dynamically during computation, the returned value
-     * is only an approximation, but one that does not ever decrease
-     * across successive calls.
-     *
-     * @return the number of tasks
-     */
-    int getPendingTaskCount();
-
-    default int getMaxTasksQueued()
-    {
-        return -1;
-    }
-
-    interface MaximumPoolSizeListener
-    {
-        /**
-         * Listener to follow changes to the maximum pool size
-         */
-        void onUpdateMaximumPoolSize(int maximumPoolSize);
-    }
-}
diff --git a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java b/src/java/org/apache/cassandra/concurrent/LocalAwareSequentialExecutorPlus.java
similarity index 62%
copy from src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
copy to src/java/org/apache/cassandra/concurrent/LocalAwareSequentialExecutorPlus.java
index bd3b8ea..dbcff6c 100644
--- a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
+++ b/src/java/org/apache/cassandra/concurrent/LocalAwareSequentialExecutorPlus.java
@@ -18,25 +18,9 @@
 
 package org.apache.cassandra.concurrent;
 
-public interface ResizableThreadPool
+/**
+ * A {@link SequentialExecutorPlus} that is aware of, and propagates to execution, any ExecutorLocals
+ */
+public interface LocalAwareSequentialExecutorPlus extends LocalAwareExecutorPlus, SequentialExecutorPlus
 {
-    /**
-     * Returns maximum pool size of thread pool.
-     */
-    public int getCorePoolSize();
-
-    /**
-     * Allows user to resize maximum size of the thread pool.
-     */
-    public void setCorePoolSize(int newCorePoolSize);
-
-    /**
-     * Returns maximum pool size of thread pool.
-     */
-    public int getMaximumPoolSize();
-
-    /**
-     * Allows user to resize maximum size of the thread pool.
-     */
-    public void setMaximumPoolSize(int newMaximumPoolSize);
 }
diff --git a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java b/src/java/org/apache/cassandra/concurrent/LocalAwareSingleThreadExecutorPlus.java
similarity index 62%
copy from src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
copy to src/java/org/apache/cassandra/concurrent/LocalAwareSingleThreadExecutorPlus.java
index bd3b8ea..cf67c1a 100644
--- a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
+++ b/src/java/org/apache/cassandra/concurrent/LocalAwareSingleThreadExecutorPlus.java
@@ -15,28 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.cassandra.concurrent;
 
-public interface ResizableThreadPool
+public class LocalAwareSingleThreadExecutorPlus extends SingleThreadExecutorPlus implements LocalAwareSequentialExecutorPlus
 {
-    /**
-     * Returns maximum pool size of thread pool.
-     */
-    public int getCorePoolSize();
-
-    /**
-     * Allows user to resize maximum size of the thread pool.
-     */
-    public void setCorePoolSize(int newCorePoolSize);
-
-    /**
-     * Returns maximum pool size of thread pool.
-     */
-    public int getMaximumPoolSize();
-
-    /**
-     * Allows user to resize maximum size of the thread pool.
-     */
-    public void setMaximumPoolSize(int newMaximumPoolSize);
+    LocalAwareSingleThreadExecutorPlus(ThreadPoolExecutorBuilder<LocalAwareSingleThreadExecutorPlus> builder)
+    {
+        super(builder, TaskFactory.localAware());
+    }
 }
diff --git a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java b/src/java/org/apache/cassandra/concurrent/LocalAwareThreadPoolExecutorPlus.java
similarity index 62%
copy from src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
copy to src/java/org/apache/cassandra/concurrent/LocalAwareThreadPoolExecutorPlus.java
index bd3b8ea..10d107e 100644
--- a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
+++ b/src/java/org/apache/cassandra/concurrent/LocalAwareThreadPoolExecutorPlus.java
@@ -15,28 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.cassandra.concurrent;
 
-public interface ResizableThreadPool
+public class LocalAwareThreadPoolExecutorPlus extends ThreadPoolExecutorPlus implements LocalAwareExecutorPlus
 {
-    /**
-     * Returns maximum pool size of thread pool.
-     */
-    public int getCorePoolSize();
-
-    /**
-     * Allows user to resize maximum size of the thread pool.
-     */
-    public void setCorePoolSize(int newCorePoolSize);
-
-    /**
-     * Returns maximum pool size of thread pool.
-     */
-    public int getMaximumPoolSize();
-
-    /**
-     * Allows user to resize maximum size of the thread pool.
-     */
-    public void setMaximumPoolSize(int newMaximumPoolSize);
+    LocalAwareThreadPoolExecutorPlus(ThreadPoolExecutorBuilder<? extends LocalAwareThreadPoolExecutorPlus> builder)
+    {
+        super(builder, TaskFactory.localAware());
+    }
 }
diff --git a/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java b/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
index 5aadb48..32df3f3 100644
--- a/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
+++ b/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
@@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import com.google.common.annotations.VisibleForTesting;
 
 import io.netty.util.concurrent.FastThreadLocalThread;
+import org.apache.cassandra.utils.JVMStabilityInspector;
 
 /**
  * This class is an implementation of the <i>ThreadFactory</i> interface. This
@@ -32,18 +33,49 @@ import io.netty.util.concurrent.FastThreadLocalThread;
 
 public class NamedThreadFactory implements ThreadFactory
 {
+    private static final AtomicInteger anonymousCounter = new AtomicInteger();
     private static volatile String globalPrefix;
+
     public static void setGlobalPrefix(String prefix) { globalPrefix = prefix; }
-    public static String globalPrefix() {
+    public static String globalPrefix()
+    {
         String prefix = globalPrefix;
         return prefix == null ? "" : prefix;
     }
 
+    public static class MetaFactory
+    {
+        protected ClassLoader contextClassLoader;
+        protected ThreadGroup threadGroup;
+        protected Thread.UncaughtExceptionHandler uncaughtExceptionHandler;
+
+        public MetaFactory(ClassLoader contextClassLoader, ThreadGroup threadGroup, Thread.UncaughtExceptionHandler uncaughtExceptionHandler)
+        {
+            this.contextClassLoader = contextClassLoader;
+            if (threadGroup == null)
+            {
+                threadGroup = Thread.currentThread().getThreadGroup();
+                while (threadGroup.getParent() != null)
+                    threadGroup = threadGroup.getParent();
+            }
+            this.threadGroup = threadGroup;
+            this.uncaughtExceptionHandler = uncaughtExceptionHandler;
+        }
+
+        NamedThreadFactory newThreadFactory(String name, int threadPriority)
+        {
+            // We create a unique thread group for each factory, so that e.g. executors can determine which threads are members of the executor
+            ThreadGroup threadGroup = this.threadGroup == null ? null : new ThreadGroup(this.threadGroup, name);
+            return new NamedThreadFactory(name, threadPriority, contextClassLoader, threadGroup, uncaughtExceptionHandler);
+        }
+    }
+
     public final String id;
     private final int priority;
     private final ClassLoader contextClassLoader;
-    private final ThreadGroup threadGroup;
+    public final ThreadGroup threadGroup;
     protected final AtomicInteger n = new AtomicInteger(1);
+    private final Thread.UncaughtExceptionHandler uncaughtExceptionHandler;
 
     public NamedThreadFactory(String id)
     {
@@ -52,33 +84,60 @@ public class NamedThreadFactory implements ThreadFactory
 
     public NamedThreadFactory(String id, int priority)
     {
-        this(id, priority, null, null);
+        this(id, priority, null, null, JVMStabilityInspector::uncaughtException);
+    }
+
+    public NamedThreadFactory(String id, ClassLoader contextClassLoader, ThreadGroup threadGroup)
+    {
+        this(id, Thread.NORM_PRIORITY, contextClassLoader, threadGroup, JVMStabilityInspector::uncaughtException);
     }
 
     public NamedThreadFactory(String id, int priority, ClassLoader contextClassLoader, ThreadGroup threadGroup)
     {
+        this(id, priority, contextClassLoader, threadGroup, JVMStabilityInspector::uncaughtException);
+    }
+    public NamedThreadFactory(String id, int priority, ClassLoader contextClassLoader, ThreadGroup threadGroup, Thread.UncaughtExceptionHandler uncaughtExceptionHandler)
+    {
         this.id = id;
         this.priority = priority;
         this.contextClassLoader = contextClassLoader;
         this.threadGroup = threadGroup;
+        this.uncaughtExceptionHandler = uncaughtExceptionHandler;
     }
 
+    @Override
     public Thread newThread(Runnable runnable)
     {
         String name = id + ':' + n.getAndIncrement();
-        Thread thread = createThread(threadGroup, runnable, name, true);
+        return newThread(threadGroup, runnable, name);
+    }
+
+    protected Thread newThread(ThreadGroup threadGroup, Runnable runnable, String name)
+    {
+        return setupThread(createThread(threadGroup, runnable, name, true));
+    }
+
+    protected <T extends Thread> T setupThread(T thread)
+    {
+        return setupThread(thread, priority, contextClassLoader, uncaughtExceptionHandler);
+    }
+
+    public static Thread createThread(ThreadGroup threadGroup, Runnable runnable, String name, int priority, ClassLoader contextClassLoader, Thread.UncaughtExceptionHandler uncaughtExceptionHandler)
+    {
+        String prefix = globalPrefix;
+        Thread thread = createThread(threadGroup, runnable, prefix != null ? prefix + name : name, true);
         thread.setPriority(priority);
         if (contextClassLoader != null)
             thread.setContextClassLoader(contextClassLoader);
+        if (uncaughtExceptionHandler != null)
+            thread.setUncaughtExceptionHandler(uncaughtExceptionHandler);
         return thread;
     }
 
-    private static final AtomicInteger threadCounter = new AtomicInteger();
-
     @VisibleForTesting
-    public static Thread createThread(Runnable runnable)
+    public static Thread createAnonymousThread(Runnable runnable)
     {
-        return createThread(null, runnable, "anonymous-" + threadCounter.incrementAndGet());
+        return createThread(null, runnable, "anonymous-" + anonymousCounter.incrementAndGet());
     }
 
     public static Thread createThread(Runnable runnable, String name)
@@ -86,7 +145,7 @@ public class NamedThreadFactory implements ThreadFactory
         return createThread(null, runnable, name);
     }
 
-    public static Thread createThread(Runnable runnable, String name, boolean daemon)
+    public Thread createThread(Runnable runnable, String name, boolean daemon)
     {
         return createThread(null, runnable, name, daemon);
     }
@@ -103,4 +162,37 @@ public class NamedThreadFactory implements ThreadFactory
         thread.setDaemon(daemon);
         return thread;
     }
+
+    public static  <T extends Thread> T setupThread(T thread, int priority, ClassLoader contextClassLoader, Thread.UncaughtExceptionHandler uncaughtExceptionHandler)
+    {
+        thread.setPriority(priority);
+        if (contextClassLoader != null)
+            thread.setContextClassLoader(contextClassLoader);
+        if (uncaughtExceptionHandler != null)
+            thread.setUncaughtExceptionHandler(uncaughtExceptionHandler);
+        return thread;
+    }
+
+    @Override
+    public String toString()
+    {
+        return id;
+    }
+
+    public void close()
+    {
+        synchronized (threadGroup)
+        {
+            threadGroup.setDaemon(true);
+            // ThreadGroup API is terrible; setDaemon does not destroy if already empty, and establishing if empty
+            // otherwise is tortuous - easier to just try to destroy and fail if currently an invalid action
+            try
+            {
+                threadGroup.destroy();
+            }
+            catch (IllegalThreadStateException ignore)
+            {
+            }
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java b/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
index bd3b8ea..760c06e 100644
--- a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
+++ b/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
@@ -21,12 +21,14 @@ package org.apache.cassandra.concurrent;
 public interface ResizableThreadPool
 {
     /**
-     * Returns maximum pool size of thread pool.
+     * Returns core pool size of thread pool, the minimum
+     * number of workers (where that makes sense for a thread pool,
+     * SEPExecutor does not have a minimum size).
      */
     public int getCorePoolSize();
 
     /**
-     * Allows user to resize maximum size of the thread pool.
+     * Allows user to resize minimum size of the thread pool.
      */
     public void setCorePoolSize(int newCorePoolSize);
 
@@ -39,4 +41,37 @@ public interface ResizableThreadPool
      * Allows user to resize maximum size of the thread pool.
      */
     public void setMaximumPoolSize(int newMaximumPoolSize);
+
+    /**
+     * Returns the approximate number of threads that are actively
+     * executing tasks.
+     *
+     * @return the number of threads
+     */
+    int getActiveTaskCount();
+
+    /**
+     * Returns the approximate total number of tasks that have
+     * completed execution. Because the states of tasks and threads
+     * may change dynamically during computation, the returned value
+     * is only an approximation, but one that does not ever decrease
+     * across successive calls.
+     *
+     * @return the number of tasks
+     */
+    long getCompletedTaskCount();
+
+    /**
+     * Returns the approximate total of tasks waiting to be executed.
+     * Because the states of tasks and threads may change dynamically
+     * during computation, the returned value is only an approximation.
+     *
+     * @return the number of tasks
+     */
+    int getPendingTaskCount();
+
+    default int getMaxTasksQueued()
+    {
+        return -1;
+    }
 }
diff --git a/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutorMBean.java b/src/java/org/apache/cassandra/concurrent/ResizableThreadPoolMXBean.java
similarity index 95%
rename from src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutorMBean.java
rename to src/java/org/apache/cassandra/concurrent/ResizableThreadPoolMXBean.java
index c2959df..1c247d6 100644
--- a/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutorMBean.java
+++ b/src/java/org/apache/cassandra/concurrent/ResizableThreadPoolMXBean.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.concurrent;
 
-public interface JMXEnabledThreadPoolExecutorMBean extends ResizableThreadPool
+public interface ResizableThreadPoolMXBean extends ResizableThreadPool
 {
     /**
      * Returns core pool size of thread pool.
diff --git a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
index 9085ee4..05b59c6 100644
--- a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.concurrent;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -26,6 +27,8 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import org.apache.cassandra.utils.WithResources;
+import org.apache.cassandra.utils.concurrent.Future;
 import org.apache.cassandra.utils.MBeanWrapper;
 import org.apache.cassandra.utils.concurrent.Condition;
 import org.slf4j.Logger;
@@ -37,9 +40,11 @@ import static org.apache.cassandra.concurrent.SEPExecutor.TakeTaskPermitResult.*
 import static org.apache.cassandra.concurrent.SEPWorker.Work;
 import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
 
-public class SEPExecutor extends AbstractLocalAwareExecutorService implements SEPExecutorMBean
+public class SEPExecutor implements LocalAwareExecutorPlus, SEPExecutorMBean
 {
     private static final Logger logger = LoggerFactory.getLogger(SEPExecutor.class);
+    private static final TaskFactory taskFactory = TaskFactory.localAware();
+
     private final SharedExecutorPool pool;
 
     private final AtomicInteger maximumPoolSize;
@@ -60,7 +65,7 @@ public class SEPExecutor extends AbstractLocalAwareExecutorService implements SE
     final Condition shutdown = newOneTimeCondition();
 
     // TODO: see if other queue implementations might improve throughput
-    protected final ConcurrentLinkedQueue<FutureTask<?>> tasks = new ConcurrentLinkedQueue<>();
+    protected final ConcurrentLinkedQueue<Runnable> tasks = new ConcurrentLinkedQueue<>();
 
     SEPExecutor(SharedExecutorPool pool, int maximumPoolSize, MaximumPoolSizeListener maximumPoolSizeListener, String jmxPath, String name)
     {
@@ -96,7 +101,7 @@ public class SEPExecutor extends AbstractLocalAwareExecutorService implements SE
         return true;
     }
 
-    protected void addTask(FutureTask<?> task)
+    protected <T extends Runnable> T addTask(T task)
     {
         // we add to the queue first, so that when a worker takes a task permit it can be certain there is a task available
         // this permits us to schedule threads non-spuriously; it also means work is serviced fairly
@@ -121,6 +126,7 @@ public class SEPExecutor extends AbstractLocalAwareExecutorService implements SE
             // worker, we simply start a worker in a spinning state
             pool.maybeStartSpinningWorker();
         }
+        return task;
     }
 
     public enum TakeTaskPermitResult
@@ -128,7 +134,7 @@ public class SEPExecutor extends AbstractLocalAwareExecutorService implements SE
         NONE_AVAILABLE,        // No task permits available
         TOOK_PERMIT,           // Took a permit and reduced task permits
         RETURNED_WORK_PERMIT   // Detected pool shrinking and returned work permit ahead of SEPWorker exit.
-    };
+    }
 
     // takes permission to perform a task, if any are available; once taken it is guaranteed
     // that a proceeding call to tasks.poll() will return some work
@@ -194,18 +200,18 @@ public class SEPExecutor extends AbstractLocalAwareExecutorService implements SE
     }
 
     @Override
-    public void maybeExecuteImmediately(Runnable command)
+    public void maybeExecuteImmediately(Runnable task)
     {
-        FutureTask<?> ft = newTaskFor(command, null);
+        task = taskFactory.toExecute(task);
         if (!takeWorkPermit(false))
         {
-            addTask(ft);
+            addTask(task);
         }
         else
         {
             try
             {
-                ft.run();
+                task.run();
             }
             finally
             {
@@ -218,6 +224,60 @@ public class SEPExecutor extends AbstractLocalAwareExecutorService implements SE
         }
     }
 
+    @Override
+    public void execute(Runnable run)
+    {
+        addTask(taskFactory.toExecute(run));
+    }
+
+    @Override
+    public void execute(WithResources withResources, Runnable run)
+    {
+        addTask(taskFactory.toExecute(withResources, run));
+    }
+
+    @Override
+    public Future<?> submit(Runnable run)
+    {
+        return addTask(taskFactory.toSubmit(run));
+    }
+
+    @Override
+    public <T> Future<T> submit(Runnable run, T result)
+    {
+        return addTask(taskFactory.toSubmit(run, result));
+    }
+
+    @Override
+    public <T> Future<T> submit(Callable<T> call)
+    {
+        return addTask(taskFactory.toSubmit(call));
+    }
+
+    @Override
+    public <T> Future<T> submit(WithResources withResources, Runnable run, T result)
+    {
+        return addTask(taskFactory.toSubmit(withResources, run, result));
+    }
+
+    @Override
+    public Future<?> submit(WithResources withResources, Runnable run)
+    {
+        return addTask(taskFactory.toSubmit(withResources, run));
+    }
+
+    @Override
+    public <T> Future<T> submit(WithResources withResources, Callable<T> call)
+    {
+        return addTask(taskFactory.toSubmit(withResources, call));
+    }
+
+    @Override
+    public boolean inExecutor()
+    {
+        throw new UnsupportedOperationException();
+    }
+
     public synchronized void shutdown()
     {
         if (shuttingDown)
diff --git a/src/java/org/apache/cassandra/concurrent/SEPWorker.java b/src/java/org/apache/cassandra/concurrent/SEPWorker.java
index efb1884..c7b9abf 100644
--- a/src/java/org/apache/cassandra/concurrent/SEPWorker.java
+++ b/src/java/org/apache/cassandra/concurrent/SEPWorker.java
@@ -48,11 +48,11 @@ final class SEPWorker extends AtomicReference<SEPWorker.Work> implements Runnabl
     long prevStopCheck = 0;
     long soleSpinnerSpinTime = 0;
 
-    SEPWorker(Long workerId, Work initialState, SharedExecutorPool pool)
+    SEPWorker(ThreadGroup threadGroup, Long workerId, Work initialState, SharedExecutorPool pool)
     {
         this.pool = pool;
         this.workerId = workerId;
-        thread = new FastThreadLocalThread(this, pool.poolName + "-Worker-" + workerId);
+        thread = new FastThreadLocalThread(threadGroup, this, threadGroup.getName() + "-Worker-" + workerId);
         thread.setDaemon(true);
         set(initialState);
         thread.start();
@@ -118,6 +118,7 @@ final class SEPWorker extends AtomicReference<SEPWorker.Work> implements Runnabl
 
                     // we know there is work waiting, as we have a work permit, so poll() will always succeed
                     task.run();
+                    assigned.onCompletion();
                     task = null;
 
                     if (shutdown = assigned.shuttingDown)
@@ -163,9 +164,14 @@ final class SEPWorker extends AtomicReference<SEPWorker.Work> implements Runnabl
             if (assigned != null)
                 assigned.returnWorkPermit();
             if (task != null)
+            {
                 logger.error("Failed to execute task, unexpected exception killed worker", t);
+                assigned.onCompletion();
+            }
             else
+            {
                 logger.error("Unexpected exception killed worker", t);
+            }
         }
     }
 
diff --git a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java b/src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java
similarity index 62%
copy from src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
copy to src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java
index bd3b8ea..0b512ac 100644
--- a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
+++ b/src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java
@@ -18,25 +18,8 @@
 
 package org.apache.cassandra.concurrent;
 
-public interface ResizableThreadPool
-{
-    /**
-     * Returns maximum pool size of thread pool.
-     */
-    public int getCorePoolSize();
-
-    /**
-     * Allows user to resize maximum size of the thread pool.
-     */
-    public void setCorePoolSize(int newCorePoolSize);
+import java.util.concurrent.ScheduledExecutorService;
 
-    /**
-     * Returns maximum pool size of thread pool.
-     */
-    public int getMaximumPoolSize();
-
-    /**
-     * Allows user to resize maximum size of the thread pool.
-     */
-    public void setMaximumPoolSize(int newMaximumPoolSize);
+public interface ScheduledExecutorPlus extends ExecutorPlus, ScheduledExecutorService
+{
 }
diff --git a/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java b/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java
index ff9d1b4..8da600c 100644
--- a/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java
+++ b/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java
@@ -24,6 +24,8 @@ import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.cassandra.utils.ExecutorUtils;
 
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
+
 /**
  * Centralized location for shared executors
  */
@@ -32,26 +34,26 @@ public class ScheduledExecutors
     /**
      * This pool is used for periodic fast (sub-microsecond) tasks.
      */
-    public static final DebuggableScheduledThreadPoolExecutor scheduledFastTasks = new DebuggableScheduledThreadPoolExecutor("ScheduledFastTasks");
+    public static final ScheduledExecutorPlus scheduledFastTasks = executorFactory().scheduled("ScheduledFastTasks");
 
     /**
      * This pool is used for periodic short (sub-second) tasks.
      */
-     public static final DebuggableScheduledThreadPoolExecutor scheduledTasks = new DebuggableScheduledThreadPoolExecutor("ScheduledTasks");
+     public static final ScheduledExecutorPlus scheduledTasks = executorFactory().scheduled("ScheduledTasks");
 
     /**
      * This executor is used for tasks that can have longer execution times, and usually are non periodic.
      */
-    public static final DebuggableScheduledThreadPoolExecutor nonPeriodicTasks = new DebuggableScheduledThreadPoolExecutor("NonPeriodicTasks");
+    public static final ScheduledExecutorPlus nonPeriodicTasks = executorFactory().scheduled("NonPeriodicTasks");
 
     /**
      * This executor is used for tasks that do not need to be waited for on shutdown/drain.
      */
-    public static final DebuggableScheduledThreadPoolExecutor optionalTasks = new DebuggableScheduledThreadPoolExecutor("OptionalTasks");
+    public static final ScheduledExecutorPlus optionalTasks = executorFactory().scheduled(false, "OptionalTasks");
 
     @VisibleForTesting
-    public static void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
+    public static void shutdownNowAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
     {
-        ExecutorUtils.shutdownNowAndWait(timeout, unit, scheduledFastTasks, scheduledTasks, nonPeriodicTasks, optionalTasks);
+        ExecutorUtils.shutdownNowAndWait(timeout, unit, scheduledTasks, scheduledFastTasks, nonPeriodicTasks, optionalTasks);
     }
 }
diff --git a/src/java/org/apache/cassandra/concurrent/ScheduledThreadPoolExecutorPlus.java b/src/java/org/apache/cassandra/concurrent/ScheduledThreadPoolExecutorPlus.java
new file mode 100644
index 0000000..efd284f
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ScheduledThreadPoolExecutorPlus.java
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.concurrent;
+
+import java.util.List;
+import java.util.concurrent.*;
+
+import org.apache.cassandra.utils.WithResources;
+import org.apache.cassandra.utils.concurrent.Future;
+import org.apache.cassandra.utils.concurrent.RunnableFuture;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.service.StorageService;
+
+import static org.apache.cassandra.concurrent.ExecutionFailure.propagating;
+import static org.apache.cassandra.concurrent.ExecutionFailure.suppressing;
+
+/**
+ * Like ExecutorPlus, ScheduledThreadPoolExecutorPlus always
+ * logs exceptions from the tasks it is given, even if Future.get is never called elsewhere.
+ *
+ * Catches exceptions during Task execution so that they don't suppress subsequent invocations of the task.
+ *
+ * Finally, there is a special rejected execution handler for tasks rejected during the shutdown hook.
+ *  - For fire and forget tasks (like ref tidy) we can safely ignore the exceptions.
+ *  - For any callers that care to know their task was rejected we cancel passed task.
+ */
+public class ScheduledThreadPoolExecutorPlus extends ScheduledThreadPoolExecutor implements ScheduledExecutorPlus
+{
+    private static final Logger logger = LoggerFactory.getLogger(ScheduledThreadPoolExecutorPlus.class);
+    private static final TaskFactory taskFactory = TaskFactory.standard();
+
+    public static final RejectedExecutionHandler rejectedExecutionHandler = (task, executor) ->
+    {
+        if (executor.isShutdown())
+        {
+            // TODO: this sequence of events seems poorly thought out
+            if (!StorageService.instance.isShutdown())
+                throw new RejectedExecutionException("ScheduledThreadPoolExecutor has shut down.");
+
+            //Give some notification to the caller the task isn't going to run
+            if (task instanceof java.util.concurrent.Future)
+                ((java.util.concurrent.Future<?>) task).cancel(false);
+
+            logger.debug("ScheduledThreadPoolExecutor has shut down as part of C* shutdown");
+        }
+        else
+        {
+            throw new AssertionError("Unknown rejection of ScheduledThreadPoolExecutor task");
+        }
+    };
+
+    ScheduledThreadPoolExecutorPlus(NamedThreadFactory threadFactory)
+    {
+        super(1, threadFactory);
+        setRejectedExecutionHandler(rejectedExecutionHandler);
+    }
+
+    @Override
+    public ScheduledFuture<?> schedule(Runnable task, long delay, TimeUnit unit)
+    {
+        return super.schedule(propagating(task), delay, unit);
+    }
+
+    @Override
+    public <V> ScheduledFuture<V> schedule(Callable<V> task, long delay, TimeUnit unit)
+    {
+        return super.schedule(propagating(task), delay, unit);
+    }
+
+    // override scheduling to suppress exceptions that would cancel future executions
+    @Override
+    public ScheduledFuture<?> scheduleAtFixedRate(Runnable task, long initialDelay, long period, TimeUnit unit)
+    {
+        return super.scheduleAtFixedRate(suppressing(task), initialDelay, period, unit);
+    }
+
+    @Override
+    public ScheduledFuture<?> scheduleWithFixedDelay(Runnable task, long initialDelay, long delay, TimeUnit unit)
+    {
+        return super.scheduleWithFixedDelay(suppressing(task), initialDelay, delay, unit);
+    }
+
+    /*======== BEGIN DIRECT COPY OF ThreadPoolExecutorPlus ===============*/
+
+    private <T extends Runnable> T addTask(T task)
+    {
+        super.execute(task);
+        return task;
+    }
+
+    @Override
+    public void execute(Runnable run)
+    {
+        addTask(taskFactory.toExecute(run));
+    }
+
+    @Override
+    public void execute(WithResources withResources, Runnable run)
+    {
+        addTask(taskFactory.toExecute(withResources, run));
+    }
+
+    @Override
+    public Future<?> submit(Runnable run)
+    {
+        return addTask(taskFactory.toSubmit(run));
+    }
+
+    @Override
+    public <T> Future<T> submit(Runnable run, T result)
+    {
+        return addTask(taskFactory.toSubmit(run, result));
+    }
+
+    @Override
+    public <T> Future<T> submit(Callable<T> call)
+    {
+        return addTask(taskFactory.toSubmit(call));
+    }
+
+    @Override
+    public <T> Future<T> submit(WithResources withResources, Runnable run, T result)
+    {
+        return addTask(taskFactory.toSubmit(withResources, run, result));
+    }
+
+    @Override
+    public Future<?> submit(WithResources withResources, Runnable run)
+    {
+        return addTask(taskFactory.toSubmit(withResources, run));
+    }
+
+    @Override
+    public <T> Future<T> submit(WithResources withResources, Callable<T> call)
+    {
+        return addTask(taskFactory.toSubmit(withResources, call));
+    }
+
+    @Override
+    public boolean inExecutor()
+    {
+        return Thread.currentThread().getThreadGroup() == getThreadFactory().threadGroup;
+    }
+
+    @Override
+    protected <T> RunnableFuture<T> newTaskFor(Runnable runnable, T value)
+    {
+        return taskFactory.toSubmit(runnable, value);
+    }
+
+    @Override
+    protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable)
+    {
+        return taskFactory.toSubmit(callable);
+    }
+
+    @Override
+    public NamedThreadFactory getThreadFactory()
+    {
+        return (NamedThreadFactory) super.getThreadFactory();
+    }
+
+    /*======== DIRECT COPY OF ThreadPoolExecutorBase ===============*/
+
+    @Override
+    public List<Runnable> shutdownNow()
+    {
+        List<Runnable> cancelled = super.shutdownNow();
+        for (Runnable c : cancelled)
+        {
+            if (c instanceof java.util.concurrent.Future<?>)
+                ((java.util.concurrent.Future<?>) c).cancel(true);
+        }
+        return cancelled;
+    }
+
+    @Override
+    protected void terminated()
+    {
+        getThreadFactory().close();
+    }
+
+    @Override
+    public int getActiveTaskCount()
+    {
+        return getActiveCount();
+    }
+
+    @Override
+    public int getPendingTaskCount()
+    {
+        return getQueue().size();
+    }
+
+    /*======== DIRECT COPY OF SingleThreadExecutorPlus ===============*/
+
+    @Override
+    public int getCorePoolSize()
+    {
+        return 1;
+    }
+    @Override
+    public void setCorePoolSize(int number)
+    {
+        throw new UnsupportedOperationException();
+    }
+    @Override
+    public int getMaximumPoolSize()
+    {
+        return 1;
+    }
+    @Override
+    public void setMaximumPoolSize(int number)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int getMaxTasksQueued()
+    {
+        return Integer.MAX_VALUE;
+    }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/SequentialExecutorPlus.java b/src/java/org/apache/cassandra/concurrent/SequentialExecutorPlus.java
new file mode 100644
index 0000000..7ea0e95
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/SequentialExecutorPlus.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.concurrent;
+
+/**
+ * An {@link ExecutorPlus} that guarantees the order of execution matches the order of task submission,
+ * and provides a simple mechanism for the recurring pattern of ensuring a job is executed at least once
+ * after some point in time (i.e. ensures that at most one copy of the task is queued, with up to one
+ * copy running as well)
+ */
+public interface SequentialExecutorPlus extends ExecutorPlus
+{
+    public interface AtLeastOnceTrigger
+    {
+        /**
+         * Ensure the job is run at least once in its entirety after this method is invoked (including any already queued)
+         */
+        public boolean trigger();
+
+        /**
+         * Run the provided task after all queued and executing jobs have completed
+         */
+        public void runAfter(Runnable run);
+
+        /**
+         * Wait until all queued and executing jobs have completed
+         */
+        public void sync();
+    }
+
+    /**
+     * Return an object for orchestrating the execution of this task at least once (in its entirety) after
+     * the trigger is invoked, i.e. saturating the number of pending tasks at 1 (2 including any possibly executing
+     * at the time of invocation)
+     */
+    public AtLeastOnceTrigger atLeastOnceTrigger(Runnable runnable);
+}
diff --git a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
index bba8e84..f74854f 100644
--- a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
+++ b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
@@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.LockSupport;
 
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.concurrent.SEPWorker.Work;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
 
@@ -57,11 +58,10 @@ import static org.apache.cassandra.utils.Clock.Global.nanoTime;
  */
 public class SharedExecutorPool
 {
-
     public static final SharedExecutorPool SHARED = new SharedExecutorPool("SharedPool");
 
     // the name assigned to workers in the pool, and the id suffix
-    final String poolName;
+    final ThreadGroup threadGroup;
     final AtomicLong workerId = new AtomicLong();
 
     // the collection of executors serviced by this pool; periodically ordered by traffic volume
@@ -80,9 +80,14 @@ public class SharedExecutorPool
 
     volatile boolean shuttingDown = false;
 
-    public SharedExecutorPool(String poolName)
+    public SharedExecutorPool(String name)
+    {
+        this(executorFactory().newThreadGroup(name));
+    }
+
+    public SharedExecutorPool(ThreadGroup threadGroup)
     {
-        this.poolName = poolName;
+        this.threadGroup = threadGroup;
     }
 
     void schedule(Work work)
@@ -97,7 +102,7 @@ public class SharedExecutorPool
                 return;
 
         if (!work.isStop())
-            new SEPWorker(workerId.incrementAndGet(), work, this);
+            new SEPWorker(threadGroup, workerId.incrementAndGet(), work, this);
     }
 
     void maybeStartSpinningWorker()
@@ -109,12 +114,12 @@ public class SharedExecutorPool
             schedule(Work.SPINNING);
     }
 
-    public synchronized LocalAwareExecutorService newExecutor(int maxConcurrency, String jmxPath, String name)
+    public synchronized LocalAwareExecutorPlus newExecutor(int maxConcurrency, String jmxPath, String name)
     {
         return newExecutor(maxConcurrency, i -> {}, jmxPath, name);
     }
 
-    public LocalAwareExecutorService newExecutor(int maxConcurrency, LocalAwareExecutorService.MaximumPoolSizeListener maximumPoolSizeListener, String jmxPath, String name)
+    public LocalAwareExecutorPlus newExecutor(int maxConcurrency, ExecutorPlus.MaximumPoolSizeListener maximumPoolSizeListener, String jmxPath, String name)
     {
         SEPExecutor executor = new SEPExecutor(this, maxConcurrency, maximumPoolSizeListener, jmxPath, name);
         executors.add(executor);
diff --git a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java b/src/java/org/apache/cassandra/concurrent/Shutdownable.java
similarity index 62%
copy from src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
copy to src/java/org/apache/cassandra/concurrent/Shutdownable.java
index bd3b8ea..db89217 100644
--- a/src/java/org/apache/cassandra/concurrent/ResizableThreadPool.java
+++ b/src/java/org/apache/cassandra/concurrent/Shutdownable.java
@@ -18,25 +18,24 @@
 
 package org.apache.cassandra.concurrent;
 
-public interface ResizableThreadPool
+import java.util.concurrent.TimeUnit;
+
+public interface Shutdownable
 {
-    /**
-     * Returns maximum pool size of thread pool.
-     */
-    public int getCorePoolSize();
+    boolean isTerminated();
 
     /**
-     * Allows user to resize maximum size of the thread pool.
+     * Shutdown once any remaining work has completed (however this is defined for the implementation).
      */
-    public void setCorePoolSize(int newCorePoolSize);
+    void shutdown();
 
     /**
-     * Returns maximum pool size of thread pool.
+     * Shutdown immediately, possibly interrupting ongoing work, and cancelling work that is queued.
      */
-    public int getMaximumPoolSize();
+    Object shutdownNow();
 
     /**
-     * Allows user to resize maximum size of the thread pool.
+     * Await termination of this object, i.e. the cessation of all current and future work.
      */
-    public void setMaximumPoolSize(int newMaximumPoolSize);
+    public boolean awaitTermination(long timeout, TimeUnit units) throws InterruptedException;
 }
diff --git a/src/java/org/apache/cassandra/concurrent/SingleThreadExecutorPlus.java b/src/java/org/apache/cassandra/concurrent/SingleThreadExecutorPlus.java
new file mode 100644
index 0000000..e72a6a4
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/SingleThreadExecutorPlus.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.concurrent;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.cassandra.utils.concurrent.Future;
+
+public class SingleThreadExecutorPlus extends ThreadPoolExecutorPlus implements SequentialExecutorPlus
+{
+    public static class AtLeastOnce extends AtomicBoolean implements AtLeastOnceTrigger, Runnable
+    {
+        protected final SequentialExecutorPlus executor;
+        protected final Runnable run;
+
+        public AtLeastOnce(SequentialExecutorPlus executor, Runnable run)
+        {
+            this.executor = executor;
+            this.run = run;
+        }
+
+        public boolean trigger()
+        {
+            boolean success;
+            if (success = compareAndSet(false, true))
+                executor.execute(this);
+            return success;
+        }
+
+        public void runAfter(Runnable run)
+        {
+            executor.execute(run);
+        }
+
+        public void sync()
+        {
+            Future<?> done = executor.submit(() -> {});
+            done.awaitThrowUncheckedOnInterrupt();
+            done.rethrowIfFailed(); // executor might get shutdown before we execute; propagate cancellation exception
+        }
+
+        public void run()
+        {
+            set(false);
+            run.run();
+        }
+    }
+
+    SingleThreadExecutorPlus(ThreadPoolExecutorBuilder<? extends SingleThreadExecutorPlus> builder)
+    {
+        this(builder, TaskFactory.standard());
+    }
+
+    SingleThreadExecutorPlus(ThreadPoolExecutorBuilder<? extends SingleThreadExecutorPlus> builder, TaskFactory taskFactory)
+    {
+        super(builder, taskFactory);
+    }
+
+    @Override
+    public int getCorePoolSize()
+    {
+        return 1;
+    }
+    @Override
+    public void setCorePoolSize(int number)
+    {
+        throw new UnsupportedOperationException();
+    }
+    @Override
+    public int getMaximumPoolSize()
+    {
+        return 1;
+    }
+    @Override
+    public void setMaximumPoolSize(int number)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public AtLeastOnce atLeastOnceTrigger(Runnable run)
+    {
+        return new AtLeastOnce(this, run);
+    }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/Stage.java b/src/java/org/apache/cassandra/concurrent/Stage.java
index a34c3d3..66cd7cb 100644
--- a/src/java/org/apache/cassandra/concurrent/Stage.java
+++ b/src/java/org/apache/cassandra/concurrent/Stage.java
@@ -20,14 +20,7 @@ package org.apache.cassandra.concurrent;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.RejectedExecutionHandler;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.function.IntSupplier;
@@ -41,12 +34,11 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.Verb;
 import org.apache.cassandra.utils.ExecutorUtils;
-
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.Future;
 
-import static java.util.concurrent.TimeUnit.SECONDS;
 import static java.util.stream.Collectors.toMap;
-import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 
 public enum Stage
 {
@@ -57,18 +49,17 @@ public enum Stage
     GOSSIP            ("GossipStage",           "internal", () -> 1,                                         null,                                            Stage::singleThreadedStage),
     REQUEST_RESPONSE  ("RequestResponseStage",  "request",  FBUtilities::getAvailableProcessors,             null,                                            Stage::multiThreadedLowSignalStage),
     ANTI_ENTROPY      ("AntiEntropyStage",      "internal", () -> 1,                                         null,                                            Stage::singleThreadedStage),
-    MIGRATION         ("MigrationStage",        "internal", () -> 1,                                         null,                                            Stage::singleThreadedStage),
+    MIGRATION         ("MigrationStage",        "internal", () -> 1,                                         null,                                            Stage::migrationStage),
     MISC              ("MiscStage",             "internal", () -> 1,                                         null,                                            Stage::singleThreadedStage),
-    TRACING           ("TracingStage",          "internal", () -> 1,                                         null,                                            Stage::tracingExecutor),
+    TRACING           ("TracingStage",          "internal", () -> 1,                                         null,                                            Stage::tracingStage),
     INTERNAL_RESPONSE ("InternalResponseStage", "internal", FBUtilities::getAvailableProcessors,             null,                                            Stage::multiThreadedStage),
     IMMEDIATE         ("ImmediateStage",        "internal", () -> 0,                                         null,                                            Stage::immediateExecutor);
 
-    public static final long KEEP_ALIVE_SECONDS = 60; // seconds to keep "extra" threads alive for when idle
     public final String jmxName;
-    private final Supplier<LocalAwareExecutorService> initialiser;
-    private volatile LocalAwareExecutorService executor = null;
+    private final Supplier<ExecutorPlus> initialiser;
+    private volatile ExecutorPlus executor = null;
 
-    Stage(String jmxName, String jmxType, IntSupplier numThreads, LocalAwareExecutorService.MaximumPoolSizeListener onSetMaximumPoolSize, ExecutorServiceInitialiser initialiser)
+    Stage(String jmxName, String jmxType, IntSupplier numThreads, LocalAwareExecutorPlus.MaximumPoolSizeListener onSetMaximumPoolSize, ExecutorServiceInitialiser initialiser)
     {
         this.jmxName = jmxName;
         this.initialiser = () -> initialiser.init(jmxName, jmxType, numThreads.getAsInt(), onSetMaximumPoolSize);
@@ -122,14 +113,14 @@ public enum Stage
     }
 
     // Convenience functions to execute on this stage
-    public void execute(Runnable command) { executor().execute(command); }
-    public void execute(Runnable command, ExecutorLocals locals) { executor().execute(command, locals); }
-    public void maybeExecuteImmediately(Runnable command) { executor().maybeExecuteImmediately(command); }
+    public void execute(Runnable task) { executor().execute(task); }
+    public void execute(ExecutorLocals locals, Runnable task) { executor().execute(locals, task); }
+    public void maybeExecuteImmediately(Runnable task) { executor().maybeExecuteImmediately(task); }
     public <T> Future<T> submit(Callable<T> task) { return executor().submit(task); }
     public Future<?> submit(Runnable task) { return executor().submit(task); }
     public <T> Future<T> submit(Runnable task, T result) { return executor().submit(task, result); }
 
-    public LocalAwareExecutorService executor()
+    public ExecutorPlus executor()
     {
         if (executor == null)
         {
@@ -144,7 +135,7 @@ public enum Stage
         return executor;
     }
 
-    private static List<ExecutorService> executors()
+    private static List<ExecutorPlus> executors()
     {
         return Stream.of(Stage.values())
                      .map(Stage::executor)
@@ -162,44 +153,52 @@ public enum Stage
     @VisibleForTesting
     public static void shutdownAndWait(long timeout, TimeUnit units) throws InterruptedException, TimeoutException
     {
-        List<ExecutorService> executors = executors();
+        List<ExecutorPlus> executors = executors();
         ExecutorUtils.shutdownNow(executors);
         ExecutorUtils.awaitTermination(timeout, units, executors);
     }
 
-    static LocalAwareExecutorService tracingExecutor(String jmxName, String jmxType, int numThreads, LocalAwareExecutorService.MaximumPoolSizeListener onSetMaximumPoolSize)
+    private static ExecutorPlus tracingStage(String jmxName, String jmxType, int numThreads, LocalAwareExecutorPlus.MaximumPoolSizeListener onSetMaximumPoolSize)
+    {
+        return executorFactory()
+                .withJmx(jmxType)
+                .configureSequential(jmxName)
+                .withQueueLimit(1000)
+                .withRejectedExecutionHandler((r, executor) -> MessagingService.instance().metrics.recordSelfDroppedMessage(Verb._TRACE)).build();
+    }
+
+    private static ExecutorPlus migrationStage(String jmxName, String jmxType, int numThreads, LocalAwareExecutorPlus.MaximumPoolSizeListener onSetMaximumPoolSize)
     {
-        RejectedExecutionHandler reh = (r, executor) -> MessagingService.instance().metrics.recordSelfDroppedMessage(Verb._TRACE);
-        return new TracingExecutor(1,
-                                   1,
-                                   KEEP_ALIVE_SECONDS,
-                                   TimeUnit.SECONDS,
-                                   new ArrayBlockingQueue<>(1000),
-                                   new NamedThreadFactory(jmxName),
-                                   reh);
+        return executorFactory()
+               .withJmx(jmxType)
+               .sequential(jmxName);
     }
 
-    static LocalAwareExecutorService multiThreadedStage(String jmxName, String jmxType, int numThreads, LocalAwareExecutorService.MaximumPoolSizeListener onSetMaximumPoolSize)
+    private static LocalAwareExecutorPlus singleThreadedStage(String jmxName, String jmxType, int numThreads, LocalAwareExecutorPlus.MaximumPoolSizeListener onSetMaximumPoolSize)
     {
-        return new JMXEnabledThreadPoolExecutor(numThreads,
-                                                KEEP_ALIVE_SECONDS,
-                                                SECONDS,
-                                                newBlockingQueue(),
-                                                new NamedThreadFactory(jmxName),
-                                                jmxType);
+        return executorFactory()
+                .localAware()
+                .withJmx(jmxType)
+                .sequential(jmxName);
     }
 
-    static LocalAwareExecutorService multiThreadedLowSignalStage(String jmxName, String jmxType, int numThreads, LocalAwareExecutorService.MaximumPoolSizeListener onSetMaximumPoolSize)
+    static LocalAwareExecutorPlus multiThreadedStage(String jmxName, String jmxType, int numThreads, LocalAwareExecutorPlus.MaximumPoolSizeListener onSetMaximumPoolSize)
     {
-        return SharedExecutorPool.SHARED.newExecutor(numThreads, onSetMaximumPoolSize, jmxType, jmxName);
+        return executorFactory()
+                .localAware()
+                .withJmx(jmxType)
+                .pooled(jmxName, numThreads);
     }
 
-    static LocalAwareExecutorService singleThreadedStage(String jmxName, String jmxType, int numThreads, LocalAwareExecutorService.MaximumPoolSizeListener onSetMaximumPoolSize)
+    static LocalAwareExecutorPlus multiThreadedLowSignalStage(String jmxName, String jmxType, int numThreads, LocalAwareExecutorPlus.MaximumPoolSizeListener onSetMaximumPoolSize)
     {
-        return new JMXEnabledSingleThreadExecutor(jmxName, jmxType);
+        return executorFactory()
+                .localAware()
+                .withJmx(jmxType)
+                .shared(jmxName, numThreads, onSetMaximumPoolSize);
     }
 
-    static LocalAwareExecutorService immediateExecutor(String jmxName, String jmxType, int numThreads, LocalAwareExecutorService.MaximumPoolSizeListener onSetMaximumPoolSize)
+    static LocalAwareExecutorPlus immediateExecutor(String jmxName, String jmxType, int numThreads, LocalAwareExecutorPlus.MaximumPoolSizeListener onSetMaximumPoolSize)
     {
         return ImmediateExecutor.INSTANCE;
     }
@@ -207,7 +206,7 @@ public enum Stage
     @FunctionalInterface
     public interface ExecutorServiceInitialiser
     {
-        public LocalAwareExecutorService init(String jmxName, String jmxType, int numThreads, LocalAwareExecutorService.MaximumPoolSizeListener onSetMaximumPoolSize);
+        public ExecutorPlus init(String jmxName, String jmxType, int numThreads, LocalAwareExecutorPlus.MaximumPoolSizeListener onSetMaximumPoolSize);
     }
 
     /**
@@ -241,38 +240,4 @@ public enum Stage
     {
         executor().setMaximumPoolSize(newMaximumPoolSize);
     }
-
-    /**
-     * The executor used for tracing.
-     */
-    private static class TracingExecutor extends ThreadPoolExecutor implements LocalAwareExecutorService
-    {
-        TracingExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory, RejectedExecutionHandler handler)
-        {
-            super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler);
-        }
-
-        public void execute(Runnable command, ExecutorLocals locals)
-        {
-            assert locals == null;
-            super.execute(command);
-        }
-
-        public void maybeExecuteImmediately(Runnable command)
-        {
-            execute(command);
-        }
-
-        @Override
-        public int getActiveTaskCount()
-        {
-            return getActiveCount();
-        }
-
-        @Override
-        public int getPendingTaskCount()
-        {
-            return getQueue().size();
-        }
-    }
 }
diff --git a/src/java/org/apache/cassandra/concurrent/SyncFutureTask.java b/src/java/org/apache/cassandra/concurrent/SyncFutureTask.java
new file mode 100644
index 0000000..19ca27f
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/SyncFutureTask.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.concurrent;
+
+import java.util.concurrent.Callable;
+
+import org.apache.cassandra.utils.Closeable;
+import org.apache.cassandra.utils.WithResources;
+import org.apache.cassandra.utils.concurrent.RunnableFuture;
+import org.apache.cassandra.utils.concurrent.SyncFuture;
+
+public class SyncFutureTask<T> extends SyncFuture<T> implements RunnableFuture<T>
+{
+    final Callable<T> call;
+
+    public SyncFutureTask(Callable<T> call)
+    {
+        this.call = call;
+    }
+
+    public SyncFutureTask(WithResources withResources, Callable<T> call)
+    {
+        this.call = () -> {
+            try (Closeable close = withResources.get())
+            {
+                return call.call();
+            }
+        };
+    }
+
+    public void run()
+    {
+        try
+        {
+            if (!setUncancellable())
+                throw new IllegalStateException();
+
+            if (!trySuccess(call.call()))
+                throw new IllegalStateException();
+        }
+        catch (Throwable t)
+        {
+            tryFailure(t);
+            Thread thread = Thread.currentThread();
+            thread.getUncaughtExceptionHandler().uncaughtException(thread, t);
+        }
+    }
+
+    @Override
+    public String description()
+    {
+        return call.toString();
+    }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/TaskFactory.java b/src/java/org/apache/cassandra/concurrent/TaskFactory.java
new file mode 100644
index 0000000..a25a45f
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/TaskFactory.java
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.concurrent;
+
+import java.util.concurrent.Callable;
+
+import org.apache.cassandra.utils.WithResources;
+import org.apache.cassandra.utils.concurrent.RunnableFuture;
+
+import static org.apache.cassandra.concurrent.FutureTask.callable;
+
+/**
+ * A simple mechanism to impose our desired semantics on the execution of a task without requiring a specialised
+ * executor service. We wrap tasks in a suitable {@link FutureTask} or encapsulating {@link Runnable}.
+ *
+ * The encapsulations handle any exceptions in our standard way, as well as ensuring {@link ExecutorLocals} are
+ * propagated in the case of {@link #localAware()}
+ */
+public interface TaskFactory
+{
+    Runnable toExecute(Runnable runnable);
+    <T> RunnableFuture<T> toSubmit(Runnable runnable);
+    <T> RunnableFuture<T> toSubmit(Runnable runnable, T result);
+    <T> RunnableFuture<T> toSubmit(Callable<T> callable);
+
+    Runnable toExecute(WithResources withResources, Runnable runnable);
+    <T> RunnableFuture<T> toSubmit(WithResources withResources, Runnable runnable);
+    <T> RunnableFuture<T> toSubmit(WithResources withResources, Runnable runnable, T result);
+    <T> RunnableFuture<T> toSubmit(WithResources withResources, Callable<T> callable);
+
+    static TaskFactory standard() { return Standard.INSTANCE; }
+    static TaskFactory localAware() { return LocalAware.INSTANCE; }
+
+    public class Standard implements TaskFactory
+    {
+        static final Standard INSTANCE = new Standard();
+        protected Standard() {}
+
+        @Override
+        public Runnable toExecute(Runnable runnable)
+        {
+            return ExecutionFailure.suppressing(runnable);
+        }
+
+        @Override
+        public <T> RunnableFuture<T> toSubmit(Runnable runnable)
+        {
+            return newTask(callable(runnable));
+        }
+
+        @Override
+        public <T> RunnableFuture<T> toSubmit(Runnable runnable, T result)
+        {
+            return newTask(callable(runnable, result));
+        }
+
+        @Override
+        public <T> RunnableFuture<T> toSubmit(Callable<T> callable)
+        {
+            return newTask(callable);
+        }
+
+        @Override
+        public Runnable toExecute(WithResources withResources, Runnable runnable)
+        {
+            return ExecutionFailure.suppressing(withResources, runnable);
+        }
+
+        @Override
+        public <T> RunnableFuture<T> toSubmit(WithResources withResources, Runnable runnable)
+        {
+            return withResources.isNoOp() ? newTask(callable(runnable))
+                                          : newTask(withResources, callable(runnable));
+        }
+
+        @Override
+        public <T> RunnableFuture<T> toSubmit(WithResources withResources, Runnable runnable, T result)
+        {
+            return withResources.isNoOp() ? newTask(callable(runnable, result))
+                                          : newTask(withResources, callable(runnable, result));
+        }
+
+        @Override
+        public <T> RunnableFuture<T> toSubmit(WithResources withResources, Callable<T> callable)
+        {
+            return withResources.isNoOp() ? newTask(callable)
+                                          : newTask(withResources, callable);
+        }
+
+        protected <T> RunnableFuture<T> newTask(Callable<T> call)
+        {
+            return new FutureTask<>(call);
+        }
+
+        protected <T> RunnableFuture<T> newTask(WithResources withResources, Callable<T> call)
+        {
+            return new FutureTaskWithResources<>(withResources, call);
+        }
+    }
+
+    public class LocalAware extends Standard
+    {
+        static final LocalAware INSTANCE = new LocalAware();
+
+        protected LocalAware() {}
+
+        @Override
+        public Runnable toExecute(Runnable runnable)
+        {
+            // no reason to propagate exception when it is inaccessible to caller
+            return ExecutionFailure.suppressing(ExecutorLocals.propagate(), runnable);
+        }
+
+        @Override
+        public <T> RunnableFuture<T> toSubmit(Runnable runnable)
+        {
+            return super.toSubmit(ExecutorLocals.propagate(), runnable);
+        }
+
+        @Override
+        public <T> RunnableFuture<T> toSubmit(Runnable runnable, T result)
+        {
+            return super.toSubmit(ExecutorLocals.propagate(), runnable, result);
+        }
+
+        @Override
+        public <T> RunnableFuture<T> toSubmit(Callable<T> callable)
+        {
+            return super.toSubmit(ExecutorLocals.propagate(), callable);
+        }
+
+        @Override
+        public Runnable toExecute(WithResources withResources, Runnable runnable)
+        {
+            return ExecutionFailure.suppressing(withLocals(withResources), runnable);
+        }
+
+        @Override
+        public <T> RunnableFuture<T> toSubmit(WithResources withResources, Runnable runnable)
+        {
+            return super.toSubmit(withLocals(withResources), runnable);
+        }
+
+        @Override
+        public <T> RunnableFuture<T> toSubmit(WithResources withResources, Runnable runnable, T result)
+        {
+            return super.toSubmit(withLocals(withResources), runnable, result);
+        }
+
+        @Override
+        public <T> RunnableFuture<T> toSubmit(WithResources withResources, Callable<T> callable)
+        {
+            return super.toSubmit(withLocals(withResources), callable);
+        }
+
+        private static WithResources withLocals(WithResources withResources)
+        {
+            return withResources instanceof ExecutorLocals ? withResources : ExecutorLocals.propagate().and(withResources);
+        }
+    }
+
+}
diff --git a/src/java/org/apache/cassandra/concurrent/ThreadPoolExecutorBase.java b/src/java/org/apache/cassandra/concurrent/ThreadPoolExecutorBase.java
new file mode 100644
index 0000000..b90485a
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ThreadPoolExecutorBase.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.concurrent;
+
+import java.util.List;
+import java.util.concurrent.*;
+
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
+
+/**
+ * This class incorporates some Executor best practices for Cassandra.  Most of the executors in the system
+ * should use or extend {@link ThreadPoolExecutorPlus}, or in rare exceptions this class.
+ *
+ * This class provides some very basic improvements:
+ * <li>We are configured by {@link ThreadPoolExecutorBuilder}
+ * <li>Tasks rejected due to overflow of the queue block the submitting thread rather than throwing {@link RejectedExecutionException}
+ * <li>{@link RunnableFuture} rejected due to executor shutdown will be cancelled
+ * <li>{@link RunnableFuture} removed by {@link #shutdownNow()} will be cancelled
+ *
+ * We also provide a shutdown hook for JMX registration cleanup.
+ */
+public class ThreadPoolExecutorBase extends ThreadPoolExecutor implements ResizableThreadPool
+{
+    public static final RejectedExecutionHandler blockingExecutionHandler = (task, executor) ->
+    {
+        BlockingQueue<Runnable> queue = executor.getQueue();
+        try
+        {
+            while (true)
+            {
+                try
+                {
+                    if (executor.isShutdown())
+                        throw new RejectedExecutionException(executor + " has shut down");
+
+                    if (queue.offer(task, 1, TimeUnit.SECONDS))
+                        break;
+                }
+                catch (InterruptedException e)
+                {
+                    throw new UncheckedInterruptedException(e);
+                }
+            }
+        }
+        catch (Throwable t)
+        {
+            //Give some notification to the caller the task isn't going to run
+            if (task instanceof java.util.concurrent.Future)
+                ((java.util.concurrent.Future<?>) task).cancel(false);
+            throw t;
+        }
+    };
+
+    private Runnable onShutdown;
+
+    // maximumPoolSize is only used when corePoolSize == 0
+    // if keepAliveTime < 0 and unit == null, we forbid core thread timeouts (e.g. single threaded executors by default)
+    public ThreadPoolExecutorBase(ThreadPoolExecutorBuilder<?> builder)
+    {
+        super(builder.coreThreads(), builder.maxThreads(), builder.keepAlive(), builder.keepAliveUnits(), builder.newQueue(), builder.newThreadFactory());
+        allowCoreThreadTimeOut(builder.allowCoreThreadTimeouts());
+
+        // block task submissions until queue has room.
+        // this is fighting TPE's design a bit because TPE rejects if queue.offer reports a full queue.
+        // we'll just override this with a handler that retries until it gets in.  ugly, but effective.
+        // (there is an extensive analysis of the options here at
+        //  http://today.java.net/pub/a/today/2008/10/23/creating-a-notifying-blocking-thread-pool-executor.html)
+        setRejectedExecutionHandler(builder.rejectedExecutionHandler(blockingExecutionHandler));
+    }
+
+    // no RejectedExecutionHandler
+    public ThreadPoolExecutorBase(int threads, int keepAlive, TimeUnit keepAliveUnits, BlockingQueue<Runnable> queue, NamedThreadFactory threadFactory)
+    {
+        super(threads, threads, keepAlive, keepAliveUnits, queue, threadFactory);
+        assert queue.isEmpty() : "Executor initialized with non-empty task queue";
+        allowCoreThreadTimeOut(true);
+    }
+
+    public void onShutdown(Runnable onShutdown)
+    {
+        this.onShutdown = onShutdown;
+    }
+
+    public Runnable onShutdown()
+    {
+        return onShutdown;
+    }
+
+    @Override
+    protected void terminated()
+    {
+        getThreadFactory().close();
+    }
+
+    @Override
+    public void shutdown()
+    {
+        try
+        {
+            super.shutdown();
+        }
+        finally
+        {
+            if (onShutdown != null)
+                onShutdown.run();
+        }
+    }
+
+    @Override
+    public List<Runnable> shutdownNow()
+    {
+        try
+        {
+            List<Runnable> cancelled = super.shutdownNow();
+            for (Runnable c : cancelled)
+            {
+                if (c instanceof java.util.concurrent.Future<?>)
+                    ((java.util.concurrent.Future<?>) c).cancel(true);
+            }
+            return cancelled;
+        }
+        finally
+        {
+            if (onShutdown != null)
+                onShutdown.run();
+        }
+    }
+    
+    @Override
+    public int getActiveTaskCount()
+    {
+        return getActiveCount();
+    }
+
+    @Override
+    public int getPendingTaskCount()
+    {
+        return getQueue().size();
+    }
+
+    public int getCoreThreads()
+    {
+        return getCorePoolSize();
+    }
+
+    public void setCoreThreads(int number)
+    {
+        setCorePoolSize(number);
+    }
+
+    public int getMaximumThreads()
+    {
+        return getMaximumPoolSize();
+    }
+
+    public void setMaximumThreads(int number)
+    {
+        setMaximumPoolSize(number);
+    }
+
+    @Override
+    public NamedThreadFactory getThreadFactory()
+    {
+        return (NamedThreadFactory) super.getThreadFactory();
+    }
+
+    public String toString()
+    {
+        return getThreadFactory().id;
+    }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/ThreadPoolExecutorBuilder.java b/src/java/org/apache/cassandra/concurrent/ThreadPoolExecutorBuilder.java
new file mode 100644
index 0000000..7c8dd93
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ThreadPoolExecutorBuilder.java
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.concurrent;
+
+import java.lang.Thread.UncaughtExceptionHandler;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.RejectedExecutionHandler;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+import org.apache.cassandra.concurrent.NamedThreadFactory.MetaFactory;
+
+import static java.lang.Thread.NORM_PRIORITY;
+import static java.util.concurrent.TimeUnit.MINUTES;
+import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
+
+/**
+ * Configure a {@link ThreadPoolExecutorPlus}, applying Cassandra's best practices by default
+ * <li>Core threads may timeout, and use a default {@link #keepAlive} time in {@link #keepAliveUnits}
+ * <li>Threads share the same {@link ThreadGroup}, which may be configurably a child of a specified {@link ThreadGroup}
+ *     descended from the same parent of the {@link MetaFactory}
+ * <li>By default queues are unbounded in length
+ * <li>The default {@link RejectedExecutionHandler} is implementation dependent, but may be overridden
+ * <li>The default {@link UncaughtExceptionHandler} is inherited from {@link MetaFactory}, which in turn receives it
+ *     from the {@link ExecutorBuilderFactory}
+ */
+class ThreadPoolExecutorBuilder<E extends ExecutorPlus> extends MetaFactory implements ExecutorBuilder<E>
+{
+    static <E extends SequentialExecutorPlus> ExecutorBuilder<E> sequential(Function<ThreadPoolExecutorBuilder<E>, E> constructor, ClassLoader contextClassLoader, ThreadGroup threadGroup, UncaughtExceptionHandler uncaughtExceptionHandler, String name)
+    {
+        ThreadPoolExecutorBuilder<E> result = new ThreadPoolExecutorBuilder<>(constructor, contextClassLoader, threadGroup, uncaughtExceptionHandler, name, 1);
+        result.withKeepAlive();
+        return result;
+    }
+
+    static <E extends SingleThreadExecutorPlus> ExecutorBuilder<E> sequentialJmx(Function<ThreadPoolExecutorBuilder<E>, E> constructor, ClassLoader contextClassLoader, ThreadGroup threadGroup, UncaughtExceptionHandler uncaughtExceptionHandler, String name, String jmxPath)
+    {
+        return new ThreadPoolExecutorJMXAdapter.Builder<>(sequential(constructor, contextClassLoader, threadGroup, uncaughtExceptionHandler, name), jmxPath);
+    }
+
+    static <E extends ExecutorPlus> ExecutorBuilder<E> pooled(Function<ThreadPoolExecutorBuilder<E>, E> constructor, ClassLoader contextClassLoader, ThreadGroup threadGroup, UncaughtExceptionHandler uncaughtExceptionHandler, String name, int threads)
+    {
+        return new ThreadPoolExecutorBuilder<>(constructor, contextClassLoader, threadGroup, uncaughtExceptionHandler, name, threads);
+    }
+
+    static <E extends ThreadPoolExecutorPlus> ExecutorBuilder<E> pooledJmx(Function<ThreadPoolExecutorBuilder<E>, E> constructor, ClassLoader contextClassLoader, ThreadGroup threadGroup, UncaughtExceptionHandler uncaughtExceptionHandler, String name, int threads, String jmxPath)
+    {
+        return new ThreadPoolExecutorJMXAdapter.Builder<>(pooled(constructor, contextClassLoader, threadGroup, uncaughtExceptionHandler, name, threads), jmxPath);
+    }
+
+    private final Function<ThreadPoolExecutorBuilder<E>, E> constructor;
+    private final String name;
+    private final int threads;
+    private int threadPriority = NORM_PRIORITY;
+    private Integer queueLimit;
+
+    private long keepAlive = 1;
+    private TimeUnit keepAliveUnits = MINUTES;
+    private boolean allowCoreThreadTimeouts = true;
+
+    private RejectedExecutionHandler rejectedExecutionHandler = null;
+
+    protected ThreadPoolExecutorBuilder(Function<ThreadPoolExecutorBuilder<E>, E> constructor, ClassLoader contextClassLoader, ThreadGroup overrideThreadGroup, UncaughtExceptionHandler uncaughtExceptionHandler, String name, int threads)
+    {
+        super(contextClassLoader, overrideThreadGroup, uncaughtExceptionHandler);
+        this.constructor = constructor;
+        this.name = name;
+        this.threads = threads;
+    }
+
+    // core and non-core threads will die after this period of inactivity
+    public ThreadPoolExecutorBuilder<E> withKeepAlive(long keepAlive, TimeUnit keepAliveUnits)
+    {
+        this.allowCoreThreadTimeouts = true;
+        this.keepAlive = keepAlive;
+        this.keepAliveUnits = keepAliveUnits;
+        return this;
+    }
+
+    // once started, core threads will never die
+    public ThreadPoolExecutorBuilder<E> withKeepAlive()
+    {
+        this.allowCoreThreadTimeouts = false;
+        return this;
+    }
+
+    public ThreadPoolExecutorBuilder<E> withThreadPriority(int threadPriority)
+    {
+        this.threadPriority = threadPriority;
+        return this;
+    }
+
+    @Override
+    public ExecutorBuilder<E> withThreadGroup(ThreadGroup threadGroup)
+    {
+        ThreadGroup current = this.threadGroup;
+
+        ThreadGroup parent = threadGroup;
+        while (parent != null && parent != current)
+            parent = parent.getParent();
+        if (parent != current)
+            throw new IllegalArgumentException("threadGroup may only be overridden with a child of the default threadGroup");
+
+        this.threadGroup = threadGroup;
+        return this;
+    }
+
+    @Override
+    public ExecutorBuilder<E> withDefaultThreadGroup()
+    {
+        this.threadGroup = null;
+        return this;
+    }
+
+    public ThreadPoolExecutorBuilder<E> withQueueLimit(int queueLimit)
+    {
+        this.queueLimit = queueLimit;
+        return this;
+    }
+
+    public ThreadPoolExecutorBuilder<E> withRejectedExecutionHandler(RejectedExecutionHandler rejectedExecutionHandler)
+    {
+        this.rejectedExecutionHandler = rejectedExecutionHandler;
+        return this;
+    }
+
+    public ThreadPoolExecutorBuilder<E> withUncaughtExceptionHandler(UncaughtExceptionHandler uncaughtExceptionHandler)
+    {
+        this.uncaughtExceptionHandler = uncaughtExceptionHandler;
+        return this;
+    }
+
+    @Override
+    public E build()
+    {
+        return constructor.apply(this);
+    }
+
+    NamedThreadFactory newThreadFactory()
+    {
+        return newThreadFactory(name, threadPriority);
+    }
+
+    BlockingQueue<Runnable> newQueue()
+    {
+        // if our pool can have an infinite number of threads, there is no point having an infinite queue length
+        int size = queueLimit != null
+                ? queueLimit
+                : threads == Integer.MAX_VALUE
+                    ? 0 : Integer.MAX_VALUE;
+        return newBlockingQueue(size);
+    }
+
+    /**
+     * If our queue blocks on/rejects all submissions, we can configure our core pool size to 0,
+     * as new threads will always be created for new work, and core threads timeout at the same
+     * rate as non-core threads.
+     */
+    int coreThreads()
+    {
+        return (queueLimit != null && queueLimit == 0) || threads == Integer.MAX_VALUE ? 0 : threads;
+    }
+
+    int maxThreads()
+    {
+        return threads;
+    }
+
+    RejectedExecutionHandler rejectedExecutionHandler(RejectedExecutionHandler ifNotSet)
+    {
+        return rejectedExecutionHandler == null ? ifNotSet : rejectedExecutionHandler;
+    }
+
+    long keepAlive()
+    {
+        return keepAlive;
+    }
+
+    TimeUnit keepAliveUnits()
+    {
+        return keepAliveUnits;
+    }
+
+    boolean allowCoreThreadTimeouts()
+    {
+        return allowCoreThreadTimeouts;
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/concurrent/ThreadPoolExecutorJMXAdapter.java b/src/java/org/apache/cassandra/concurrent/ThreadPoolExecutorJMXAdapter.java
new file mode 100644
index 0000000..c596d2b
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ThreadPoolExecutorJMXAdapter.java
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.concurrent;
+
+import java.util.concurrent.RejectedExecutionHandler;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.metrics.ThreadPoolMetrics;
+import org.apache.cassandra.utils.MBeanWrapper;
+
+/**
+ * A {@link ThreadPoolExecutorBase} adapter to expose it via JMX.
+ * The executor is not itself modified to maximise code re-use.
+ * Only its rejected execution handler is updated, and a shutdown listener is registered.
+ */
+@VisibleForTesting
+public class ThreadPoolExecutorJMXAdapter implements Runnable, ResizableThreadPoolMXBean
+{
+    /**
+     * A builder wrapper that delegates all methods except {@link Builder#build()}
+     * @param <E>
+     */
+    static class Builder<E extends ThreadPoolExecutorBase> implements ExecutorBuilder<E>
+    {
+        final ExecutorBuilder<E> wrapped;
+        final String jmxPath;
+        Builder(ExecutorBuilder<E> wrapped, String jmxPath)
+        {
+            this.wrapped = wrapped;
+            this.jmxPath = jmxPath;
+        }
+
+        @Override
+        public ExecutorBuilder<E> withKeepAlive(long keepAlive, TimeUnit keepAliveUnits)
+        {
+            wrapped.withKeepAlive(keepAlive, keepAliveUnits);
+            return this;
+        }
+
+        @Override
+        public ExecutorBuilder<E> withKeepAlive()
+        {
+            wrapped.withKeepAlive();
+            return this;
+        }
+
+        @Override
+        public ExecutorBuilder<E> withThreadPriority(int threadPriority)
+        {
+            wrapped.withThreadPriority(threadPriority);
+            return this;
+        }
+
+        @Override
+        public ExecutorBuilder<E> withQueueLimit(int queueLimit)
+        {
+            wrapped.withQueueLimit(queueLimit);
+            return this;
+        }
+
+        @Override
+        public ExecutorBuilder<E> withThreadGroup(ThreadGroup threadGroup)
+        {
+            wrapped.withThreadGroup(threadGroup);
+            return this;
+        }
+
+        @Override
+        public ExecutorBuilder<E> withDefaultThreadGroup()
+        {
+            wrapped.withDefaultThreadGroup();
+            return this;
+        }
+
+        @Override
+        public ExecutorBuilder<E> withRejectedExecutionHandler(RejectedExecutionHandler rejectedExecutionHandler)
+        {
+            wrapped.withRejectedExecutionHandler(rejectedExecutionHandler);
+            return this;
+        }
+
+        @Override
+        public ExecutorBuilder<E> withUncaughtExceptionHandler(Thread.UncaughtExceptionHandler uncaughtExceptionHandler)
+        {
+            wrapped.withUncaughtExceptionHandler(uncaughtExceptionHandler);
+            return this;
+        }
+
+        /**
+         * Invoke {@link ExecutorBuilder#build()} on {@link #wrapped}, and register the resultant
+         * {@link ThreadPoolExecutorBase} with a new {@link ThreadPoolExecutorJMXAdapter}.
+         *
+         * The executor constructed by {@link #wrapped} is returned.
+         */
+        @Override
+        public E build()
+        {
+            E result = wrapped.build();
+            register(jmxPath, result);
+            return result;
+        }
+    }
+
+    public static void register(String jmxPath, ThreadPoolExecutorBase executor)
+    {
+        new ThreadPoolExecutorJMXAdapter(jmxPath, executor);
+    }
+
+    final String mbeanName;
+    final ThreadPoolExecutorBase executor;
+    final ThreadPoolMetrics metrics;
+    boolean released;
+
+    private ThreadPoolExecutorJMXAdapter(String jmxPath, ThreadPoolExecutorBase executor)
+    {
+        this.executor = executor;
+        this.mbeanName = "org.apache.cassandra." + jmxPath + ":type=" + executor.getThreadFactory().id;
+        this.metrics = new ThreadPoolMetrics(executor, jmxPath, executor.getThreadFactory().id).register();
+        executor.setRejectedExecutionHandler(rejectedExecutionHandler(metrics, executor.getRejectedExecutionHandler()));
+        MBeanWrapper.instance.registerMBean(this, mbeanName);
+        executor.onShutdown(this);
+    }
+
+    @Override
+    public synchronized void run()
+    {
+        if (released)
+            return;
+
+        MBeanWrapper.instance.unregisterMBean(mbeanName);
+        metrics.release();
+        released = true;
+    }
+
+    public ThreadPoolMetrics metrics()
+    {
+        return metrics;
+    }
+
+    @Override
+    public int getActiveTaskCount()
+    {
+        return executor.getActiveTaskCount();
+    }
+
+    @Override
+    public int getPendingTaskCount()
+    {
+        return executor.getPendingTaskCount();
+    }
+
+    @Override
+    public int getCoreThreads()
+    {
+        return executor.getCoreThreads();
+    }
+
+    @Override
+    public void setCoreThreads(int number)
+    {
+        executor.setCoreThreads(number);
+    }
+
+    @Override
+    public int getMaximumThreads()
+    {
+        return executor.getMaximumThreads();
+    }
+
+    @Override
+    public void setMaximumThreads(int number)
+    {
+        executor.setMaximumThreads(number);
+    }
+
+    @Override
+    public void setCorePoolSize(int corePoolSize)
+    {
+        executor.setCorePoolSize(corePoolSize);
+    }
+
+    @Override
+    public int getCorePoolSize()
+    {
+        return executor.getCorePoolSize();
+    }
+
+    @Override
+    public void setMaximumPoolSize(int maximumPoolSize)
+    {
+        executor.setMaximumPoolSize(maximumPoolSize);
+    }
+
+    @Override
+    public int getMaximumPoolSize()
+    {
+        return executor.getMaximumPoolSize();
+    }
+
+    @Override
+    public long getCompletedTaskCount()
+    {
+        return executor.getCompletedTaskCount();
+    }
+
+    @Override
+    public int getMaxTasksQueued()
+    {
+        return executor.getMaxTasksQueued();
+    }
+
+    static RejectedExecutionHandler rejectedExecutionHandler(ThreadPoolMetrics metrics, RejectedExecutionHandler wrap)
+    {
+        return (task, executor) ->
+        {
+            metrics.totalBlocked.inc();
+            metrics.currentBlocked.inc();
+            try
+            {
+                wrap.rejectedExecution(task, executor);
+            }
+            finally
+            {
+                metrics.currentBlocked.dec();
+            }
+        };
+    }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/ThreadPoolExecutorPlus.java b/src/java/org/apache/cassandra/concurrent/ThreadPoolExecutorPlus.java
new file mode 100644
index 0000000..ad735d9
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ThreadPoolExecutorPlus.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.concurrent;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.RunnableFuture;
+
+import org.apache.cassandra.utils.WithResources;
+import org.apache.cassandra.utils.concurrent.Future;
+
+/**
+ * This class inherits Executor best practices from {@link ThreadPoolExecutorBase}
+ * and {@link ThreadPoolExecutorBuilder}. Most Cassandra executors should use or extend this.
+ *
+ * This class' addition is to abstract the semantics of task encapsulation to handle
+ * exceptions and {@link ExecutorLocals}. See {@link TaskFactory} for more detail.
+ */
+public class ThreadPoolExecutorPlus extends ThreadPoolExecutorBase implements ExecutorPlus
+{
+    final TaskFactory taskFactory;
+
+    ThreadPoolExecutorPlus(ThreadPoolExecutorBuilder<? extends ThreadPoolExecutorPlus> builder)
+    {
+        this(builder, TaskFactory.standard());
+    }
+
+    ThreadPoolExecutorPlus(ThreadPoolExecutorBuilder<? extends ThreadPoolExecutorPlus> builder, TaskFactory taskFactory)
+    {
+        super(builder);
+        this.taskFactory = taskFactory;
+    }
+
+    private <T extends Runnable> T addTask(T task)
+    {
+        super.execute(task);
+        return task;
+    }
+
+    @Override
+    public void execute(Runnable run)
+    {
+        addTask(taskFactory.toExecute(run));
+    }
+
+    @Override
+    public void execute(WithResources withResources, Runnable run)
+    {
+        addTask(taskFactory.toExecute(withResources, run));
+    }
+
+    @Override
+    public Future<?> submit(Runnable run)
+    {
+        return addTask(taskFactory.toSubmit(run));
+    }
+
+    @Override
+    public <T> Future<T> submit(Runnable run, T result)
+    {
+        return addTask(taskFactory.toSubmit(run, result));
+    }
+
+    @Override
+    public <T> Future<T> submit(Callable<T> call)
+    {
+        return addTask(taskFactory.toSubmit(call));
+    }
+
+    @Override
+    public <T> Future<T> submit(WithResources withResources, Runnable run, T result)
+    {
+        return addTask(taskFactory.toSubmit(withResources, run, result));
+    }
+
+    @Override
+    public Future<?> submit(WithResources withResources, Runnable run)
+    {
+        return addTask(taskFactory.toSubmit(withResources, run));
+    }
+
+    @Override
+    public <T> Future<T> submit(WithResources withResources, Callable<T> call)
+    {
+        return addTask(taskFactory.toSubmit(withResources, call));
+    }
+
+    @Override
+    public boolean inExecutor()
+    {
+        return Thread.currentThread().getThreadGroup() == getThreadFactory().threadGroup;
+    }
+
+    @Override
+    protected <T> RunnableFuture<T> newTaskFor(Runnable runnable, T value)
+    {
+        return taskFactory.toSubmit(runnable, value);
+    }
+
+    @Override
+    protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable)
+    {
+        return taskFactory.toSubmit(callable);
+    }
+
+    @Override
+    public int getMaxTasksQueued()
+    {
+        return getQueue().size();
+    }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/WrappedExecutorPlus.java b/src/java/org/apache/cassandra/concurrent/WrappedExecutorPlus.java
new file mode 100644
index 0000000..7408c33
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/WrappedExecutorPlus.java
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.concurrent;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.cassandra.utils.WithResources;
+import org.apache.cassandra.utils.concurrent.Future;
+
+public class WrappedExecutorPlus implements ExecutorPlus
+{
+    protected final ExecutorPlus executor;
+
+    public WrappedExecutorPlus(ExecutorPlus executor)
+    {
+        this.executor = executor;
+    }
+
+    public void maybeExecuteImmediately(Runnable task)
+    {
+        executor.maybeExecuteImmediately(task);
+    }
+
+    public void execute(WithResources withResources, Runnable task)
+    {
+        executor.execute(withResources, task);
+    }
+
+    @Override
+    public <T> Future<T> submit(WithResources withResources, Callable<T> task)
+    {
+        return executor.submit(withResources, task);
+    }
+
+    @Override
+    public <T> Future<T> submit(WithResources withResources, Runnable task, T result)
+    {
+        return executor.submit(withResources, task, result);
+    }
+
+    @Override
+    public Future<?> submit(WithResources withResources, Runnable task)
+    {
+        return executor.submit(withResources, task);
+    }
+
+    @Override
+    public boolean inExecutor()
+    {
+        return executor.inExecutor();
+    }
+
+    public <T> Future<T> submit(Callable<T> task)
+    {
+        return executor.submit(task);
+    }
+
+    public <T> Future<T> submit(Runnable task, T result)
+    {
+        return executor.submit(task, result);
+    }
+
+    public Future<?> submit(Runnable task)
+    {
+        return executor.submit(task);
+    }
+
+    public int getActiveTaskCount()
+    {
+        return executor.getActiveTaskCount();
+    }
+
+    public long getCompletedTaskCount()
+    {
+        return executor.getCompletedTaskCount();
+    }
+
+    public int getPendingTaskCount()
+    {
+        return executor.getPendingTaskCount();
+    }
+
+    public int getMaxTasksQueued()
+    {
+        return executor.getMaxTasksQueued();
+    }
+
+    public int getCorePoolSize()
+    {
+        return executor.getCorePoolSize();
+    }
+
+    public void setCorePoolSize(int newCorePoolSize)
+    {
+        executor.setCorePoolSize(newCorePoolSize);
+    }
+
+    public int getMaximumPoolSize()
+    {
+        return executor.getMaximumPoolSize();
+    }
+
+    public void setMaximumPoolSize(int newMaximumPoolSize)
+    {
+        executor.setMaximumPoolSize(newMaximumPoolSize);
+    }
+
+    public <T> List<java.util.concurrent.Future<T>> invokeAll(Collection<? extends Callable<T>> tasks) throws InterruptedException
+    {
+        return executor.invokeAll(tasks);
+    }
+
+    public <T> List<java.util.concurrent.Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
+    {
+        return executor.invokeAll(tasks, timeout, unit);
+    }
+
+    public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
+    {
+        return executor.invokeAny(tasks);
+    }
+
+    public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+    {
+        return executor.invokeAny(tasks, timeout, unit);
+    }
+
+    public void shutdown()
+    {
+        executor.shutdown();
+    }
+
+    public List<Runnable> shutdownNow()
+    {
+        return executor.shutdownNow();
+    }
+
+    public boolean isShutdown()
+    {
+        return executor.isShutdown();
+    }
+
+    public boolean isTerminated()
+    {
+        return executor.isTerminated();
+    }
+
+    public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException
+    {
+        return executor.awaitTermination(timeout, unit);
+    }
+
+    public void execute(Runnable task)
+    {
+        executor.execute(task);
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java b/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
index d2bac5f..3200c88 100644
--- a/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
@@ -74,7 +74,7 @@ public final class JavaBasedUDFunction extends UDFunction
 
     private static final AtomicInteger classSequence = new AtomicInteger();
 
-    // use a JVM standard ExecutorService as DebuggableThreadPoolExecutor references internal
+    // use a JVM standard ExecutorService as ExecutorPlus references internal
     // classes, which triggers AccessControlException from the UDF sandbox
     private static final UDFExecutorService executor =
         new UDFExecutorService(new NamedThreadFactory("UserDefinedFunctions",
diff --git a/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDFunction.java b/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDFunction.java
index d7e5eb8..e42fbe9 100644
--- a/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDFunction.java
@@ -94,7 +94,7 @@ final class ScriptBasedUDFunction extends UDFunction
     "org.apache.cassandra.cql3.functions.types.utils"
     };
 
-    // use a JVM standard ExecutorService as DebuggableThreadPoolExecutor references internal
+    // use a JVM standard ExecutorService as ExecutorPlus references internal
     // classes, which triggers AccessControlException from the UDF sandbox
     private static final UDFExecutorService executor =
         new UDFExecutorService(new NamedThreadFactory("UserDefinedScriptFunctions",
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFExecutorService.java b/src/java/org/apache/cassandra/cql3/functions/UDFExecutorService.java
index a6e3a92..3b7631f 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFExecutorService.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFExecutorService.java
@@ -17,37 +17,47 @@
  */
 package org.apache.cassandra.cql3.functions;
 
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
+import org.apache.cassandra.concurrent.ThreadPoolExecutorJMXAdapter;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.concurrent.ThreadPoolExecutorBase;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.cassandra.utils.FBUtilities.getAvailableProcessors;
 import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
 /**
- * Executor service which exposes stats via JMX, but which doesn't reference
- * internal classes in its beforeExecute & afterExecute methods as these are
- * forbidden by the UDF execution sandbox
+ * Executor service which exposes stats via JMX, but which doesn't reference internal classes
+ * as these are forbidden by the UDF execution sandbox.
+ *
+ * TODO: see if we can port to ExecutorPlus to avoid duplication
  */
-final class UDFExecutorService extends JMXEnabledThreadPoolExecutor
+final class UDFExecutorService extends ThreadPoolExecutorBase
 {
-    private static int KEEPALIVE = Integer.getInteger("cassandra.udf_executor_thread_keepalive_ms", 30000);
+    private static final int KEEPALIVE = Integer.getInteger("cassandra.udf_executor_thread_keepalive_ms", 30000);
+
+    public UDFExecutorService(NamedThreadFactory threadFactory, String jmxPath)
+    {
+        super(getAvailableProcessors(), KEEPALIVE, MILLISECONDS, newBlockingQueue(), threadFactory);
+        ThreadPoolExecutorJMXAdapter.register(jmxPath, this);
+    }
+
+    public int getCoreThreads()
+    {
+        return getCorePoolSize();
+    }
 
-    UDFExecutorService(NamedThreadFactory threadFactory, String jmxPath)
+    public void setCoreThreads(int newCorePoolSize)
     {
-        super(getAvailableProcessors(),
-              KEEPALIVE,
-              MILLISECONDS,
-              newBlockingQueue(),
-              threadFactory,
-              jmxPath);
+        setCorePoolSize(newCorePoolSize);
     }
 
-    protected void afterExecute(Runnable r, Throwable t)
+    public int getMaximumThreads()
     {
+        return getMaximumPoolSize();
     }
 
-    protected void beforeExecute(Thread t, Runnable r)
+    public void setMaximumThreads(int maxPoolSize)
     {
+        setMaximumPoolSize(maxPoolSize);
     }
 }
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
index 2b3c7aa..172b998 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@ -32,7 +32,6 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -42,6 +41,7 @@ import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.concurrent.ImmediateExecutor;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
@@ -281,7 +281,7 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
         {
             protected ExecutorService executor()
             {
-                return Executors.newSingleThreadExecutor();
+                return ImmediateExecutor.INSTANCE;
             }
 
             protected Object executeAggregateUserDefined(ProtocolVersion protocolVersion, Object firstParam, List<ByteBuffer> parameters)
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 224236b..e46e8e0 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -27,7 +27,11 @@ import java.nio.file.Files;
 import java.time.Instant;
 import java.util.*;
 import java.util.Objects;
-import java.util.concurrent.*;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Pattern;
@@ -40,6 +44,10 @@ import com.google.common.base.*;
 import com.google.common.base.Throwables;
 import com.google.common.collect.*;
 import com.google.common.util.concurrent.*;
+
+import org.apache.cassandra.utils.concurrent.AsyncPromise;
+import org.apache.cassandra.utils.concurrent.CountDownLatch;
+import org.apache.cassandra.utils.concurrent.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -91,6 +99,7 @@ import org.apache.cassandra.service.snapshot.TableSnapshot;
 import org.apache.cassandra.streaming.TableStreamManager;
 import org.apache.cassandra.utils.*;
 import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.concurrent.Promise;
 import org.apache.cassandra.utils.concurrent.Refs;
 import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 import org.apache.cassandra.utils.memory.MemtableAllocator;
@@ -98,6 +107,9 @@ import org.apache.cassandra.utils.memory.MemtableAllocator;
 import static com.google.common.base.Throwables.propagate;
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
+import static org.apache.cassandra.config.DatabaseDescriptor.getFlushWriters;
+import static org.apache.cassandra.db.commitlog.CommitLog.instance;
 import static org.apache.cassandra.db.commitlog.CommitLogPosition.NONE;
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
@@ -117,32 +129,23 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     are finished. By having flushExecutor size the same size as each of the perDiskflushExecutors we make sure we can
     have that many flushes going at the same time.
     */
-    private static final ThreadPoolExecutor flushExecutor = new JMXEnabledThreadPoolExecutor(DatabaseDescriptor.getFlushWriters(),
-                                                                                             Stage.KEEP_ALIVE_SECONDS,
-                                                                                             TimeUnit.SECONDS,
-                                                                                             new LinkedBlockingQueue<>(),
-                                                                                             new NamedThreadFactory("MemtableFlushWriter"),
-                                                                                             "internal");
+    private static final ExecutorPlus flushExecutor = executorFactory()
+            .withJmxInternal()
+            .pooled("MemtableFlushWriter", getFlushWriters());
+
+    // post-flush executor is single threaded to provide guarantee that any flush Future on a CF will never return until prior flushes have completed
+    private static final ExecutorPlus postFlushExecutor = executorFactory()
+            .withJmxInternal()
+            .sequential("MemtablePostFlush");
+
+    private static final ExecutorPlus reclaimExecutor = executorFactory()
+            .withJmxInternal()
+            .sequential("MemtableReclaimMemory");
 
     private static final PerDiskFlushExecutors perDiskflushExecutors = new PerDiskFlushExecutors(DatabaseDescriptor.getFlushWriters(),
                                                                                                  DatabaseDescriptor.getNonLocalSystemKeyspacesDataFileLocations(),
                                                                                                  DatabaseDescriptor.useSpecificLocationForLocalSystemData());
 
-    // post-flush executor is single threaded to provide guarantee that any flush Future on a CF will never return until prior flushes have completed
-    private static final ThreadPoolExecutor postFlushExecutor = new JMXEnabledThreadPoolExecutor(1,
-                                                                                                 Stage.KEEP_ALIVE_SECONDS,
-                                                                                                 TimeUnit.SECONDS,
-                                                                                                 new LinkedBlockingQueue<>(),
-                                                                                                 new NamedThreadFactory("MemtablePostFlush"),
-                                                                                                 "internal");
-
-    private static final ThreadPoolExecutor reclaimExecutor = new JMXEnabledThreadPoolExecutor(1,
-                                                                                               Stage.KEEP_ALIVE_SECONDS,
-                                                                                               TimeUnit.SECONDS,
-                                                                                               new LinkedBlockingQueue<>(),
-                                                                                               new NamedThreadFactory("MemtableReclaimMemory"),
-                                                                                               "internal");
-
     private static final String[] COUNTER_NAMES = new String[]{"table", "count", "error", "value"};
     private static final String[] COUNTER_DESCS = new String[]
     { "keyspace.tablename",
@@ -833,7 +836,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      *
      * @param memtable
      */
-    public ListenableFuture<CommitLogPosition> switchMemtableIfCurrent(Memtable memtable)
+    public Future<CommitLogPosition> switchMemtableIfCurrent(Memtable memtable)
     {
         synchronized (data)
         {
@@ -851,7 +854,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      * not complete until the Memtable (and all prior Memtables) have been successfully flushed, and the CL
      * marked clean up to the position owned by the Memtable.
      */
-    public ListenableFuture<CommitLogPosition> switchMemtable()
+    public Future<CommitLogPosition> switchMemtable()
     {
         synchronized (data)
         {
@@ -900,7 +903,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      * @return a Future yielding the commit log position that can be guaranteed to have been successfully written
      *         to sstables for this table once the future completes
      */
-    public ListenableFuture<CommitLogPosition> forceFlush()
+    public Future<CommitLogPosition> forceFlush()
     {
         synchronized (data)
         {
@@ -919,7 +922,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      * @return a Future yielding the commit log position that can be guaranteed to have been successfully written
      *         to sstables for this table once the future completes
      */
-    public ListenableFuture<?> forceFlush(CommitLogPosition flushIfDirtyBefore)
+    public Future<?> forceFlush(CommitLogPosition flushIfDirtyBefore)
     {
         // we don't loop through the remaining memtables since here we only care about commit log dirtiness
         // and this does not vary between a table and its table-backed indexes
@@ -933,17 +936,15 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      * @return a Future yielding the commit log position that can be guaranteed to have been successfully written
      *         to sstables for this table once the future completes
      */
-    private ListenableFuture<CommitLogPosition> waitForFlushes()
+    private Future<CommitLogPosition> waitForFlushes()
     {
         // we grab the current memtable; once any preceding memtables have flushed, we know its
         // commitLogLowerBound has been set (as this it is set with the upper bound of the preceding memtable)
         final Memtable current = data.getView().getCurrentMemtable();
-        ListenableFutureTask<CommitLogPosition> task = ListenableFutureTask.create(() -> {
+        return postFlushExecutor.submit(() -> {
             logger.debug("forceFlush requested but everything is clean in {}", name);
             return current.getCommitLogLowerBound();
         });
-        postFlushExecutor.execute(task);
-        return task;
     }
 
     public CommitLogPosition forceBlockingFlush()
@@ -957,7 +958,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      */
     private final class PostFlush implements Callable<CommitLogPosition>
     {
-        final org.apache.cassandra.utils.concurrent.CountDownLatch latch = newCountDownLatch(1);
+        final CountDownLatch latch = newCountDownLatch(1);
         final List<Memtable> memtables;
         volatile Throwable flushFailure = null;
 
@@ -1009,7 +1010,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         final OpOrder.Barrier writeBarrier;
         final List<Memtable> memtables = new ArrayList<>();
-        final ListenableFutureTask<CommitLogPosition> postFlushTask;
+        final FutureTask<CommitLogPosition> postFlushTask;
         final PostFlush postFlush;
         final boolean truncate;
 
@@ -1054,7 +1055,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             // commit log segment position have also completed, i.e. the memtables are done and ready to flush
             writeBarrier.issue();
             postFlush = new PostFlush(memtables);
-            postFlushTask = ListenableFutureTask.create(postFlush);
+            postFlushTask = new FutureTask<>(postFlush);
         }
 
         public void run()
@@ -1127,7 +1128,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 {
                     // flush the memtable
                     flushRunnables = memtable.flushRunnables(txn);
-                    ExecutorService[] executors = perDiskflushExecutors.getExecutorsFor(keyspace.getName(), name);
+                    ExecutorPlus[] executors = perDiskflushExecutors.getExecutorsFor(keyspace.getName(), name);
 
                     for (int i = 0; i < flushRunnables.size(); i++)
                         futures.add(executors[i].submit(flushRunnables.get(i)));
@@ -1249,7 +1250,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      * Finds the largest memtable, as a percentage of *either* on- or off-heap memory limits, and immediately
      * queues it for flushing. If the memtable selected is flushed before this completes, no work is done.
      */
-    public static CompletableFuture<Boolean> flushLargestMemtable()
+    public static Future<Boolean> flushLargestMemtable()
     {
         float largestRatio = 0f;
         Memtable largest = null;
@@ -1285,7 +1286,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             liveOffHeap += offHeap;
         }
 
-        CompletableFuture<Boolean> returnFuture = new CompletableFuture<>();
+        Promise<Boolean> returnFuture = new AsyncPromise<>();
 
         if (largest != null)
         {
@@ -1299,24 +1300,24 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                          largest.cfs, ratio(usedOnHeap, usedOffHeap), ratio(liveOnHeap, liveOffHeap),
                          ratio(flushingOnHeap, flushingOffHeap), ratio(thisOnHeap, thisOffHeap));
 
-            ListenableFuture<CommitLogPosition> flushFuture = largest.cfs.switchMemtableIfCurrent(largest);
+            Future<CommitLogPosition> flushFuture = largest.cfs.switchMemtableIfCurrent(largest);
             flushFuture.addListener(() -> {
                 try
                 {
                     flushFuture.get();
-                    returnFuture.complete(true);
+                    returnFuture.trySuccess(true);
                 }
                 catch (Throwable t)
                 {
-                    returnFuture.completeExceptionally(t);
+                    returnFuture.tryFailure(t);
                 }
-            }, MoreExecutors.directExecutor());
+            });
         }
         else
         {
             logger.debug("Flushing of largest memtable, not done, no memtable found");
 
-            returnFuture.complete(false);
+            returnFuture.trySuccess(false);
         }
 
         return returnFuture;
@@ -2355,7 +2356,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             }
         };
 
-        runWithCompactionsDisabled(Executors.callable(truncateRunnable), true, true);
+        runWithCompactionsDisabled(FutureTask.callable(truncateRunnable), true, true);
 
         viewManager.build();
 
@@ -2897,12 +2898,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         /**
          * The flush executors for non local system keyspaces.
          */
-        private final ExecutorService[] nonLocalSystemflushExecutors;
+        private final ExecutorPlus[] nonLocalSystemflushExecutors;
 
         /**
          * The flush executors for the local system keyspaces.
          */
-        private final ExecutorService[] localSystemDiskFlushExecutors;
+        private final ExecutorPlus[] localSystemDiskFlushExecutors;
 
         /**
          * {@code true} if local system keyspaces are stored in their own directory and use an extra flush executor,
@@ -2914,32 +2915,26 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                                      String[] locationsForNonSystemKeyspaces,
                                      boolean useSpecificLocationForSystemKeyspaces)
         {
-            ExecutorService[] flushExecutors = createPerDiskFlushWriters(locationsForNonSystemKeyspaces.length, flushWriters);
+            ExecutorPlus[] flushExecutors = createPerDiskFlushWriters(locationsForNonSystemKeyspaces.length, flushWriters);
             nonLocalSystemflushExecutors = flushExecutors;
             useSpecificExecutorForSystemKeyspaces = useSpecificLocationForSystemKeyspaces;
-            localSystemDiskFlushExecutors = useSpecificLocationForSystemKeyspaces ? new ExecutorService[] {newThreadPool("LocalSystemKeyspacesDiskMemtableFlushWriter", flushWriters)}
-                                                                                  : new ExecutorService[] {flushExecutors[0]};
+            localSystemDiskFlushExecutors = useSpecificLocationForSystemKeyspaces ? new ExecutorPlus[] {newThreadPool("LocalSystemKeyspacesDiskMemtableFlushWriter", flushWriters)}
+                                                                                  : new ExecutorPlus[] {flushExecutors[0]};
         }
 
-        private static ExecutorService[] createPerDiskFlushWriters(int numberOfExecutors, int flushWriters)
+        private static ExecutorPlus[] createPerDiskFlushWriters(int numberOfExecutors, int flushWriters)
         {
-            ExecutorService[] flushExecutors = new ExecutorService[numberOfExecutors];
-
-            for (int i = 0; i < numberOfExecutors; i++)
+            ExecutorPlus[] flushExecutors = new ExecutorPlus[numberOfExecutors];
+            for (int i = 0; i < DatabaseDescriptor.getAllDataFileLocations().length; i++)
             {
-                flushExecutors[i] = newThreadPool("PerDiskMemtableFlushWriter_" + i, flushWriters);
+                flushExecutors[i] = newThreadPool("PerDiskMemtableFlushWriter_"+i, flushWriters);
             }
             return flushExecutors;
         }
 
-        private static JMXEnabledThreadPoolExecutor newThreadPool(String poolName, int size)
+        private static ExecutorPlus newThreadPool(String poolName, int size)
         {
-            return new JMXEnabledThreadPoolExecutor(size,
-                                                    Stage.KEEP_ALIVE_SECONDS,
-                                                    TimeUnit.SECONDS,
-                                                    new LinkedBlockingQueue<>(),
-                                                    new NamedThreadFactory(poolName),
-                                                    "internal");
+            return executorFactory().withJmxInternal().pooled(poolName, size);
         }
 
         /**
@@ -2949,7 +2944,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
          * @param tableName the table name
          * @return the flush executors that should be used for flushing the memtables of the specified keyspace.
          */
-        public ExecutorService[] getExecutorsFor(String keyspaceName, String tableName)
+        public ExecutorPlus[] getExecutorsFor(String keyspaceName, String tableName)
         {
             return Directories.isStoredInLocalSystemKeyspacesDataLocation(keyspaceName, tableName) ? localSystemDiskFlushExecutors
                                                                                                    : nonLocalSystemflushExecutors;
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 39cb746..2a99690 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -28,10 +28,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
-import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Lock;
 import java.util.stream.Stream;
@@ -39,6 +37,9 @@ import java.util.stream.Stream;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.RateLimiter;
+
+import org.apache.cassandra.utils.concurrent.AsyncPromise;
+import org.apache.cassandra.utils.concurrent.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -73,6 +74,7 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
+import org.apache.cassandra.utils.concurrent.Promise;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
@@ -482,15 +484,15 @@ public class Keyspace
         }
     }
 
-    public CompletableFuture<?> applyFuture(Mutation mutation, boolean writeCommitLog, boolean updateIndexes)
+    public Future<?> applyFuture(Mutation mutation, boolean writeCommitLog, boolean updateIndexes)
     {
-        return applyInternal(mutation, writeCommitLog, updateIndexes, true, true, new CompletableFuture<>());
+        return applyInternal(mutation, writeCommitLog, updateIndexes, true, true, new AsyncPromise<>());
     }
 
-    public CompletableFuture<?> applyFuture(Mutation mutation, boolean writeCommitLog, boolean updateIndexes, boolean isDroppable,
+    public Future<?> applyFuture(Mutation mutation, boolean writeCommitLog, boolean updateIndexes, boolean isDroppable,
                                             boolean isDeferrable)
     {
-        return applyInternal(mutation, writeCommitLog, updateIndexes, isDroppable, isDeferrable, new CompletableFuture<>());
+        return applyInternal(mutation, writeCommitLog, updateIndexes, isDroppable, isDeferrable, new AsyncPromise<>());
     }
 
     public void apply(Mutation mutation, boolean writeCommitLog, boolean updateIndexes)
@@ -533,12 +535,12 @@ public class Keyspace
      * @param isDroppable    true if this should throw WriteTimeoutException if it does not acquire lock within write_request_timeout_in_ms
      * @param isDeferrable   true if caller is not waiting for future to complete, so that future may be deferred
      */
-    private CompletableFuture<?> applyInternal(final Mutation mutation,
+    private Future<?> applyInternal(final Mutation mutation,
                                                final boolean makeDurable,
                                                boolean updateIndexes,
                                                boolean isDroppable,
                                                boolean isDeferrable,
-                                               CompletableFuture<?> future)
+                                               Promise<?> future)
     {
         if (TEST_FAIL_WRITES && metadata.name.equals(TEST_FAIL_WRITES_KS))
             throw new RuntimeException("Testing write failures");
@@ -582,7 +584,7 @@ public class Keyspace
                             Tracing.trace("Could not acquire MV lock");
                             if (future != null)
                             {
-                                future.completeExceptionally(new WriteTimeoutException(WriteType.VIEW, ConsistencyLevel.LOCAL_ONE, 0, 1));
+                                future.tryFailure(new WriteTimeoutException(WriteType.VIEW, ConsistencyLevel.LOCAL_ONE, 0, 1));
                                 return future;
                             }
                             else
@@ -595,9 +597,8 @@ public class Keyspace
 
                             // This view update can't happen right now. so rather than keep this thread busy
                             // we will re-apply ourself to the queue and try again later
-                            final CompletableFuture<?> mark = future;
                             Stage.MUTATION.execute(() ->
-                                                   applyInternal(mutation, makeDurable, true, isDroppable, true, mark)
+                                                   applyInternal(mutation, makeDurable, true, isDroppable, true, future)
                             );
                             return future;
                         }
@@ -675,7 +676,7 @@ public class Keyspace
             }
 
             if (future != null) {
-                future.complete(null);
+                future.trySuccess(null);
             }
             return future;
         }
diff --git a/src/java/org/apache/cassandra/db/Mutation.java b/src/java/org/apache/cassandra/db/Mutation.java
index 8a1ffc1..a30b567 100644
--- a/src/java/org/apache/cassandra/db/Mutation.java
+++ b/src/java/org/apache/cassandra/db/Mutation.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db;
 
 import java.io.IOException;
 import java.util.*;
-import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -38,6 +37,7 @@ import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.concurrent.Future;
 
 import static org.apache.cassandra.net.MessagingService.VERSION_30;
 import static org.apache.cassandra.net.MessagingService.VERSION_3014;
@@ -204,7 +204,7 @@ public class Mutation implements IMutation
         return new Mutation(ks, key, modifications.build(), approxTime.now());
     }
 
-    public CompletableFuture<?> applyFuture()
+    public Future<?> applyFuture()
     {
         Keyspace ks = Keyspace.open(keyspaceName);
         return ks.applyFuture(this, Keyspace.open(keyspaceName).getMetadata().params.durableWrites, true);
diff --git a/src/java/org/apache/cassandra/db/MutationVerbHandler.java b/src/java/org/apache/cassandra/db/MutationVerbHandler.java
index 1d4f868..9c0335e 100644
--- a/src/java/org/apache/cassandra/db/MutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/MutationVerbHandler.java
@@ -55,10 +55,7 @@ public class MutationVerbHandler implements IVerbHandler<Mutation>
 
         try
         {
-            message.payload.applyFuture().thenAccept(o -> respond(message, respondToAddress)).exceptionally(wto -> {
-                failed();
-                return null;
-            });
+            message.payload.applyFuture().addCallback(o -> respond(message, respondToAddress), wto -> failed());
         }
         catch (WriteTimeoutException wto)
         {
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index f7946b9..279c99e 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -37,7 +37,6 @@ import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.SetMultimap;
 import com.google.common.collect.Sets;
 import com.google.common.io.ByteStreams;
-import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -65,6 +64,7 @@ import org.apache.cassandra.service.paxos.PaxosState;
 import org.apache.cassandra.streaming.StreamOperation;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.concurrent.Future;
 
 import static java.lang.String.format;
 import static java.util.Collections.emptyMap;
@@ -828,7 +828,7 @@ public final class SystemKeyspace
     {
         if (!DatabaseDescriptor.isUnsafeSystem())
         {
-            List<ListenableFuture<CommitLogPosition>> futures = new ArrayList<>();
+            List<Future<CommitLogPosition>> futures = new ArrayList<>();
 
             for (String cfname : cfnames)
             {
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
index edba125..2f5983b 100755
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
@@ -20,30 +20,33 @@ package org.apache.cassandra.db.commitlog;
 import java.io.File;
 import java.io.IOException;
 import java.util.*;
-import java.util.concurrent.*;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.BooleanSupplier;
+import java.util.function.Consumer;
 
-import com.codahale.metrics.Timer.Context;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.codahale.metrics.Timer.Context;
 import net.nicoulaj.compilecommand.annotations.DontInline;
-import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.concurrent.Interruptible;
+import org.apache.cassandra.concurrent.Interruptible.TerminateException;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.SimpleCachedBufferPool;
 import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.db.*;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.utils.*;
-import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
-import org.apache.cassandra.utils.concurrent.WaitQueue;
+import org.apache.cassandra.utils.concurrent.*;
 
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation;
 import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue;
 
@@ -88,10 +91,9 @@ public abstract class AbstractCommitLogSegmentManager
     private final AtomicLong size = new AtomicLong();
 
     @VisibleForTesting
-    Thread managerThread;
+    Interruptible executor;
     protected final CommitLog commitLog;
-    private volatile boolean shutdown;
-    private final BooleanSupplier managerThreadWaitCondition = () -> (availableSegment == null && !atSegmentBufferLimit()) || shutdown;
+    private final BooleanSupplier managerThreadWaitCondition = () -> (availableSegment == null && !atSegmentBufferLimit());
     private final WaitQueue managerThreadWaitQueue = newWaitQueue();
 
     private volatile SimpleCachedBufferPool bufferPool;
@@ -104,52 +106,55 @@ public abstract class AbstractCommitLogSegmentManager
 
     void start()
     {
+        // used for synchronization to prevent thread interrupts while performing IO operations
+        final Object monitor = new Object();
         // The run loop for the manager thread
-        Runnable runnable = new WrappedRunnable()
-        {
-            public void runMayThrow() throws Exception
+        Interruptible.Task runnable = state -> {
+
+            try
             {
-                while (!shutdown)
+                switch (state)
                 {
-                    try
-                    {
+                    case SHUTTING_DOWN:
+                        // If shutdown() started and finished during segment creation, we are now left with a
+                        // segment that no one will consume. Discard it.
+                        discardAvailableSegment();
+                        return;
+
+                    case NORMAL:
                         assert availableSegment == null;
-                        logger.trace("No segments in reserve; creating a fresh one");
-                        availableSegment = createSegment();
-                        if (shutdown)
+
+                        synchronized (monitor)
                         {
-                            // If shutdown() started and finished during segment creation, we are now left with a
-                            // segment that no one will consume. Discard it.
-                            discardAvailableSegment();
-                            return;
-                        }
+                            logger.trace("No segments in reserve; creating a fresh one");
+                            availableSegment = createSegment();
 
-                        segmentPrepared.signalAll();
-                        Thread.yield();
-
-                        if (availableSegment == null && !atSegmentBufferLimit())
-                            // Writing threads need another segment now.
-                            continue;
-
-                        // Writing threads are not waiting for new segments, we can spend time on other tasks.
-                        // flush old Cfs if we're full
-                        maybeFlushToReclaim();
-                    }
-                    catch (Throwable t)
-                    {
-                        if (!CommitLog.handleCommitError("Failed managing commit log segments", t))
-                            return;
-                        // sleep some arbitrary period to avoid spamming CL
-                        Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-                        // If we offered a segment, wait for it to be taken before reentering the loop.
-                        // There could be a new segment in next not offered, but only on failure to discard it while
-                        // shutting down-- nothing more can or needs to be done in that case.
-                    }
-
-                    WaitQueue.waitOnCondition(managerThreadWaitCondition, managerThreadWaitQueue);
+                            segmentPrepared.signalAll();
+                            Thread.yield();
+
+                            if (availableSegment == null && !atSegmentBufferLimit())
+                                // Writing threads need another segment now.
+                                return;
+
+                            // Writing threads are not waiting for new segments, we can spend time on other tasks.
+                            // flush old Cfs if we're full
+                            maybeFlushToReclaim();
+                        }
                 }
             }
+            catch (Throwable t)
+            {
+                if (!CommitLog.handleCommitError("Failed managing commit log segments", t))
+                    throw new TerminateException();
+
+                // sleep some arbitrary period to avoid spamming CL
+                Thread.sleep(TimeUnit.SECONDS.toMillis(1L));
+
+                // If we offered a segment, wait for it to be taken before reentering the loop.
+                // There could be a new segment in next not offered, but only on failure to discard it while
+                // shutting down-- nothing more can or needs to be done in that case.
+            }
+            WaitQueue.waitOnCondition(managerThreadWaitCondition, managerThreadWaitQueue);
         };
 
         // For encrypted segments we want to keep the compression buffers on-heap as we need those bytes for encryption,
@@ -162,15 +167,23 @@ public abstract class AbstractCommitLogSegmentManager
                                                      DatabaseDescriptor.getCommitLogSegmentSize(),
                                                      bufferType);
 
-        shutdown = false;
-
-        managerThread = NamedThreadFactory.createThread(runnable, "COMMIT-LOG-ALLOCATOR");
-        managerThread.start();
+        Consumer<Thread> interruptHandler = interruptHandler(monitor);
+        executor = executorFactory().infiniteLoop("COMMIT-LOG-ALLOCATOR", runnable, true, interruptHandler);
 
         // for simplicity, ensure the first segment is allocated before continuing
         advanceAllocatingFrom(null);
     }
 
+    private Consumer<Thread> interruptHandler(final Object monitor)
+    {
+        return thread -> {
+            synchronized (monitor)
+            {
+                thread.interrupt();
+            }
+        };
+    }
+
     private boolean atSegmentBufferLimit()
     {
         return CommitLogSegment.usesBufferPool(commitLog) && bufferPool.atLimit();
@@ -387,11 +400,11 @@ public abstract class AbstractCommitLogSegmentManager
     private Future<?> flushDataFrom(List<CommitLogSegment> segments, boolean force)
     {
         if (segments.isEmpty())
-            return Futures.immediateFuture(null);
+            return ImmediateFuture.success(null);
         final CommitLogPosition maxCommitLogPosition = segments.get(segments.size() - 1).getCurrentCommitLogPosition();
 
         // a map of CfId -> forceFlush() to ensure we only queue one flush per cf
-        final Map<TableId, ListenableFuture<?>> flushes = new LinkedHashMap<>();
+        final Map<TableId, Future<?>> flushes = new LinkedHashMap<>();
 
         for (CommitLogSegment segment : segments)
         {
@@ -415,7 +428,7 @@ public abstract class AbstractCommitLogSegmentManager
             }
         }
 
-        return Futures.allAsList(flushes.values());
+        return FutureCombiner.allOf(flushes.values());
     }
 
     /**
@@ -477,9 +490,7 @@ public abstract class AbstractCommitLogSegmentManager
      */
     public void shutdown()
     {
-        assert !shutdown;
-        shutdown = true;
-
+        executor.shutdownNow();
         // Release the management thread and delete prepared segment.
         // Do not block as another thread may claim the segment (this can happen during unit test initialization).
         discardAvailableSegment();
@@ -488,7 +499,7 @@ public abstract class AbstractCommitLogSegmentManager
 
     private void discardAvailableSegment()
     {
-        CommitLogSegment next = null;
+        CommitLogSegment next;
         synchronized (this)
         {
             next = availableSegment;
@@ -503,12 +514,7 @@ public abstract class AbstractCommitLogSegmentManager
      */
     public void awaitTermination() throws InterruptedException
     {
-        if (managerThread != null)
-        {
-            managerThread.join();
-            managerThread = null;
-        }
-
+        executor.awaitTermination(1L, TimeUnit.MINUTES);
         for (CommitLogSegment segment : activeSegments)
             segment.close();
 
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
index 07c505b..9d030d7 100644
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
@@ -17,24 +17,31 @@
  */
 package org.apache.cassandra.db.commitlog;
 
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.LockSupport;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.concurrent.Interruptible;
+import org.apache.cassandra.concurrent.Interruptible.TerminateException;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation;
 import org.apache.cassandra.utils.MonotonicClock;
 import org.apache.cassandra.utils.NoSpamLogger;
+import org.apache.cassandra.utils.concurrent.Semaphore;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
-import static com.codahale.metrics.Timer.*;
+import static com.codahale.metrics.Timer.Context;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.MINUTES;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
+import static org.apache.cassandra.concurrent.Interruptible.State.NORMAL;
+import static org.apache.cassandra.concurrent.Interruptible.State.SHUTTING_DOWN;
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+import static org.apache.cassandra.utils.concurrent.Semaphore.newSemaphore;
 import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue;
 
 public abstract class AbstractCommitLogService
@@ -45,8 +52,7 @@ public abstract class AbstractCommitLogService
      */
     static final long DEFAULT_MARKER_INTERVAL_MILLIS = 100;
 
-    private volatile Thread thread;
-    private volatile boolean shutdown = false;
+    private volatile Interruptible executor;
 
     // all Allocations written before this time will be synced
     protected volatile long lastSyncedAt = currentTimeMillis();
@@ -57,6 +63,7 @@ public abstract class AbstractCommitLogService
 
     // signal that writers can wait on to be notified of a completed sync
     protected final WaitQueue syncComplete = newWaitQueue();
+    protected final Semaphore haveWork = newSemaphore(1);
 
     final CommitLog commitLog;
     private final String name;
@@ -106,7 +113,11 @@ public abstract class AbstractCommitLogService
         this.name = name;
 
         final long markerIntervalMillis;
-        if (markHeadersFaster && syncIntervalMillis > DEFAULT_MARKER_INTERVAL_MILLIS)
+        if (syncIntervalMillis < 0)
+        {
+            markerIntervalMillis = -1;
+        }
+        else if (markHeadersFaster && syncIntervalMillis > DEFAULT_MARKER_INTERVAL_MILLIS)
         {
             markerIntervalMillis = DEFAULT_MARKER_INTERVAL_MILLIS;
             long modulo = syncIntervalMillis % markerIntervalMillis;
@@ -118,29 +129,28 @@ public abstract class AbstractCommitLogService
                 if (modulo >= markerIntervalMillis / 2)
                     syncIntervalMillis += markerIntervalMillis;
             }
+            assert syncIntervalMillis % markerIntervalMillis == 0;
             logger.debug("Will update the commitlog markers every {}ms and flush every {}ms", markerIntervalMillis, syncIntervalMillis);
         }
         else
         {
             markerIntervalMillis = syncIntervalMillis;
         }
-        assert syncIntervalMillis % markerIntervalMillis == 0;
-        this.markerIntervalNanos = TimeUnit.NANOSECONDS.convert(markerIntervalMillis, TimeUnit.MILLISECONDS);
-        this.syncIntervalNanos = TimeUnit.NANOSECONDS.convert(syncIntervalMillis, TimeUnit.MILLISECONDS);
+        this.markerIntervalNanos = NANOSECONDS.convert(markerIntervalMillis, MILLISECONDS);
+        this.syncIntervalNanos = NANOSECONDS.convert(syncIntervalMillis, MILLISECONDS);
     }
 
     // Separated into individual method to ensure relevant objects are constructed before this is started.
     void start()
     {
-        if (syncIntervalNanos < 1)
+        if (syncIntervalNanos < 1 && !(this instanceof BatchCommitLogService)) // permit indefinite waiting with batch, as perfectly sensible
             throw new IllegalArgumentException(String.format("Commit log flush interval must be positive: %fms",
                                                              syncIntervalNanos * 1e-6));
-        shutdown = false;
-        thread = NamedThreadFactory.createThread(new SyncRunnable(MonotonicClock.preciseTime), name);
-        thread.start();
+
+        executor = executorFactory().infiniteLoop(name, new SyncRunnable(MonotonicClock.preciseTime), true);
     }
 
-    class SyncRunnable implements Runnable
+    class SyncRunnable implements Interruptible.Task
     {
         private final MonotonicClock clock;
         private long firstLagAt = 0;
@@ -154,25 +164,13 @@ public abstract class AbstractCommitLogService
             this.clock = clock;
         }
 
-        public void run()
-        {
-            while (true)
-            {
-                if (!sync())
-                    break;
-            }
-        }
-
-        boolean sync()
+        public void run(Interruptible.State state) throws InterruptedException
         {
-            // always run once after shutdown signalled
-            boolean shutdownRequested = shutdown;
-
             try
             {
                 // sync and signal
                 long pollStarted = clock.now();
-                boolean flushToDisk = lastSyncedAt + syncIntervalNanos <= pollStarted || shutdownRequested || syncRequested;
+                boolean flushToDisk = lastSyncedAt + syncIntervalNanos <= pollStarted || state != NORMAL || syncRequested;
                 if (flushToDisk)
                 {
                     // in this branch, we want to flush the commit log to disk
@@ -188,27 +186,31 @@ public abstract class AbstractCommitLogService
                     commitLog.sync(false);
                 }
 
-                long now = clock.now();
-                if (flushToDisk)
-                    maybeLogFlushLag(pollStarted, now);
+                if (state == SHUTTING_DOWN)
+                    return;
 
-                if (shutdownRequested)
-                    return false;
+                if (markerIntervalNanos <= 0)
+                {
+                    haveWork.acquire(1);
+                }
+                else
+                {
+                    long now = clock.now();
+                    if (flushToDisk)
+                        maybeLogFlushLag(pollStarted, now);
 
-                long wakeUpAt = pollStarted + markerIntervalNanos;
-                if (wakeUpAt > now)
-                    LockSupport.parkNanos(wakeUpAt - now);
+                    long wakeUpAt = pollStarted + markerIntervalNanos;
+                    if (wakeUpAt > now)
+                        haveWork.tryAcquireUntil(1, wakeUpAt);
+                }
             }
             catch (Throwable t)
             {
                 if (!CommitLog.handleCommitError("Failed to persist commits to disk", t))
-                    return false;
-
-                // sleep for full poll-interval after an error, so we don't spam the log file
-                LockSupport.parkNanos(markerIntervalNanos);
+                    throw new TerminateException();
+                else // sleep for full poll-interval after an error, so we don't spam the log file
+                    haveWork.tryAcquire(1, markerIntervalNanos, NANOSECONDS);
             }
-
-            return true;
         }
 
         /**
@@ -242,7 +244,7 @@ public abstract class AbstractCommitLogService
                 boolean logged = NoSpamLogger.log(logger,
                                                   NoSpamLogger.Level.WARN,
                                                   5,
-                                                  TimeUnit.MINUTES,
+                                                  MINUTES,
                                                   "Out of {} commit log syncs over the past {}s with average duration of {}ms, {} have exceeded the configured commit interval by an average of {}ms",
                                                   syncCount,
                                                   String.format("%.2f", (now - firstLagAt) * 1e-9d),
@@ -278,14 +280,14 @@ public abstract class AbstractCommitLogService
      */
     void requestExtraSync()
     {
+        // note: cannot simply invoke executor.interrupt() as some filesystems don't like it (jimfs, at least)
         syncRequested = true;
-        LockSupport.unpark(thread);
+        haveWork.release(1);
     }
 
     public void shutdown()
     {
-        shutdown = true;
-        requestExtraSync();
+        executor.shutdown();
     }
 
     /**
@@ -316,8 +318,7 @@ public abstract class AbstractCommitLogService
 
     public void awaitTermination() throws InterruptedException
     {
-        if (thread != null)
-            thread.join();
+        executor.awaitTermination(5L, MINUTES);
     }
 
     public long getCompletedTasks()
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
index e1d0874..f94b269 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
@@ -32,7 +32,6 @@ import java.util.concurrent.*;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.CompressionParams;
@@ -45,6 +44,8 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Strings;
 
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
+
 public class CommitLogArchiver
 {
     private static final Logger logger = LoggerFactory.getLogger(CommitLogArchiver.class);
@@ -75,7 +76,11 @@ public class CommitLogArchiver
         this.restoreDirectories = restoreDirectories;
         this.restorePointInTime = restorePointInTime;
         this.precision = precision;
-        executor = !Strings.isNullOrEmpty(archiveCommand) ? new JMXEnabledThreadPoolExecutor("CommitLogArchiver") : null;
+        executor = !Strings.isNullOrEmpty(archiveCommand)
+                ? executorFactory()
+                    .withJmxInternal()
+                    .sequential("CommitLogArchiver")
+                : null;
     }
 
     public static CommitLogArchiver disabled()
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index 39777ec..501ee02 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@ -21,7 +21,6 @@ package org.apache.cassandra.db.commitlog;
 import java.io.File;
 import java.io.IOException;
 import java.util.*;
-import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -32,6 +31,8 @@ import com.google.common.collect.Multimap;
 import com.google.common.collect.Ordering;
 
 import org.apache.commons.lang3.StringUtils;
+
+import org.apache.cassandra.utils.concurrent.Future;
 import org.cliffc.high_scale_lib.NonBlockingHashSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index 4a68e8b..86442d8 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -514,7 +514,7 @@ public abstract class CommitLogSegment
         {
             WaitQueue.Signal signal = syncComplete.register();
             if (lastSyncedOffset < position)
-                signal.awaitUninterruptibly();
+                signal.awaitThrowUncheckedOnInterrupt();
             else
                 signal.cancel();
         }
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
index 66c8a39..6561137 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
@@ -39,6 +39,8 @@ import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.DirectorySizeCalculator;
 import org.apache.cassandra.utils.NoSpamLogger;
 
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
+
 public class CommitLogSegmentManagerCDC extends AbstractCommitLogSegmentManager
 {
     static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentManagerCDC.class);
@@ -208,7 +210,11 @@ public class CommitLogSegmentManagerCDC extends AbstractCommitLogSegmentManager
         public void start()
         {
             size = 0;
-            cdcSizeCalculationExecutor = new ThreadPoolExecutor(1, 1, 1000, TimeUnit.SECONDS, new SynchronousQueue<>(), new ThreadPoolExecutor.DiscardPolicy());
+            cdcSizeCalculationExecutor = executorFactory().configureSequential("CDCSizeCalculationExecutor")
+                                                          .withRejectedExecutionHandler(new ThreadPoolExecutor.DiscardPolicy())
+                                                          .withQueueLimit(0)
+                                                          .withKeepAlive(1000, TimeUnit.SECONDS)
+                                                          .build();
         }
 
         /**
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java b/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java
index a533f95..e8250b4 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java
@@ -41,10 +41,13 @@ import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.node.ArrayNode;
 import com.fasterxml.jackson.databind.node.JsonNodeFactory;
 import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.ExecutorUtils;
 import org.apache.cassandra.utils.NoSpamLogger;
 
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
 
 public class CompactionLogger
@@ -105,7 +108,7 @@ public class CompactionLogger
 
     private static final JsonNodeFactory json = JsonNodeFactory.instance;
     private static final Logger logger = LoggerFactory.getLogger(CompactionLogger.class);
-    private static final Writer serializer = new CompactionLogSerializer();
+    private static final CompactionLogSerializer serializer = new CompactionLogSerializer();
     private final WeakReference<ColumnFamilyStore> cfsRef;
     private final WeakReference<CompactionStrategyManager> csmRef;
     private final AtomicInteger identifier = new AtomicInteger(0);
@@ -297,7 +300,7 @@ public class CompactionLogger
     private static class CompactionLogSerializer implements Writer
     {
         private static final String logDirectory = System.getProperty("cassandra.logdir", ".");
-        private final ExecutorService loggerService = Executors.newFixedThreadPool(1);
+        private final ExecutorPlus loggerService = executorFactory().sequential("CompactionLogger");
         // This is only accessed on the logger service thread, so it does not need to be thread safe
         private final Set<Object> rolled = new HashSet<>();
         private OutputStreamWriter stream;
@@ -359,4 +362,10 @@ public class CompactionLogger
             });
         }
     }
+
+    public static void shutdownNowAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
+    {
+        ExecutorUtils.shutdownNowAndWait(timeout, unit, serializer.loggerService);
+    }
+
 }
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index ace83d1..e1f940b 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -20,7 +20,11 @@ package org.apache.cassandra.db.compaction;
 import java.io.File;
 import java.io.IOException;
 import java.util.*;
-import java.util.concurrent.*;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.BooleanSupplier;
 import java.util.function.Predicate;
@@ -31,18 +35,17 @@ import javax.management.openmbean.TabularData;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.*;
-import com.google.common.util.concurrent.*;
+import com.google.common.util.concurrent.RateLimiter;
+import com.google.common.util.concurrent.Uninterruptibles;
 
+import org.apache.cassandra.concurrent.ExecutorFactory;
+import org.apache.cassandra.concurrent.WrappedExecutorPlus;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import io.netty.util.concurrent.FastThreadLocal;
 import org.apache.cassandra.cache.AutoSavingCache;
-import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
-import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.repair.NoSuchRepairSessionException;
 import org.apache.cassandra.schema.TableMetadata;
@@ -80,9 +83,15 @@ import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.concurrent.Future;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
 import org.apache.cassandra.utils.concurrent.Refs;
 
 import static java.util.Collections.singleton;
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
+import static org.apache.cassandra.concurrent.FutureTask.callable;
+import static org.apache.cassandra.config.DatabaseDescriptor.getConcurrentCompactors;
+import static org.apache.cassandra.db.compaction.CompactionManager.CompactionExecutor.compactionThreadGroup;
 import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
 import static org.apache.cassandra.service.ActiveRepairService.UNREPAIRED_SSTABLE;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
@@ -107,17 +116,6 @@ public class CompactionManager implements CompactionManagerMBean
     public static final int NO_GC = Integer.MIN_VALUE;
     public static final int GC_ALL = Integer.MAX_VALUE;
 
-    // A thread local that tells us if the current thread is owned by the compaction manager. Used
-    // by CounterContext to figure out if it should log a warning for invalid counter shards.
-    public static final FastThreadLocal<Boolean> isCompactionManager = new FastThreadLocal<Boolean>()
-    {
-        @Override
-        protected Boolean initialValue()
-        {
-            return false;
-        }
-    };
-
     static
     {
         instance = new CompactionManager();
@@ -193,7 +191,7 @@ public class CompactionManager implements CompactionManagerMBean
          * are idle threads stil. (CASSANDRA-4310)
          */
         int count = compactingCF.count(cfs);
-        if (count > 0 && executor.getActiveCount() >= executor.getMaximumPoolSize())
+        if (count > 0 && executor.getActiveTaskCount() >= executor.getMaximumPoolSize())
         {
             logger.trace("Background compaction is still running for {}.{} ({} remaining). Skipping",
                          cfs.keyspace.getName(), cfs.name, count);
@@ -719,12 +717,12 @@ public class CompactionManager implements CompactionManagerMBean
     /**
      * Splits the given token ranges of the given sstables into a pending repair silo
      */
-    public ListenableFuture<?> submitPendingAntiCompaction(ColumnFamilyStore cfs,
-                                                           RangesAtEndpoint tokenRanges,
-                                                           Refs<SSTableReader> sstables,
-                                                           LifecycleTransaction txn,
-                                                           UUID sessionId,
-                                                           BooleanSupplier isCancelled)
+    public Future<Void> submitPendingAntiCompaction(ColumnFamilyStore cfs,
+                                                    RangesAtEndpoint tokenRanges,
+                                                    Refs<SSTableReader> sstables,
+                                                    LifecycleTransaction txn,
+                                                    UUID sessionId,
+                                                    BooleanSupplier isCancelled)
     {
         Runnable runnable = new WrappedRunnable()
         {
@@ -737,7 +735,7 @@ public class CompactionManager implements CompactionManagerMBean
             }
         };
 
-        ListenableFuture<?> task = null;
+        Future<Void> task = null;
         try
         {
             task = executor.submitIfRunning(runnable, "pending anticompaction");
@@ -1720,7 +1718,7 @@ public class CompactionManager implements CompactionManagerMBean
     }
 
     @VisibleForTesting
-    ListenableFuture<?> submitIndexBuild(final SecondaryIndexBuilder builder, ActiveCompactionsTracker activeCompactions)
+    Future<?> submitIndexBuild(final SecondaryIndexBuilder builder, ActiveCompactionsTracker activeCompactions)
     {
         Runnable runnable = new Runnable()
         {
@@ -1744,7 +1742,7 @@ public class CompactionManager implements CompactionManagerMBean
     /**
      * Is not scheduled, because it is performing disjoint work from sstable compaction.
      */
-    public ListenableFuture<?> submitIndexBuild(final SecondaryIndexBuilder builder)
+    public Future<?> submitIndexBuild(final SecondaryIndexBuilder builder)
     {
         return submitIndexBuild(builder, active);
     }
@@ -1813,13 +1811,13 @@ public class CompactionManager implements CompactionManagerMBean
         return cfs.isIndex() ? nowInSec : cfs.gcBefore(nowInSec);
     }
 
-    public ListenableFuture<Long> submitViewBuilder(final ViewBuilderTask task)
+    public Future<Long> submitViewBuilder(final ViewBuilderTask task)
     {
         return submitViewBuilder(task, active);
     }
 
     @VisibleForTesting
-    ListenableFuture<Long> submitViewBuilder(final ViewBuilderTask task, ActiveCompactionsTracker activeCompactions)
+    Future<Long> submitViewBuilder(final ViewBuilderTask task, ActiveCompactionsTracker activeCompactions)
     {
         return viewBuildExecutor.submitIfRunning(() -> {
             activeCompactions.beginCompaction(task);
@@ -1839,63 +1837,39 @@ public class CompactionManager implements CompactionManagerMBean
         return active.getCompactions().size();
     }
 
-    static class CompactionExecutor extends JMXEnabledThreadPoolExecutor
+    public static boolean isCompactor(Thread thread)
     {
-        protected CompactionExecutor(int minThreads, int maxThreads, String name, BlockingQueue<Runnable> queue)
-        {
-            super(minThreads, maxThreads, 60, TimeUnit.SECONDS, queue, new NamedThreadFactory(name, Thread.MIN_PRIORITY), "internal");
-        }
+        return thread.getThreadGroup().getParent() == compactionThreadGroup;
+    }
 
-        private CompactionExecutor(int threadCount, String name)
-        {
-            this(threadCount, threadCount, name, new LinkedBlockingQueue<Runnable>());
-        }
+    // TODO: this is a bit ugly, but no uglier than it was
+    static class CompactionExecutor extends WrappedExecutorPlus
+    {
+        static final ThreadGroup compactionThreadGroup = executorFactory().newThreadGroup("compaction");
+        private static final WithResources RESCHEDULE_FAILED = () -> SnapshotDeletingTask::rescheduleFailedTasks;
 
         public CompactionExecutor()
         {
-            this(Math.max(1, DatabaseDescriptor.getConcurrentCompactors()), "CompactionExecutor");
+            this(executorFactory(), getConcurrentCompactors(), "CompactionExecutor", Integer.MAX_VALUE);
         }
 
-        protected void beforeExecute(Thread t, Runnable r)
+        public CompactionExecutor(int threads, String name, int queueSize)
         {
-            // can't set this in Thread factory, so we do it redundantly here
-            isCompactionManager.set(true);
-            super.beforeExecute(t, r);
+            this(executorFactory(), threads, name, queueSize);
         }
 
-        // modified from DebuggableThreadPoolExecutor so that CompactionInterruptedExceptions are not logged
-        @Override
-        public void afterExecute(Runnable r, Throwable t)
+        protected CompactionExecutor(ExecutorFactory executorFactory, int threads, String name, int queueSize)
         {
-            DebuggableThreadPoolExecutor.maybeResetTraceSessionWrapper(r);
-
-            if (t == null)
-                t = DebuggableThreadPoolExecutor.extractThrowable(r);
-
-            if (t != null)
-            {
-                if (t instanceof CompactionInterruptedException)
-                {
-                    logger.info(t.getMessage());
-                    if (t.getSuppressed() != null && t.getSuppressed().length > 0)
-                        logger.warn("Interruption of compaction encountered exceptions:", t);
-                    else
-                        logger.trace("Full interruption stack trace:", t);
-                }
-                else
-                {
-                    DebuggableThreadPoolExecutor.handleOrLog(t);
-                }
-            }
-
-            // Snapshots cannot be deleted on Windows while segments of the root element are mapped in NTFS. Compactions
-            // unmap those segments which could free up a snapshot for successful deletion.
-            SnapshotDeletingTask.rescheduleFailedTasks();
+            super(executorFactory
+                    .withJmxInternal()
+                    .configurePooled(name, threads)
+                    .withThreadGroup(compactionThreadGroup)
+                    .withQueueLimit(queueSize).build());
         }
 
-        public ListenableFuture<?> submitIfRunning(Runnable task, String name)
+        public Future<Void> submitIfRunning(Runnable task, String name)
         {
-            return submitIfRunning(Executors.callable(task, null), name);
+            return submitIfRunning(callable(name, task), name);
         }
 
         /**
@@ -1908,19 +1882,11 @@ public class CompactionManager implements CompactionManagerMBean
          * @return the future that will deliver the task result, or a future that has already been
          *         cancelled if the task could not be submitted.
          */
-        public <T> ListenableFuture<T> submitIfRunning(Callable<T> task, String name)
+        public <T> Future<T> submitIfRunning(Callable<T> task, String name)
         {
-            if (isShutdown())
-            {
-                logger.info("Executor has been shut down, not submitting {}", name);
-                return Futures.immediateCancelledFuture();
-            }
-
             try
             {
-                ListenableFutureTask<T> ret = ListenableFutureTask.create(task);
-                execute(ret);
-                return ret;
+                return submit(task);
             }
             catch (RejectedExecutionException ex)
             {
@@ -1929,15 +1895,35 @@ public class CompactionManager implements CompactionManagerMBean
                 else
                     logger.error("Failed to submit {}", name, ex);
 
-                return Futures.immediateCancelledFuture();
+                return ImmediateFuture.cancelled();
             }
         }
+
+        public void execute(Runnable command)
+        {
+            executor.execute(RESCHEDULE_FAILED, command);
+        }
+
+        public <T> Future<T> submit(Callable<T> task)
+        {
+            return executor.submit(RESCHEDULE_FAILED, task);
+        }
+
+        public <T> Future<T> submit(Runnable task, T result)
+        {
+            return submit(callable(task, result));
+        }
+
+        public Future<?> submit(Runnable task)
+        {
+            return submit(task, null);
+        }
     }
 
     // TODO: pull out relevant parts of CompactionExecutor and move to ValidationManager
     public static class ValidationExecutor extends CompactionExecutor
     {
-        // CompactionExecutor, and by extension ValidationExecutor, use DebuggableThreadPoolExecutor's
+        // CompactionExecutor, and by extension ValidationExecutor, use ExecutorPlus's
         // default RejectedExecutionHandler which blocks the submitting thread when the work queue is
         // full. The calling thread in this case is AntiEntropyStage, so in most cases we don't actually
         // want to block when the ValidationExecutor is saturated as this prevents progress on all
@@ -1952,11 +1938,8 @@ public class CompactionManager implements CompactionManagerMBean
         public ValidationExecutor()
         {
             super(DatabaseDescriptor.getConcurrentValidations(),
-                  DatabaseDescriptor.getConcurrentValidations(),
                   "ValidationExecutor",
-                  new LinkedBlockingQueue<>());
-
-            allowCoreThreadTimeOut(true);
+                  Integer.MAX_VALUE);
         }
 
         public void adjustPoolSize()
@@ -1970,7 +1953,7 @@ public class CompactionManager implements CompactionManagerMBean
     {
         public ViewBuildExecutor()
         {
-            super(DatabaseDescriptor.getConcurrentViewBuilders(), "ViewBuildExecutor");
+            super(DatabaseDescriptor.getConcurrentViewBuilders(), "ViewBuildExecutor", Integer.MAX_VALUE);
         }
     }
 
@@ -1978,7 +1961,7 @@ public class CompactionManager implements CompactionManagerMBean
     {
         public CacheCleanupExecutor()
         {
-            super(1, "CacheCleanupExecutor");
+            super(1, "CacheCleanupExecutor", Integer.MAX_VALUE);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 867c23b..d271bb3 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -41,6 +41,7 @@ import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.Closeable;
 import org.apache.cassandra.utils.concurrent.Refs;
 import org.apache.cassandra.utils.memory.HeapAllocator;
 
diff --git a/src/java/org/apache/cassandra/db/context/CounterContext.java b/src/java/org/apache/cassandra/db/context/CounterContext.java
index 7db8192..37bd3ca 100644
--- a/src/java/org/apache/cassandra/db/context/CounterContext.java
+++ b/src/java/org/apache/cassandra/db/context/CounterContext.java
@@ -465,7 +465,7 @@ public class CounterContext
                 if (leftClock == rightClock)
                 {
                     // Can happen if an sstable gets lost and disk failure policy is set to 'best effort'
-                    if (leftCount != rightCount && CompactionManager.isCompactionManager.get())
+                    if (leftCount != rightCount && CompactionManager.isCompactor(Thread.currentThread()))
                     {
                         logger.warn("invalid global counter shard detected; ({}, {}, {}) and ({}, {}, {}) differ only in "
                                     + "count; will pick highest to self-heal on compaction",
@@ -506,7 +506,7 @@ public class CounterContext
             // We should never see non-local shards w/ same id+clock but different counts. However, if we do
             // we should "heal" the problem by being deterministic in our selection of shard - and
             // log the occurrence so that the operator will know something is wrong.
-            if (leftCount != rightCount && CompactionManager.isCompactionManager.get())
+            if (leftCount != rightCount && CompactionManager.isCompactor(Thread.currentThread()))
             {
                 logger.warn("invalid remote counter shard detected; ({}, {}, {}) and ({}, {}, {}) differ only in "
                             + "count; will pick highest to self-heal on compaction",
diff --git a/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java b/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
index d08c62f..52d6160 100644
--- a/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
+++ b/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
@@ -23,7 +23,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
diff --git a/src/java/org/apache/cassandra/db/repair/CassandraKeyspaceRepairManager.java b/src/java/org/apache/cassandra/db/repair/CassandraKeyspaceRepairManager.java
index 4fa8650..29950fc 100644
--- a/src/java/org/apache/cassandra/db/repair/CassandraKeyspaceRepairManager.java
+++ b/src/java/org/apache/cassandra/db/repair/CassandraKeyspaceRepairManager.java
@@ -19,16 +19,16 @@
 package org.apache.cassandra.db.repair;
 
 import java.util.Collection;
+import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 import java.util.function.BooleanSupplier;
 
-import com.google.common.util.concurrent.ListenableFuture;
-
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.apache.cassandra.repair.KeyspaceRepairManager;
+import org.apache.cassandra.utils.concurrent.Future;
 
 public class CassandraKeyspaceRepairManager implements KeyspaceRepairManager
 {
@@ -40,11 +40,11 @@ public class CassandraKeyspaceRepairManager implements KeyspaceRepairManager
     }
 
     @Override
-    public ListenableFuture prepareIncrementalRepair(UUID sessionID,
-                                                     Collection<ColumnFamilyStore> tables,
-                                                     RangesAtEndpoint tokenRanges,
-                                                     ExecutorService executor,
-                                                     BooleanSupplier isCancelled)
+    public Future<List<Void>> prepareIncrementalRepair(UUID sessionID,
+                                                       Collection<ColumnFamilyStore> tables,
+                                                       RangesAtEndpoint tokenRanges,
+                                                       ExecutorService executor,
+                                                       BooleanSupplier isCancelled)
     {
         PendingAntiCompaction pac = new PendingAntiCompaction(sessionID, tables, tokenRanges, executor, isCancelled);
         return pac.run();
diff --git a/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java b/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java
index 0d89282..59eff55 100644
--- a/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java
+++ b/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java
@@ -27,18 +27,18 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.function.BooleanSupplier;
+import java.util.function.Function;
 import java.util.stream.Collectors;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
-import com.google.common.util.concurrent.AsyncFunction;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListenableFutureTask;
-import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.Uninterruptibles;
 
+import org.apache.cassandra.concurrent.FutureTask;
+import org.apache.cassandra.utils.concurrent.Future;
+import org.apache.cassandra.utils.concurrent.FutureCombiner;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -256,7 +256,7 @@ public class PendingAntiCompaction
         }
     }
 
-    static class AcquisitionCallback implements AsyncFunction<List<AcquireResult>, Object>
+    static class AcquisitionCallback implements Function<List<AcquireResult>, Future<List<Void>>>
     {
         private final UUID parentRepairSession;
         private final RangesAtEndpoint tokenRanges;
@@ -269,7 +269,7 @@ public class PendingAntiCompaction
             this.isCancelled = isCancelled;
         }
 
-        ListenableFuture<?> submitPendingAntiCompaction(AcquireResult result)
+        Future<Void> submitPendingAntiCompaction(AcquireResult result)
         {
             return CompactionManager.instance.submitPendingAntiCompaction(result.cfs, tokenRanges, result.refs, result.txn, parentRepairSession, isCancelled);
         }
@@ -288,7 +288,7 @@ public class PendingAntiCompaction
             });
         }
 
-        public ListenableFuture apply(List<AcquireResult> results) throws Exception
+        public Future<List<Void>> apply(List<AcquireResult> results)
         {
             if (Iterables.any(results, AcquisitionCallback::shouldAbort))
             {
@@ -306,21 +306,21 @@ public class PendingAntiCompaction
                                                "This is usually caused by running multiple incremental repairs on nodes that share token ranges",
                                                parentRepairSession);
                 logger.warn(message);
-                return Futures.immediateFailedFuture(new SSTableAcquisitionException(message));
+                return ImmediateFuture.failure(new SSTableAcquisitionException(message));
             }
             else
             {
-                List<ListenableFuture<?>> pendingAntiCompactions = new ArrayList<>(results.size());
+                List<Future<Void>> pendingAntiCompactions = new ArrayList<>(results.size());
                 for (AcquireResult result : results)
                 {
                     if (result.txn != null)
                     {
-                        ListenableFuture<?> future = submitPendingAntiCompaction(result);
+                        Future<Void> future = submitPendingAntiCompaction(result);
                         pendingAntiCompactions.add(future);
                     }
                 }
 
-                return Futures.allAsList(pendingAntiCompactions);
+                return FutureCombiner.allOf(pendingAntiCompactions);
             }
         }
     }
@@ -360,19 +360,19 @@ public class PendingAntiCompaction
         this.isCancelled = isCancelled;
     }
 
-    public ListenableFuture run()
+    public Future<List<Void>> run()
     {
-        List<ListenableFutureTask<AcquireResult>> tasks = new ArrayList<>(tables.size());
+        List<FutureTask<AcquireResult>> tasks = new ArrayList<>(tables.size());
         for (ColumnFamilyStore cfs : tables)
         {
             cfs.forceBlockingFlush();
-            ListenableFutureTask<AcquireResult> task = ListenableFutureTask.create(getAcquisitionCallable(cfs, tokenRanges.ranges(), prsId, acquireRetrySeconds, acquireSleepMillis));
+            FutureTask<AcquireResult> task = new FutureTask<>(getAcquisitionCallable(cfs, tokenRanges.ranges(), prsId, acquireRetrySeconds, acquireSleepMillis));
             executor.submit(task);
             tasks.add(task);
         }
-        ListenableFuture<List<AcquireResult>> acquisitionResults = Futures.successfulAsList(tasks);
-        ListenableFuture compactionResult = Futures.transformAsync(acquisitionResults, getAcquisitionCallback(prsId, tokenRanges), MoreExecutors.directExecutor());
-        return compactionResult;
+
+        Future<List<AcquireResult>> acquisitionResults = FutureCombiner.successfulOf(tasks);
+        return acquisitionResults.andThenAsync(getAcquisitionCallback(prsId, tokenRanges));
     }
 
     @VisibleForTesting
diff --git a/src/java/org/apache/cassandra/db/view/ViewBuilder.java b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
index a88ffbe..d087224 100644
--- a/src/java/org/apache/cassandra/db/view/ViewBuilder.java
+++ b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
@@ -22,16 +22,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.MoreExecutors;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -49,6 +45,9 @@ import org.apache.cassandra.repair.SystemDistributedKeyspace;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.concurrent.Future;
+import org.apache.cassandra.utils.concurrent.FutureCombiner;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
 
 import static java.util.stream.Collectors.toList;
 
@@ -73,7 +72,7 @@ class ViewBuilder
     private final Set<ViewBuilderTask> tasks = Sets.newConcurrentHashSet();
     private volatile long keysBuilt = 0;
     private volatile boolean isStopped = false;
-    private volatile Future<?> future = Futures.immediateFuture(null);
+    private volatile Future<?> future = ImmediateFuture.success(null);
 
     ViewBuilder(ColumnFamilyStore baseCfs, View view)
     {
@@ -162,21 +161,21 @@ class ViewBuilder
 
         // Submit a new view build task for each building range.
         // We keep record of all the submitted tasks to be able of stopping them.
-        List<ListenableFuture<Long>> futures = pendingRanges.entrySet()
-                                                            .stream()
-                                                            .map(e -> new ViewBuilderTask(baseCfs,
-                                                                                          view,
-                                                                                          e.getKey(),
-                                                                                          e.getValue().left,
-                                                                                          e.getValue().right))
-                                                            .peek(tasks::add)
-                                                            .map(CompactionManager.instance::submitViewBuilder)
-                                                            .collect(toList());
+        List<Future<Long>> futures = pendingRanges.entrySet()
+                                                  .stream()
+                                                  .map(e -> new ViewBuilderTask(baseCfs,
+                                                                                view,
+                                                                                e.getKey(),
+                                                                                e.getValue().left,
+                                                                                e.getValue().right))
+                                                  .peek(tasks::add)
+                                                  .map(CompactionManager.instance::submitViewBuilder)
+                                                  .collect(toList());
 
         // Add a callback to process any eventual new local range and mark the view as built, doing a delayed retry if
         // the tasks don't succeed
-        ListenableFuture<List<Long>> future = Futures.allAsList(futures);
-        Futures.addCallback(future, new FutureCallback<List<Long>>()
+        Future<List<Long>> future = FutureCombiner.allOf(futures);
+        future.addCallback(new FutureCallback<List<Long>>()
         {
             public void onSuccess(List<Long> result)
             {
@@ -200,7 +199,7 @@ class ViewBuilder
                     logger.warn("Materialized View failed to complete, sleeping 5 minutes before restarting", t);
                 }
             }
-        }, MoreExecutors.directExecutor());
+        });
         this.future = future;
     }
 
@@ -228,10 +227,16 @@ class ViewBuilder
     /**
      * Stops the view building.
      */
-    synchronized void stop()
+    void stop()
     {
-        boolean wasStopped = isStopped;
-        internalStop(false);
+        boolean wasStopped;
+        synchronized (this)
+        {
+            wasStopped = isStopped;
+            internalStop(false);
+        }
+        // TODO: very unclear what the goal is here. why do we wait only if we were the first to invoke stop?
+        // but we wait outside the synchronized block to avoid a deadlock with `build` in the future callback
         if (!wasStopped)
             FBUtilities.waitOnFuture(future);
     }
diff --git a/src/java/org/apache/cassandra/db/view/ViewBuilderTask.java b/src/java/org/apache/cassandra/db/view/ViewBuilderTask.java
index 70f7bb7..2f32db0 100644
--- a/src/java/org/apache/cassandra/db/view/ViewBuilderTask.java
+++ b/src/java/org/apache/cassandra/db/view/ViewBuilderTask.java
@@ -31,7 +31,6 @@ import com.google.common.base.Function;
 import com.google.common.base.Objects;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.PeekingIterator;
-import com.google.common.util.concurrent.Futures;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java
index 39ebf78..9575f6c 100644
--- a/src/java/org/apache/cassandra/dht/BootStrapper.java
+++ b/src/java/org/apache/cassandra/dht/BootStrapper.java
@@ -20,8 +20,7 @@ package org.apache.cassandra.dht;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import com.google.common.util.concurrent.ListenableFuture;
-
+import org.apache.cassandra.utils.concurrent.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -61,7 +60,7 @@ public class BootStrapper extends ProgressEventNotifierSupport
         this.tokenMetadata = tmd;
     }
 
-    public ListenableFuture<StreamState> bootstrap(StreamStateStore stateStore, boolean useStrictConsistency)
+    public Future<StreamState> bootstrap(StreamStateStore stateStore, boolean useStrictConsistency)
     {
         logger.trace("Beginning bootstrap process");
 
diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java
index 7dd73c7..98068db 100644
--- a/src/java/org/apache/cassandra/gms/Gossiper.java
+++ b/src/java/org/apache/cassandra/gms/Gossiper.java
@@ -20,7 +20,14 @@ package org.apache.cassandra.gms;
 import java.net.UnknownHostException;
 import java.util.*;
 import java.util.Map.Entry;
-import java.util.concurrent.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.BooleanSupplier;
 import java.util.function.Supplier;
@@ -36,10 +43,10 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.ListenableFutureTask;
 import com.google.common.util.concurrent.Uninterruptibles;
 
-import org.apache.cassandra.concurrent.JMXEnabledSingleThreadExecutor;
+import org.apache.cassandra.concurrent.*;
+import org.apache.cassandra.concurrent.FutureTask;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.NoPayload;
 import org.apache.cassandra.net.Verb;
@@ -53,8 +60,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import io.netty.util.concurrent.FastThreadLocal;
-import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.SystemKeyspace;
@@ -73,6 +78,7 @@ import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.RecomputingSupplier;
 import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.config.CassandraRelevantProperties.GOSSIPER_QUARANTINE_DELAY;
 import static org.apache.cassandra.net.NoPayload.noPayload;
 import static org.apache.cassandra.net.Verb.ECHO_REQ;
@@ -104,7 +110,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         public static final String DISABLE_THREAD_VALIDATION = "cassandra.gossip.disable_thread_validation";
     }
 
-    private static final DebuggableScheduledThreadPoolExecutor executor = new DebuggableScheduledThreadPoolExecutor("GossipTasks");
+    private static final ScheduledExecutorPlus executor = executorFactory().scheduled("GossipTasks");
 
     static final ApplicationState[] STATES = ApplicationState.values();
     static final List<String> DEAD_STATES = Arrays.asList(VersionedValue.REMOVING_TOKEN, VersionedValue.REMOVED_TOKEN,
@@ -261,7 +267,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
 
     private static boolean isInGossipStage()
     {
-        return ((JMXEnabledSingleThreadExecutor) Stage.GOSSIP.executor()).isExecutedBy(Thread.currentThread());
+        return Stage.GOSSIP.executor().inExecutor();
     }
 
     private static void checkProperThreadForStateMutation()
@@ -526,7 +532,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
             return;
         }
 
-        ListenableFutureTask task = ListenableFutureTask.create(runnable, null);
+        FutureTask task = new FutureTask<>(runnable);
         Stage.GOSSIP.execute(task);
         try
         {
@@ -1031,7 +1037,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         long now = currentTimeMillis();
         long nowNano = nanoTime();
 
-        long pending = ((JMXEnabledThreadPoolExecutor) Stage.GOSSIP.executor()).metrics.pendingTasks.getValue();
+        long pending = Stage.GOSSIP.executor().getPendingTaskCount();
         if (pending > 0 && lastProcessedMessageAt < now - 1000)
         {
             // if some new messages just arrived, give the executor some time to work on them
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
index 57fb238..bb6881b 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
@@ -55,6 +55,7 @@ import org.apache.cassandra.hadoop.*;
 import org.apache.cassandra.utils.*;
 
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 
 /**
  * Hadoop InputFormat allowing map/reduce against Cassandra rows within one ColumnFamily.
@@ -135,7 +136,7 @@ public class CqlInputFormat extends org.apache.hadoop.mapreduce.InputFormat<Long
         logger.trace("partitioner is {}", partitioner);
 
         // canonical ranges, split into pieces, fetching the splits in parallel
-        ExecutorService executor = new ThreadPoolExecutor(0, 128, 60L, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>());
+        ExecutorService executor = executorFactory().pooled("HadoopInput", 128);
         List<org.apache.hadoop.mapreduce.InputSplit> splits = new ArrayList<>();
 
         String[] inputInitialAddress = ConfigHelper.getInputInitialAddress(conf).split(",");
diff --git a/src/java/org/apache/cassandra/hints/Hint.java b/src/java/org/apache/cassandra/hints/Hint.java
index 11ac515..3089894 100644
--- a/src/java/org/apache/cassandra/hints/Hint.java
+++ b/src/java/org/apache/cassandra/hints/Hint.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.hints;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Throwables;
@@ -34,6 +33,8 @@ import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.utils.concurrent.Future;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
 import org.apache.cassandra.utils.vint.VIntCoding;
 import org.assertj.core.util.VisibleForTesting;
 
@@ -94,7 +95,7 @@ public final class Hint
     /**
      * Applies the contained mutation unless it's expired, filtering out any updates for truncated tables
      */
-    CompletableFuture<?> applyFuture()
+    Future<?> applyFuture()
     {
         if (isLive())
         {
@@ -108,7 +109,7 @@ public final class Hint
                 return filtered.applyFuture();
         }
 
-        return CompletableFuture.completedFuture(null);
+        return ImmediateFuture.success(null);
     }
 
     void apply()
diff --git a/src/java/org/apache/cassandra/hints/HintVerbHandler.java b/src/java/org/apache/cassandra/hints/HintVerbHandler.java
index 2fbe475..e6758d0 100644
--- a/src/java/org/apache/cassandra/hints/HintVerbHandler.java
+++ b/src/java/org/apache/cassandra/hints/HintVerbHandler.java
@@ -94,7 +94,7 @@ public final class HintVerbHandler implements IVerbHandler<HintMessage>
         else
         {
             // the common path - the node is both the destination and a valid replica for the hint.
-            hint.applyFuture().thenAccept(o -> respond(message)).exceptionally(e -> {logger.debug("Failed to apply hint", e); return null;});
+            hint.applyFuture().addCallback(o -> respond(message), e -> logger.debug("Failed to apply hint", e));
         }
     }
 
diff --git a/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java b/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
index 2a5239c..705715c 100644
--- a/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
+++ b/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
@@ -20,28 +20,27 @@ package org.apache.cassandra.hints;
 import java.io.File;
 import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.BooleanSupplier;
 import java.util.function.Predicate;
 import java.util.function.Supplier;
 
 import com.google.common.util.concurrent.RateLimiter;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
-import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
+import org.apache.cassandra.utils.concurrent.Future;
 
-import static java.lang.Thread.MIN_PRIORITY;
-import static java.util.concurrent.TimeUnit.MINUTES;
-import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 
 /**
  * A multi-threaded (by default) executor for dispatching hints.
@@ -53,7 +52,7 @@ final class HintsDispatchExecutor
     private static final Logger logger = LoggerFactory.getLogger(HintsDispatchExecutor.class);
 
     private final File hintsDirectory;
-    private final ExecutorService executor;
+    private final ExecutorPlus executor;
     private final AtomicBoolean isPaused;
     private final Predicate<InetAddressAndPort> isAlive;
     private final Map<UUID, Future> scheduledDispatches;
@@ -65,10 +64,11 @@ final class HintsDispatchExecutor
         this.isAlive = isAlive;
 
         scheduledDispatches = new ConcurrentHashMap<>();
-        executor = new JMXEnabledThreadPoolExecutor(maxThreads, 1, MINUTES,
-                                                    newBlockingQueue(),
-                                                    new NamedThreadFactory("HintsDispatcher", MIN_PRIORITY),
-                                                    "internal");
+        executor = executorFactory()
+                .withJmxInternal()
+                .configurePooled("HintsDispatcher", maxThreads)
+                .withThreadPriority(Thread.MIN_PRIORITY)
+                .build();
     }
 
     /*
diff --git a/src/java/org/apache/cassandra/hints/HintsService.java b/src/java/org/apache/cassandra/hints/HintsService.java
index a399a2f..b727b35 100644
--- a/src/java/org/apache/cassandra/hints/HintsService.java
+++ b/src/java/org/apache/cassandra/hints/HintsService.java
@@ -23,7 +23,9 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
-import java.util.concurrent.*;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
@@ -32,6 +34,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableMap;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.utils.concurrent.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/hints/HintsWriteExecutor.java b/src/java/org/apache/cassandra/hints/HintsWriteExecutor.java
index 26e3103..c4bfff0 100644
--- a/src/java/org/apache/cassandra/hints/HintsWriteExecutor.java
+++ b/src/java/org/apache/cassandra/hints/HintsWriteExecutor.java
@@ -20,18 +20,22 @@ package org.apache.cassandra.hints;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Iterator;
-import java.util.concurrent.*;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.concurrent.ExecutorPlus;
+import org.apache.cassandra.utils.concurrent.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
+
 /**
  * A single threaded executor that exclusively writes all the hints and otherwise manipulate the writers.
  *
@@ -47,14 +51,14 @@ final class HintsWriteExecutor
 
     private final HintsCatalog catalog;
     private final ByteBuffer writeBuffer;
-    private final ExecutorService executor;
+    private final ExecutorPlus executor;
 
     HintsWriteExecutor(HintsCatalog catalog)
     {
         this.catalog = catalog;
 
         writeBuffer = ByteBuffer.allocateDirect(WRITE_BUFFER_SIZE);
-        executor = DebuggableThreadPoolExecutor.createWithFixedPoolSize("HintsWriteExecutor", 1);
+        executor = executorFactory().sequential("HintsWriteExecutor");
     }
 
     /*
diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index a3fd881..d75b488 100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@ -19,7 +19,9 @@ package org.apache.cassandra.index;
 
 import java.lang.reflect.Constructor;
 import java.util.*;
-import java.util.concurrent.*;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Function;
 import java.util.stream.Collectors;
@@ -28,26 +30,15 @@ import java.util.stream.Stream;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
+import com.google.common.collect.*;
 import com.google.common.primitives.Longs;
 import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-import com.google.common.util.concurrent.SettableFuture;
-
 import org.apache.commons.lang3.StringUtils;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
-import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.concurrent.ExecutorPlus;
+import org.apache.cassandra.concurrent.ImmediateExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.*;
@@ -58,7 +49,8 @@ import org.apache.cassandra.db.filter.DataLimits;
 import org.apache.cassandra.db.filter.RowFilter;
 import org.apache.cassandra.db.lifecycle.SSTableSet;
 import org.apache.cassandra.db.lifecycle.View;
-import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.index.Index.IndexBuildingSupport;
@@ -76,13 +68,11 @@ import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.concurrent.Refs;
+import org.apache.cassandra.utils.concurrent.*;
 
-import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.cassandra.concurrent.Stage.KEEP_ALIVE_SECONDS;
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.utils.ExecutorUtils.awaitTermination;
 import static org.apache.cassandra.utils.ExecutorUtils.shutdown;
-import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
 /**
  * Handles the core maintenance functionality associated with indexes: adding/removing them to or from
@@ -162,16 +152,12 @@ public class SecondaryIndexManager implements IndexRegistry, INotificationConsum
     private final Map<String, AtomicInteger> inProgressBuilds = Maps.newConcurrentMap();
 
     // executes tasks returned by Indexer#addIndexColumn which may require index(es) to be (re)built
-    private static final ListeningExecutorService asyncExecutor = MoreExecutors.listeningDecorator(
-    new JMXEnabledThreadPoolExecutor(1,
-                                     KEEP_ALIVE_SECONDS,
-                                     SECONDS,
-                                     newBlockingQueue(),
-                                     new NamedThreadFactory("SecondaryIndexManagement"),
-                                     "internal"));
+    private static final ExecutorPlus asyncExecutor = executorFactory()
+            .withJmxInternal()
+            .sequential("SecondaryIndexManagement");
 
     // executes all blocking tasks produced by Indexers e.g. getFlushTask, getMetadataReloadTask etc
-    private static final ListeningExecutorService blockingExecutor = MoreExecutors.newDirectExecutorService();
+    private static final ExecutorPlus blockingExecutor = ImmediateExecutor.INSTANCE;
 
     /**
      * The underlying column family containing the source data for these indexes
@@ -209,12 +195,12 @@ public class SecondaryIndexManager implements IndexRegistry, INotificationConsum
         Index index = indexes.get(indexDef.name);
         Callable<?> reloadTask = index.getMetadataReloadTask(indexDef);
         return reloadTask == null
-               ? Futures.immediateFuture(null)
+               ? ImmediateFuture.success(null)
                : blockingExecutor.submit(reloadTask);
     }
 
     @SuppressWarnings("unchecked")
-    private synchronized Future<?> createIndex(IndexMetadata indexDef, boolean isNewCF)
+    private synchronized Future<Void> createIndex(IndexMetadata indexDef, boolean isNewCF)
     {
         final Index index = createInstance(indexDef);
         index.register(this);
@@ -242,27 +228,21 @@ public class SecondaryIndexManager implements IndexRegistry, INotificationConsum
         if (initialBuildTask == null)
         {
             markIndexBuilt(index, true);
-            return Futures.immediateFuture(null);
+            return ImmediateFuture.success(null);
         }
 
         // otherwise run the initialization task asynchronously with a callback to mark it built or failed
-        final SettableFuture initialization = SettableFuture.create();
-        Futures.addCallback(asyncExecutor.submit(initialBuildTask), new FutureCallback()
-        {
-            @Override
-            public void onFailure(Throwable t)
-            {
-                logAndMarkIndexesFailed(Collections.singleton(index), t, true);
-                initialization.setException(t);
-            }
-
-            @Override
-            public void onSuccess(Object o)
-            {
-                markIndexBuilt(index, true);
-                initialization.set(o);
-            }
-        }, MoreExecutors.directExecutor());
+        final Promise<Void> initialization = new AsyncPromise<>();
+        asyncExecutor.submit(initialBuildTask)
+                     .addCallback(
+                         success -> {
+                             markIndexBuilt(index, true);
+                             initialization.trySuccess(null);
+                         },
+                         failure -> {
+                             logAndMarkIndexesFailed(Collections.singleton(index), failure, true);
+                             initialization.tryFailure(failure);
+                         });
 
         return initialization;
     }
@@ -520,15 +500,15 @@ public class SecondaryIndexManager implements IndexRegistry, INotificationConsum
             byType.forEach((buildingSupport, groupedIndexes) ->
                            {
                                SecondaryIndexBuilder builder = buildingSupport.getIndexBuildTask(baseCfs, groupedIndexes, sstables);
-                               final SettableFuture build = SettableFuture.create();
-                               Futures.addCallback(CompactionManager.instance.submitIndexBuild(builder), new FutureCallback()
+                               final AsyncPromise<Object> build = new AsyncPromise<>();
+                               CompactionManager.instance.submitIndexBuild(builder).addCallback(new FutureCallback()
                                {
                                    @Override
                                    public void onFailure(Throwable t)
                                    {
                                        logAndMarkIndexesFailed(groupedIndexes, t, false);
                                        unbuiltIndexes.addAll(groupedIndexes);
-                                       build.setException(t);
+                                       build.tryFailure(t);
                                    }
 
                                    @Override
@@ -537,9 +517,9 @@ public class SecondaryIndexManager implements IndexRegistry, INotificationConsum
                                        groupedIndexes.forEach(i -> markIndexBuilt(i, isFullRebuild));
                                        logger.info("Index build of {} completed", getIndexNames(groupedIndexes));
                                        builtIndexes.addAll(groupedIndexes);
-                                       build.set(o);
+                                       build.trySuccess(o);
                                    }
-                               }, MoreExecutors.directExecutor());
+                               });
                                futures.add(build);
                            });
 
@@ -1526,17 +1506,17 @@ public class SecondaryIndexManager implements IndexRegistry, INotificationConsum
         }
     }
 
-    private void executeBlocking(Callable<?> task, FutureCallback<Object> callback)
+    private void executeBlocking(Callable<?> task, FutureCallback callback)
     {
         if (null != task)
         {
-            ListenableFuture<?> f = blockingExecutor.submit(task);
-            if (callback != null) Futures.addCallback(f, callback, MoreExecutors.directExecutor());
+            Future<?> f = blockingExecutor.submit(task);
+            if (callback != null) f.addCallback(callback);
             FBUtilities.waitOnFuture(f);
         }
     }
 
-    private void executeAllBlocking(Stream<Index> indexers, Function<Index, Callable<?>> function, FutureCallback<Object> callback)
+    private void executeAllBlocking(Stream<Index> indexers, Function<Index, Callable<?>> function, FutureCallback callback)
     {
         if (function == null)
         {
@@ -1550,8 +1530,8 @@ public class SecondaryIndexManager implements IndexRegistry, INotificationConsum
                              Callable<?> task = function.apply(indexer);
                              if (null != task)
                              {
-                                 ListenableFuture<?> f = blockingExecutor.submit(task);
-                                 if (callback != null) Futures.addCallback(f, callback, MoreExecutors.directExecutor());
+                                 Future<?> f = blockingExecutor.submit(task);
+                                 if (callback != null) f.addCallback(callback);
                                  waitFor.add(f);
                              }
                          });
diff --git a/src/java/org/apache/cassandra/index/sasi/TermIterator.java b/src/java/org/apache/cassandra/index/sasi/TermIterator.java
index 3473234..c84b209 100644
--- a/src/java/org/apache/cassandra/index/sasi/TermIterator.java
+++ b/src/java/org/apache/cassandra/index/sasi/TermIterator.java
@@ -24,7 +24,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
 import io.netty.util.concurrent.FastThreadLocal;
-import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.concurrent.ExecutorFactory;
+import org.apache.cassandra.concurrent.ImmediateExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.index.sasi.disk.Token;
 import org.apache.cassandra.index.sasi.plan.Expression;
@@ -39,6 +40,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static java.lang.String.format;
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder.Mode.CONTAINS;
 import static org.apache.cassandra.index.sasi.plan.Expression.Op.PREFIX;
 import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownLatch;
@@ -57,16 +59,8 @@ public class TermIterator extends RangeIterator<Long, Token>
             logger.info("Search Concurrency Factor is set to {} for {}", concurrencyFactor, currentThread);
 
             return (concurrencyFactor <= 1)
-                    ? MoreExecutors.newDirectExecutorService()
-                    : Executors.newFixedThreadPool(concurrencyFactor, new ThreadFactory()
-            {
-                public final AtomicInteger count = new AtomicInteger();
-
-                public Thread newThread(Runnable task)
-                {
-                    return NamedThreadFactory.createThread(task, currentThread + "-SEARCH-" + count.incrementAndGet(), true);
-                }
-            });
+                    ? ImmediateExecutor.INSTANCE
+                    : executorFactory().pooled(currentThread + "-SEARCH-", concurrencyFactor);
         }
     };
 
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java b/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
index e4ac3f3..444db83 100644
--- a/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
+++ b/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
@@ -24,8 +24,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.*;
 
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
-import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.compaction.OperationType;
@@ -43,40 +42,33 @@ import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.concurrent.CountDownLatch;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.Futures;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
-import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownLatch;
-import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
 public class PerSSTableIndexWriter implements SSTableFlushObserver
 {
     private static final Logger logger = LoggerFactory.getLogger(PerSSTableIndexWriter.class);
 
     private static final int POOL_SIZE = 8;
-    private static final ThreadPoolExecutor INDEX_FLUSHER_MEMTABLE;
-    private static final ThreadPoolExecutor INDEX_FLUSHER_GENERAL;
+    private static final ExecutorPlus INDEX_FLUSHER_MEMTABLE;
+    private static final ExecutorPlus INDEX_FLUSHER_GENERAL;
 
     static
     {
-        INDEX_FLUSHER_GENERAL = new JMXEnabledThreadPoolExecutor(POOL_SIZE, POOL_SIZE, 1, MINUTES,
-                                                                 newBlockingQueue(),
-                                                                 new NamedThreadFactory("SASI-General"),
-                                                                 "internal");
-        INDEX_FLUSHER_GENERAL.allowCoreThreadTimeOut(true);
-
-        INDEX_FLUSHER_MEMTABLE = new JMXEnabledThreadPoolExecutor(POOL_SIZE, POOL_SIZE, 1, MINUTES,
-                                                                  newBlockingQueue(),
-                                                                  new NamedThreadFactory("SASI-Memtable"),
-                                                                  "internal");
-        INDEX_FLUSHER_MEMTABLE.allowCoreThreadTimeOut(true);
+        INDEX_FLUSHER_GENERAL = executorFactory().withJmxInternal()
+                                                 .pooled("SASI-General", POOL_SIZE);
+
+        INDEX_FLUSHER_MEMTABLE = executorFactory().withJmxInternal()
+                                                  .pooled("SASI-Memtable", POOL_SIZE);
     }
 
     private final int nowInSec = FBUtilities.nowInSeconds();
@@ -298,7 +290,7 @@ public class PerSSTableIndexWriter implements SSTableFlushObserver
                     {
                         @SuppressWarnings("resource")
                         OnDiskIndex last = scheduleSegmentFlush(false).call();
-                        segments.add(Futures.immediateFuture(last));
+                        segments.add(ImmediateFuture.success(last));
                     }
 
                     int index = 0;
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
index 4147ce5..aac0be4 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
@@ -30,10 +30,10 @@ import java.util.concurrent.TimeoutException;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
+import org.apache.cassandra.concurrent.ScheduledExecutorPlus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
@@ -51,6 +51,7 @@ import org.apache.cassandra.utils.MBeanWrapper;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.WrappedRunnable;
 
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 /**
  * Manages the fixed-size memory pool for index summaries, periodically resizing them
  * in order to give more memory to hot sstables and less memory to cold sstables.
@@ -64,7 +65,7 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
     private int resizeIntervalInMinutes = 0;
     private long memoryPoolBytes;
 
-    private final DebuggableScheduledThreadPoolExecutor executor;
+    private final ScheduledExecutorPlus executor;
 
     // our next scheduled resizing run
     private ScheduledFuture future;
@@ -77,7 +78,7 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
 
     private IndexSummaryManager()
     {
-        executor = new DebuggableScheduledThreadPoolExecutor(1, "IndexSummaryManager", Thread.MIN_PRIORITY);
+        executor = executorFactory().scheduled(false, "IndexSummaryManager", Thread.MIN_PRIORITY);
 
         long indexSummarySizeInMB = DatabaseDescriptor.getIndexSummaryCapacityInMB();
         int interval = DatabaseDescriptor.getIndexSummaryResizeIntervalInMinutes();
@@ -278,6 +279,6 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
     @VisibleForTesting
     public void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
     {
-        ExecutorUtils.shutdownAndWait(timeout, unit, executor);
+        ExecutorUtils.shutdownNowAndWait(timeout, unit, executor);
     }
 }
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index aef054e..1d5aaa0 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -22,22 +22,24 @@ import java.io.IOException;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Throwables;
 
 import io.netty.util.concurrent.FastThreadLocalThread;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.RegularAndStaticColumns;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.EncodingStats;
+import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.rows.SerializationHelper;
 import org.apache.cassandra.db.rows.UnfilteredSerializer;
-import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
+import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
 /**
  * A SSTable writer that doesn't assume rows are in sorted order.
@@ -61,7 +63,7 @@ class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
     private final SerializationHeader header;
     private final SerializationHelper helper;
 
-    private final BlockingQueue<Buffer> writeQueue = new SynchronousQueue<Buffer>();
+    private final BlockingQueue<Buffer> writeQueue = newBlockingQueue(0);
     private final DiskWriter diskWriter = new DiskWriter();
 
     SSTableSimpleUnsortedWriter(File directory, TableMetadataRef metadata, RegularAndStaticColumns columns, long bufferSizeInMB)
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index a946e4f..d5dee48 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -30,6 +30,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Ordering;
 import com.google.common.primitives.Longs;
 import com.google.common.util.concurrent.RateLimiter;
+import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,8 +40,8 @@ import com.clearspring.analytics.stream.cardinality.ICardinality;
 
 import org.apache.cassandra.cache.InstrumentingCache;
 import org.apache.cassandra.cache.KeyCacheKey;
-import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.concurrent.ScheduledExecutorPlus;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
@@ -71,6 +72,7 @@ import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.*;
 import org.apache.cassandra.utils.concurrent.*;
 
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR;
 import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
@@ -137,17 +139,19 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 {
     private static final Logger logger = LoggerFactory.getLogger(SSTableReader.class);
 
-    private static final ScheduledThreadPoolExecutor syncExecutor = initSyncExecutor();
-    private static ScheduledThreadPoolExecutor initSyncExecutor()
+    private static final ScheduledExecutorPlus syncExecutor = initSyncExecutor();
+    private static ScheduledExecutorPlus initSyncExecutor()
     {
         if (DatabaseDescriptor.isClientOrToolInitialized())
             return null;
 
         // Do NOT start this thread pool in client mode
 
-        ScheduledThreadPoolExecutor syncExecutor = new ScheduledThreadPoolExecutor(1, new NamedThreadFactory("read-hotness-tracker"));
+        ScheduledExecutorPlus syncExecutor = executorFactory().scheduled("read-hotness-tracker");
         // Immediately remove readMeter sync task when cancelled.
-        syncExecutor.setRemoveOnCancelPolicy(true);
+        // TODO: should we set this by default on all scheduled executors?
+        if (syncExecutor instanceof ScheduledThreadPoolExecutor)
+            ((ScheduledThreadPoolExecutor)syncExecutor).setRemoveOnCancelPolicy(true);
         return syncExecutor;
     }
     private static final RateLimiter meterSyncThrottle = RateLimiter.create(100.0);
@@ -525,37 +529,43 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
     {
         final Collection<SSTableReader> sstables = newBlockingQueue();
 
-        ExecutorService executor = DebuggableThreadPoolExecutor.createWithFixedPoolSize("SSTableBatchOpen", FBUtilities.getAvailableProcessors());
-        for (final Map.Entry<Descriptor, Set<Component>> entry : entries)
+        ExecutorPlus executor = executorFactory().pooled("SSTableBatchOpen", FBUtilities.getAvailableProcessors());
+        try
         {
-            Runnable runnable = new Runnable()
+            for (final Map.Entry<Descriptor, Set<Component>> entry : entries)
             {
-                public void run()
+                Runnable runnable = new Runnable()
                 {
-                    SSTableReader sstable;
-                    try
+                    public void run()
                     {
-                        sstable = open(entry.getKey(), entry.getValue(), metadata);
-                    }
-                    catch (CorruptSSTableException ex)
-                    {
-                        JVMStabilityInspector.inspectThrowable(ex);
-                        logger.error("Corrupt sstable {}; skipping table", entry, ex);
-                        return;
-                    }
-                    catch (FSError ex)
-                    {
-                        JVMStabilityInspector.inspectThrowable(ex);
-                        logger.error("Cannot read sstable {}; file system error, skipping table", entry, ex);
-                        return;
+                        SSTableReader sstable;
+                        try
+                        {
+                            sstable = open(entry.getKey(), entry.getValue(), metadata);
+                        }
+                        catch (CorruptSSTableException ex)
+                        {
+                            JVMStabilityInspector.inspectThrowable(ex);
+                            logger.error("Corrupt sstable {}; skipping table", entry, ex);
+                            return;
+                        }
+                        catch (FSError ex)
+                        {
+                            JVMStabilityInspector.inspectThrowable(ex);
+                            logger.error("Cannot read sstable {}; file system error, skipping table", entry, ex);
+                            return;
+                        }
+                        sstables.add(sstable);
                     }
-                    sstables.add(sstable);
-                }
-            };
-            executor.submit(runnable);
+                };
+                executor.submit(runnable);
+            }
+        }
+        finally
+        {
+            executor.shutdown();
         }
 
-        executor.shutdown();
         try
         {
             executor.awaitTermination(7, TimeUnit.DAYS);
diff --git a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
index 7d0b969..375adc4 100644
--- a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
@@ -18,12 +18,12 @@
 package org.apache.cassandra.metrics;
 
 import java.util.*;
-import java.util.concurrent.ThreadPoolExecutor;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Meter;
 
+import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionInfo;
@@ -62,7 +62,7 @@ public class CompactionMetrics
     /** Total number of compactions which have outright failed due to lack of disk space */
     public final Counter compactionsAborted;
 
-    public CompactionMetrics(final ThreadPoolExecutor... collectors)
+    public CompactionMetrics(final ExecutorPlus... collectors)
     {
         pendingTasks = Metrics.register(factory.createMetricName("PendingTasks"), new Gauge<Integer>()
         {
@@ -137,7 +137,7 @@ public class CompactionMetrics
             public Long getValue()
             {
                 long completedTasks = 0;
-                for (ThreadPoolExecutor collector : collectors)
+                for (ExecutorPlus collector : collectors)
                     completedTasks += collector.getCompletedTaskCount();
                 return completedTasks;
             }
diff --git a/src/java/org/apache/cassandra/metrics/Sampler.java b/src/java/org/apache/cassandra/metrics/Sampler.java
index cfe3f3b..90cc90c 100644
--- a/src/java/org/apache/cassandra/metrics/Sampler.java
+++ b/src/java/org/apache/cassandra/metrics/Sampler.java
@@ -19,18 +19,19 @@ package org.apache.cassandra.metrics;
 
 import java.io.Serializable;
 import java.util.List;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
-import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.Verb;
+import org.apache.cassandra.utils.ExecutorUtils;
 import org.apache.cassandra.utils.MonotonicClock;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
+
 public abstract class Sampler<T>
 {
     public enum SamplerType
@@ -42,19 +43,12 @@ public abstract class Sampler<T>
     MonotonicClock clock = MonotonicClock.approxTime;
 
     @VisibleForTesting
-    static final ThreadPoolExecutor samplerExecutor = new JMXEnabledThreadPoolExecutor(1, 1,
-            TimeUnit.SECONDS,
-            new ArrayBlockingQueue<Runnable>(1000),
-            new NamedThreadFactory("Sampler"),
-            "internal");
-
-    static
-    {
-        samplerExecutor.setRejectedExecutionHandler((runnable, executor) ->
-        {
-            MessagingService.instance().metrics.recordSelfDroppedMessage(Verb._SAMPLE);
-        });
-    }
+    static final ExecutorPlus samplerExecutor = executorFactory()
+            .withJmxInternal()
+            .configureSequential("Sampler")
+            .withQueueLimit(1000)
+            .withRejectedExecutionHandler((runnable, executor) -> MessagingService.instance().metrics.recordSelfDroppedMessage(Verb._SAMPLE))
+            .build();
 
     public void addSample(final T item, final int value)
     {
@@ -94,4 +88,9 @@ public abstract class Sampler<T>
             return "Sample [value=" + value + ", count=" + count + ", error=" + error + "]";
         }
     }
+
+    public static void shutdownNowAndWait(long time, TimeUnit units) throws InterruptedException, TimeoutException
+    {
+        ExecutorUtils.shutdownNowAndWait(time, units, samplerExecutor);
+    }
 }
diff --git a/src/java/org/apache/cassandra/metrics/ThreadPoolMetrics.java b/src/java/org/apache/cassandra/metrics/ThreadPoolMetrics.java
index 3ba984a..62ef177 100644
--- a/src/java/org/apache/cassandra/metrics/ThreadPoolMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ThreadPoolMetrics.java
@@ -21,7 +21,7 @@ import java.util.concurrent.ThreadPoolExecutor;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Gauge;
-import org.apache.cassandra.concurrent.LocalAwareExecutorService;
+import org.apache.cassandra.concurrent.ResizableThreadPool;
 import org.apache.cassandra.metrics.CassandraMetricsRegistry.MetricName;
 
 import static java.lang.String.format;
@@ -75,7 +75,7 @@ public class ThreadPoolMetrics
      * @param path Type of thread pool
      * @param poolName Name of thread pool to identify metrics
      */
-    public ThreadPoolMetrics(LocalAwareExecutorService executor, String path, String poolName)
+    public ThreadPoolMetrics(ResizableThreadPool executor, String path, String poolName)
     {
         this.path = path;
         this.poolName = poolName;
diff --git a/src/java/org/apache/cassandra/net/AsyncChannelPromise.java b/src/java/org/apache/cassandra/net/AsyncChannelPromise.java
index c652b8c..142c3e2 100644
--- a/src/java/org/apache/cassandra/net/AsyncChannelPromise.java
+++ b/src/java/org/apache/cassandra/net/AsyncChannelPromise.java
@@ -31,7 +31,7 @@ import org.apache.cassandra.utils.concurrent.AsyncPromise;
  *
  * This class is all boiler plate, just ensuring we return ourselves and invoke the correct Promise method.
  */
-public class AsyncChannelPromise extends AsyncPromise<Void> implements ChannelPromise
+public class AsyncChannelPromise extends AsyncPromise.WithExecutor<Void> implements ChannelPromise
 {
     private final Channel channel;
 
diff --git a/src/java/org/apache/cassandra/net/FutureCombiner.java b/src/java/org/apache/cassandra/net/FutureCombiner.java
deleted file mode 100644
index 865ca46..0000000
--- a/src/java/org/apache/cassandra/net/FutureCombiner.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.net;
-
-import java.util.Collection;
-import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
-import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
-
-import io.netty.util.concurrent.Future;
-import io.netty.util.concurrent.GenericFutureListener;
-import io.netty.util.concurrent.GlobalEventExecutor;
-import io.netty.util.concurrent.Promise;
-import org.apache.cassandra.utils.concurrent.AsyncPromise;
-
-/**
- * Netty's PromiseCombiner is not threadsafe, and we combine futures from multiple event executors.
- *
- * This class groups a number of Future into a single logical Future, by registering a listener to each that
- * decrements a shared counter; if any of them fail, the FutureCombiner is completed with the first cause,
- * but in all scenario only completes when all underlying future have completed (exceptionally or otherwise)
- *
- * This Future is always uncancellable.
- *
- * We extend FutureDelegate, and simply provide it an uncancellable Promise that will be completed by the listeners
- * registered to the input futures.
- */
-class FutureCombiner extends FutureDelegate<Void>
-{
-    private volatile boolean failed;
-
-    private volatile Throwable firstCause;
-    private static final AtomicReferenceFieldUpdater<FutureCombiner, Throwable> firstCauseUpdater =
-        AtomicReferenceFieldUpdater.newUpdater(FutureCombiner.class, Throwable.class, "firstCause");
-
-    private volatile int waitingOn;
-    private static final AtomicIntegerFieldUpdater<FutureCombiner> waitingOnUpdater =
-        AtomicIntegerFieldUpdater.newUpdater(FutureCombiner.class, "waitingOn");
-
-    FutureCombiner(Collection<? extends Future<?>> combine)
-    {
-        this(AsyncPromise.uncancellable(GlobalEventExecutor.INSTANCE), combine);
-    }
-
-    private FutureCombiner(Promise<Void> combined, Collection<? extends Future<?>> combine)
-    {
-        super(combined);
-
-        if (0 == (waitingOn = combine.size()))
-            combined.trySuccess(null);
-
-        GenericFutureListener<? extends Future<Object>> listener = result ->
-        {
-            if (!result.isSuccess())
-            {
-                firstCauseUpdater.compareAndSet(this, null, result.cause());
-                failed = true;
-            }
-
-            if (0 == waitingOnUpdater.decrementAndGet(this))
-            {
-                if (failed)
-                    combined.tryFailure(firstCause);
-                else
-                    combined.trySuccess(null);
-            }
-        };
-
-        for (Future<?> future : combine)
-            future.addListener(listener);
-    }
-}
diff --git a/src/java/org/apache/cassandra/net/FutureDelegate.java b/src/java/org/apache/cassandra/net/FutureDelegate.java
index f04a432..7e6f9c6 100644
--- a/src/java/org/apache/cassandra/net/FutureDelegate.java
+++ b/src/java/org/apache/cassandra/net/FutureDelegate.java
@@ -29,7 +29,7 @@ import io.netty.util.concurrent.GenericFutureListener;
  *
  * See {@link FutureCombiner} and {@link FutureResult}
  */
-class FutureDelegate<V> implements Future<V>
+public class FutureDelegate<V> implements Future<V>
 {
     final Future<V> delegate;
 
@@ -53,26 +53,6 @@ class FutureDelegate<V> implements Future<V>
         return delegate.cause();
     }
 
-    public Future<V> addListener(GenericFutureListener<? extends Future<? super V>> genericFutureListener)
-    {
-        return delegate.addListener(genericFutureListener);
-    }
-
-    public Future<V> addListeners(GenericFutureListener<? extends Future<? super V>>... genericFutureListeners)
-    {
-        return delegate.addListeners(genericFutureListeners);
-    }
-
-    public Future<V> removeListener(GenericFutureListener<? extends Future<? super V>> genericFutureListener)
-    {
-        return delegate.removeListener(genericFutureListener);
-    }
-
-    public Future<V> removeListeners(GenericFutureListener<? extends Future<? super V>>... genericFutureListeners)
-    {
-        return delegate.removeListeners(genericFutureListeners);
-    }
-
     public Future<V> sync() throws InterruptedException
     {
         return delegate.sync();
@@ -142,4 +122,28 @@ class FutureDelegate<V> implements Future<V>
     {
         return delegate.get(timeout, unit);
     }
+
+    @Override
+    public io.netty.util.concurrent.Future<V> addListener(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>> genericFutureListener)
+    {
+        return delegate.addListener(genericFutureListener);
+    }
+
+    @Override
+    public io.netty.util.concurrent.Future<V> addListeners(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>>... genericFutureListeners)
+    {
+        return delegate.addListeners(genericFutureListeners);
+    }
+
+    @Override
+    public io.netty.util.concurrent.Future<V> removeListener(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>> genericFutureListener)
+    {
+        return delegate.removeListener(genericFutureListener);
+    }
+
+    @Override
+    public io.netty.util.concurrent.Future<V> removeListeners(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>>... genericFutureListeners)
+    {
+        return delegate.removeListeners(genericFutureListeners);
+    }
 }
diff --git a/src/java/org/apache/cassandra/net/FutureResult.java b/src/java/org/apache/cassandra/net/FutureResult.java
index 8d43dbe..8ad2e7b 100644
--- a/src/java/org/apache/cassandra/net/FutureResult.java
+++ b/src/java/org/apache/cassandra/net/FutureResult.java
@@ -21,10 +21,9 @@ import io.netty.util.concurrent.Future;
 
 /**
  * An abstraction for yielding a result performed by an asynchronous task,
- * for whom we may wish to offer cancellation,
- * but no other access to the underlying task
+ * for whom we may wish to offer cancellation, but no other access to the underlying task
  */
-class FutureResult<V> extends FutureDelegate<V>
+public class FutureResult<V> extends FutureDelegate<V>
 {
     private final Future<?> tryCancel;
 
@@ -32,7 +31,7 @@ class FutureResult<V> extends FutureDelegate<V>
      * @param result the Future that will be completed by {@link #cancel}
      * @param cancel the Future that is performing the work, and to whom any cancellation attempts will be proxied
      */
-    FutureResult(Future<V> result, Future<?> cancel)
+    public FutureResult(Future<V> result, Future<?> cancel)
     {
         super(result);
         this.tryCancel = cancel;
diff --git a/src/java/org/apache/cassandra/net/InboundMessageHandler.java b/src/java/org/apache/cassandra/net/InboundMessageHandler.java
index f29b3ec..c1b51be 100644
--- a/src/java/org/apache/cassandra/net/InboundMessageHandler.java
+++ b/src/java/org/apache/cassandra/net/InboundMessageHandler.java
@@ -396,7 +396,7 @@ public class InboundMessageHandler extends AbstractMessageHandler
         if (state != null) state.trace("{} message received from {}", header.verb, header.from);
 
         callbacks.onDispatched(task.size(), header);
-        header.verb.stage.execute(task, ExecutorLocals.create(state));
+        header.verb.stage.execute(ExecutorLocals.create(state), task);
     }
 
     private abstract class ProcessMessage implements Runnable
diff --git a/src/java/org/apache/cassandra/net/InboundSockets.java b/src/java/org/apache/cassandra/net/InboundSockets.java
index d1a4f7b..d1f76d3 100644
--- a/src/java/org/apache/cassandra/net/InboundSockets.java
+++ b/src/java/org/apache/cassandra/net/InboundSockets.java
@@ -39,6 +39,7 @@ import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.AsyncPromise;
+import org.apache.cassandra.utils.concurrent.FutureCombiner;
 
 class InboundSockets
 {
@@ -126,7 +127,7 @@ class InboundSockets
                 if (listen != null)
                     closing.add(listen.close());
                 closing.add(connections.close());
-                new FutureCombiner(closing)
+                FutureCombiner.nettySuccessListener(closing)
                        .addListener(future -> {
                            executor.shutdownGracefully();
                            shutdownExecutors.accept(executor);
@@ -225,7 +226,7 @@ class InboundSockets
         for (InboundSocket socket : sockets)
             opening.add(socket.open(pipelineInjector));
 
-        return new FutureCombiner(opening);
+        return FutureCombiner.nettySuccessListener(opening);
     }
 
     public Future<Void> open()
@@ -233,7 +234,7 @@ class InboundSockets
         List<Future<Void>> opening = new ArrayList<>();
         for (InboundSocket socket : sockets)
             opening.add(socket.open());
-        return new FutureCombiner(opening);
+        return FutureCombiner.nettySuccessListener(opening);
     }
 
     public boolean isListening()
@@ -249,7 +250,7 @@ class InboundSockets
         List<Future<Void>> closing = new ArrayList<>();
         for (InboundSocket address : sockets)
             closing.add(address.close(shutdownExecutors));
-        return new FutureCombiner(closing);
+        return FutureCombiner.nettySuccessListener(closing);
     }
     public Future<Void> close()
     {
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index 747d740..4359094 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -26,6 +26,7 @@ import java.util.concurrent.TimeoutException;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import org.apache.cassandra.utils.concurrent.FutureCombiner;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -451,7 +452,7 @@ public final class MessagingService extends MessagingServiceMBeanImpl
                 closing.add(pool.close(true));
 
             long deadline = nanoTime() + units.toNanos(timeout);
-            maybeFail(() -> new FutureCombiner(closing).get(timeout, units),
+            maybeFail(() -> FutureCombiner.nettySuccessListener(closing).get(timeout, units),
                       () -> {
                           List<ExecutorService> inboundExecutors = new ArrayList<>();
                           inboundSockets.close(synchronizedList(inboundExecutors)::add).get();
@@ -475,7 +476,7 @@ public final class MessagingService extends MessagingServiceMBeanImpl
                 closing.add(pool.close(false));
 
             long deadline = nanoTime() + units.toNanos(timeout);
-            maybeFail(() -> new FutureCombiner(closing).get(timeout, units),
+            maybeFail(() -> FutureCombiner.nettySuccessListener(closing).get(timeout, units),
                       () -> {
                           if (shutdownExecutors)
                               shutdownExecutors(deadline);
diff --git a/src/java/org/apache/cassandra/net/OutboundConnection.java b/src/java/org/apache/cassandra/net/OutboundConnection.java
index 0179166..ca66ed0 100644
--- a/src/java/org/apache/cassandra/net/OutboundConnection.java
+++ b/src/java/org/apache/cassandra/net/OutboundConnection.java
@@ -1100,7 +1100,7 @@ public class OutboundConnection
 
                 if (hasPending())
                 {
-                    Promise<Result<MessagingSuccess>> result = new AsyncPromise<>(eventLoop);
+                    Promise<Result<MessagingSuccess>> result = AsyncPromise.withExecutor(eventLoop);
                     state = new Connecting(state.disconnected(), result, eventLoop.schedule(() -> attempt(result), max(100, retryRateMillis), MILLISECONDS));
                     retryRateMillis = min(1000, retryRateMillis * 2);
                 }
@@ -1229,7 +1229,7 @@ public class OutboundConnection
 
             Future<Result<MessagingSuccess>> initiate()
             {
-                Promise<Result<MessagingSuccess>> result = new AsyncPromise<>(eventLoop);
+                Promise<Result<MessagingSuccess>> result = AsyncPromise.withExecutor(eventLoop);
                 state = new Connecting(state.disconnected(), result);
                 attempt(result);
                 return result;
diff --git a/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java b/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java
index 71eb132..dad8526 100644
--- a/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java
+++ b/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java
@@ -24,7 +24,10 @@ import java.nio.channels.ClosedChannelException;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.Promise;
 import org.apache.cassandra.utils.concurrent.AsyncPromise;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -46,9 +49,6 @@ import io.netty.handler.logging.LoggingHandler;
 import io.netty.handler.ssl.SslClosedEngineException;
 import io.netty.handler.ssl.SslContext;
 import io.netty.handler.ssl.SslHandler;
-import io.netty.util.concurrent.FailedFuture;
-import io.netty.util.concurrent.Future;
-import io.netty.util.concurrent.Promise;
 import io.netty.util.concurrent.ScheduledFuture;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.OutboundConnectionInitiator.Result.MessagingSuccess;
@@ -108,7 +108,7 @@ public class OutboundConnectionInitiator<SuccessType extends OutboundConnectionI
      */
     public static Future<Result<StreamingSuccess>> initiateStreaming(EventLoop eventLoop, OutboundConnectionSettings settings, int requestMessagingVersion)
     {
-        return new OutboundConnectionInitiator<StreamingSuccess>(STREAMING, settings, requestMessagingVersion, new AsyncPromise<>(eventLoop))
+        return new OutboundConnectionInitiator<StreamingSuccess>(STREAMING, settings, requestMessagingVersion, AsyncPromise.withExecutor(eventLoop))
                .initiate(eventLoop);
     }
 
@@ -134,13 +134,13 @@ public class OutboundConnectionInitiator<SuccessType extends OutboundConnectionI
         {
             // interrupt other connections, so they must attempt to re-authenticate
             MessagingService.instance().interruptOutbound(settings.to);
-            return new FailedFuture<>(eventLoop, new IOException("authentication failed to " + settings.connectToId()));
+            return ImmediateFuture.failure(new IOException("authentication failed to " + settings.connectToId()));
         }
 
         // this is a bit ugly, but is the easiest way to ensure that if we timeout we can propagate a suitable error message
         // and still guarantee that, if on timing out we raced with success, the successfully created channel is handled
         AtomicBoolean timedout = new AtomicBoolean();
-        Future<Void> bootstrap = createBootstrap(eventLoop)
+        io.netty.util.concurrent.Future<Void> bootstrap = createBootstrap(eventLoop)
                                  .connect()
                                  .addListener(future -> {
                                      eventLoop.execute(() -> {
diff --git a/src/java/org/apache/cassandra/net/OutboundConnections.java b/src/java/org/apache/cassandra/net/OutboundConnections.java
index 68157c3..9808e3f 100644
--- a/src/java/org/apache/cassandra/net/OutboundConnections.java
+++ b/src/java/org/apache/cassandra/net/OutboundConnections.java
@@ -27,6 +27,7 @@ import java.util.function.Function;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 
+import org.apache.cassandra.utils.concurrent.FutureCombiner;
 import org.apache.cassandra.utils.concurrent.Condition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -125,7 +126,7 @@ public class OutboundConnections
     synchronized Future<Void> reconnectWithNewIp(InetAddressAndPort addr)
     {
         template = template.withConnectTo(addr);
-        return new FutureCombiner(
+        return FutureCombiner.nettySuccessListener(
             apply(c -> c.reconnectWith(template))
         );
     }
@@ -139,7 +140,7 @@ public class OutboundConnections
     {
         // immediately release our metrics, so that if we need to re-open immediately we can safely register a new one
         releaseMetrics();
-        return new FutureCombiner(
+        return FutureCombiner.nettySuccessListener(
             apply(c -> c.scheduleClose(time, unit, flushQueues))
         );
     }
@@ -153,7 +154,7 @@ public class OutboundConnections
     {
         // immediately release our metrics, so that if we need to re-open immediately we can safely register a new one
         releaseMetrics();
-        return new FutureCombiner(
+        return FutureCombiner.nettySuccessListener(
             apply(c -> c.close(flushQueues))
         );
     }
diff --git a/src/java/org/apache/cassandra/net/RequestCallbacks.java b/src/java/org/apache/cassandra/net/RequestCallbacks.java
index 9adec9b..8cec096 100644
--- a/src/java/org/apache/cassandra/net/RequestCallbacks.java
+++ b/src/java/org/apache/cassandra/net/RequestCallbacks.java
@@ -20,17 +20,16 @@ package org.apache.cassandra.net;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeoutException;
 
 import javax.annotation.Nullable;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import org.apache.cassandra.concurrent.ScheduledExecutorPlus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Mutation;
@@ -47,6 +46,7 @@ import org.apache.cassandra.utils.FBUtilities;
 import static java.lang.String.format;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.concurrent.Stage.INTERNAL_RESPONSE;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
 import static org.apache.cassandra.utils.MonotonicClock.preciseTime;
@@ -65,7 +65,7 @@ public class RequestCallbacks implements OutboundMessageCallbacks
     private static final Logger logger = LoggerFactory.getLogger(RequestCallbacks.class);
 
     private final MessagingService messagingService;
-    private final ScheduledExecutorService executor = new DebuggableScheduledThreadPoolExecutor("Callback-Map-Reaper");
+    private final ScheduledExecutorPlus executor = executorFactory().scheduled("Callback-Map-Reaper");
     private final ConcurrentMap<CallbackKey, CallbackInfo> callbacks = new ConcurrentHashMap<>();
 
     RequestCallbacks(MessagingService messagingService)
diff --git a/src/java/org/apache/cassandra/net/SocketFactory.java b/src/java/org/apache/cassandra/net/SocketFactory.java
index 8300c2a..f97e2fe 100644
--- a/src/java/org/apache/cassandra/net/SocketFactory.java
+++ b/src/java/org/apache/cassandra/net/SocketFactory.java
@@ -24,7 +24,6 @@ import java.nio.channels.ClosedChannelException;
 import java.nio.channels.spi.SelectorProvider;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeoutException;
 import javax.annotation.Nullable;
@@ -72,6 +71,7 @@ import org.apache.cassandra.utils.FBUtilities;
 import static io.netty.channel.unix.Errors.ERRNO_ECONNRESET_NEGATIVE;
 import static io.netty.channel.unix.Errors.ERROR_ECONNREFUSED_NEGATIVE;
 import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.utils.Throwables.isCausedBy;
 
 /**
@@ -177,7 +177,7 @@ public final class SocketFactory
     private final EventLoopGroup defaultGroup;
     // we need a separate EventLoopGroup for outbound streaming because sendFile is blocking
     private final EventLoopGroup outboundStreamingGroup;
-    final ExecutorService synchronousWorkExecutor = Executors.newCachedThreadPool(new NamedThreadFactory("Messaging-SynchronousWork"));
+    final ExecutorService synchronousWorkExecutor = executorFactory().pooled("Messaging-SynchronousWork", Integer.MAX_VALUE);
 
     SocketFactory()
     {
diff --git a/src/java/org/apache/cassandra/repair/KeyspaceRepairManager.java b/src/java/org/apache/cassandra/repair/KeyspaceRepairManager.java
index 0739f10..c8131cb 100644
--- a/src/java/org/apache/cassandra/repair/KeyspaceRepairManager.java
+++ b/src/java/org/apache/cassandra/repair/KeyspaceRepairManager.java
@@ -19,14 +19,14 @@
 package org.apache.cassandra.repair;
 
 import java.util.Collection;
+import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 import java.util.function.BooleanSupplier;
 
-import com.google.common.util.concurrent.ListenableFuture;
-
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.utils.concurrent.Future;
 
 /**
  * Keyspace level hook for repair.
@@ -38,9 +38,9 @@ public interface KeyspaceRepairManager
      * been notified that the repair session has been completed, the data associated with the given session id must
      * not be combined with repaired or unrepaired data, or data from other repair sessions.
      */
-    ListenableFuture prepareIncrementalRepair(UUID sessionID,
-                                              Collection<ColumnFamilyStore> tables,
-                                              RangesAtEndpoint tokenRanges,
-                                              ExecutorService executor,
-                                              BooleanSupplier isCancelled);
+    Future<List<Void>> prepareIncrementalRepair(UUID sessionID,
+                                                Collection<ColumnFamilyStore> tables,
+                                                RangesAtEndpoint tokenRanges,
+                                                ExecutorService executor,
+                                                BooleanSupplier isCancelled);
 }
diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java
index 7747e18..1e203c4 100644
--- a/src/java/org/apache/cassandra/repair/RepairJob.java
+++ b/src/java/org/apache/cassandra/repair/RepairJob.java
@@ -26,6 +26,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.*;
+import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.apache.cassandra.utils.concurrent.AsyncFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -46,6 +47,9 @@ import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.MerkleTrees;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.concurrent.Future;
+import org.apache.cassandra.utils.concurrent.FutureCombiner;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
 
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
 
@@ -59,8 +63,8 @@ public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
     private final RepairSession session;
     private final RepairJobDesc desc;
     private final RepairParallelism parallelismDegree;
-    private final ListeningExecutorService taskExecutor;
-    
+    private final ExecutorPlus taskExecutor;
+
     @VisibleForTesting
     final List<ValidationTask> validationTasks = new ArrayList<>();
 
@@ -106,39 +110,35 @@ public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
         List<InetAddressAndPort> allEndpoints = new ArrayList<>(session.commonRange.endpoints);
         allEndpoints.add(FBUtilities.getBroadcastAddressAndPort());
 
-        ListenableFuture<List<TreeResponse>> treeResponses;
+        Future<List<TreeResponse>> treeResponses;
         // Create a snapshot at all nodes unless we're using pure parallel repairs
         if (parallelismDegree != RepairParallelism.PARALLEL)
         {
-            ListenableFuture<List<InetAddressAndPort>> allSnapshotTasks;
+            Future<List<InetAddressAndPort>> allSnapshotTasks;
             if (session.isIncremental)
             {
                 // consistent repair does it's own "snapshotting"
-                allSnapshotTasks = Futures.immediateFuture(allEndpoints);
+                allSnapshotTasks = ImmediateFuture.success(allEndpoints);
             }
             else
             {
                 // Request snapshot to all replica
-                List<ListenableFuture<InetAddressAndPort>> snapshotTasks = new ArrayList<>(allEndpoints.size());
+                List<Future<InetAddressAndPort>> snapshotTasks = new ArrayList<>(allEndpoints.size());
                 for (InetAddressAndPort endpoint : allEndpoints)
                 {
                     SnapshotTask snapshotTask = new SnapshotTask(desc, endpoint);
                     snapshotTasks.add(snapshotTask);
                     taskExecutor.execute(snapshotTask);
                 }
-                allSnapshotTasks = Futures.allAsList(snapshotTasks);
+                allSnapshotTasks = FutureCombiner.allOf(snapshotTasks);
             }
 
             // When all snapshot complete, send validation requests
-            treeResponses = Futures.transformAsync(allSnapshotTasks, new AsyncFunction<List<InetAddressAndPort>, List<TreeResponse>>()
-            {
-                public ListenableFuture<List<TreeResponse>> apply(List<InetAddressAndPort> endpoints)
-                {
-                    if (parallelismDegree == RepairParallelism.SEQUENTIAL)
-                        return sendSequentialValidationRequest(endpoints);
-                    else
-                        return sendDCAwareValidationRequest(endpoints);
-                }
+            treeResponses = allSnapshotTasks.andThenAsync(endpoints -> {
+                if (parallelismDegree == RepairParallelism.SEQUENTIAL)
+                    return sendSequentialValidationRequest(endpoints);
+                else
+                    return sendDCAwareValidationRequest(endpoints);
             }, taskExecutor);
         }
         else
@@ -148,12 +148,10 @@ public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
         }
 
         // When all validations complete, submit sync tasks
-        ListenableFuture<List<SyncStat>> syncResults = Futures.transformAsync(treeResponses,
-                                                                              session.optimiseStreams && !session.pullRepair ? this::optimisedSyncing : this::standardSyncing,
-                                                                              taskExecutor);
+        Future<List<SyncStat>> syncResults = treeResponses.andThenAsync(session.optimiseStreams && !session.pullRepair ? this::optimisedSyncing : this::standardSyncing, taskExecutor);
 
         // When all sync complete, set the final result
-        Futures.addCallback(syncResults, new FutureCallback<List<SyncStat>>()
+        syncResults.addCallback(new FutureCallback<List<SyncStat>>()
         {
             public void onSuccess(List<SyncStat> stats)
             {
@@ -180,7 +178,9 @@ public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
                     SystemDistributedKeyspace.failedRepairJob(session.getId(), desc.keyspace, desc.columnFamily, t);
                 }
                 cfs.metric.repairsCompleted.inc();
-                tryFailure(t);
+                tryFailure(t instanceof NoSuchRepairSessionExceptionWrapper
+                           ? ((NoSuchRepairSessionExceptionWrapper) t).wrapped
+                           : t);
             }
         }, taskExecutor);
     }
@@ -190,7 +190,7 @@ public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
         return session.commonRange.transEndpoints.contains(ep);
     }
 
-    private ListenableFuture<List<SyncStat>> standardSyncing(List<TreeResponse> trees) throws NoSuchRepairSessionException
+    private Future<List<SyncStat>> standardSyncing(List<TreeResponse> trees)
     {
         List<SyncTask> syncTasks = createStandardSyncTasks(desc,
                                                            trees,
@@ -269,7 +269,7 @@ public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
         return syncTasks;
     }
 
-    private ListenableFuture<List<SyncStat>> optimisedSyncing(List<TreeResponse> trees) throws NoSuchRepairSessionException
+    private Future<List<SyncStat>> optimisedSyncing(List<TreeResponse> trees)
     {
         List<SyncTask> syncTasks = createOptimisedSyncingSyncTasks(desc,
                                                                    trees,
@@ -284,17 +284,39 @@ public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
 
     @SuppressWarnings("UnstableApiUsage")
     @VisibleForTesting
-    ListenableFuture<List<SyncStat>> executeTasks(List<SyncTask> syncTasks) throws NoSuchRepairSessionException
+    Future<List<SyncStat>> executeTasks(List<SyncTask> syncTasks)
     {
-        ActiveRepairService.instance.getParentRepairSession(desc.parentSessionId);
-        for (SyncTask task : syncTasks)
+        try
         {
-            if (!task.isLocal())
-                session.trackSyncCompletion(Pair.create(desc, task.nodePair()), (CompletableRemoteSyncTask) task);
-            taskExecutor.submit(task);
+            ActiveRepairService.instance.getParentRepairSession(desc.parentSessionId);
+            for (SyncTask task : syncTasks)
+            {
+                if (!task.isLocal())
+                    session.trackSyncCompletion(Pair.create(desc, task.nodePair()), (CompletableRemoteSyncTask) task);
+                taskExecutor.execute(task);
+            }
+
+            return FutureCombiner.allOf(syncTasks);
+        }
+        catch (NoSuchRepairSessionException e)
+        {
+            throw new NoSuchRepairSessionExceptionWrapper(e);
         }
+    }
 
-        return Futures.allAsList(syncTasks);
+    // provided so we can throw NoSuchRepairSessionException from executeTasks without
+    // having to make it unchecked. Required as this is called as from standardSyncing/
+    // optimisedSyncing passed as a Function to transform merkle tree responses and so
+    // can't throw checked exceptions. These are unwrapped in the onFailure callback of
+    // that transformation so as to not pollute the checked usage of
+    // NoSuchRepairSessionException in the rest of the codebase.
+    private static class NoSuchRepairSessionExceptionWrapper extends RuntimeException
+    {
+        private final NoSuchRepairSessionException wrapped;
+        private NoSuchRepairSessionExceptionWrapper(NoSuchRepairSessionException wrapped)
+        {
+            this.wrapped = wrapped;
+        }
     }
 
     static List<SyncTask> createOptimisedSyncingSyncTasks(RepairJobDesc desc,
@@ -372,13 +394,13 @@ public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
      * @param endpoints Endpoint addresses to send validation request
      * @return Future that can get all {@link TreeResponse} from replica, if all validation succeed.
      */
-    private ListenableFuture<List<TreeResponse>> sendValidationRequest(Collection<InetAddressAndPort> endpoints)
+    private Future<List<TreeResponse>> sendValidationRequest(Collection<InetAddressAndPort> endpoints)
     {
         String message = String.format("Requesting merkle trees for %s (to %s)", desc.columnFamily, endpoints);
         logger.info("{} {}", session.previewKind.logPrefix(desc.sessionId), message);
         Tracing.traceRepair(message);
         int nowInSec = getNowInSeconds();
-        List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
+        List<Future<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
         for (InetAddressAndPort endpoint : endpoints)
         {
             ValidationTask task = newValidationTask(endpoint, nowInSec);
@@ -386,19 +408,19 @@ public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
             session.trackValidationCompletion(Pair.create(desc, endpoint), task);
             taskExecutor.execute(task);
         }
-        return Futures.allAsList(tasks);
+        return FutureCombiner.allOf(tasks);
     }
 
     /**
      * Creates {@link ValidationTask} and submit them to task executor so that tasks run sequentially.
      */
-    private ListenableFuture<List<TreeResponse>> sendSequentialValidationRequest(Collection<InetAddressAndPort> endpoints)
+    private Future<List<TreeResponse>> sendSequentialValidationRequest(Collection<InetAddressAndPort> endpoints)
     {
         String message = String.format("Requesting merkle trees for %s (to %s)", desc.columnFamily, endpoints);
         logger.info("{} {}", session.previewKind.logPrefix(desc.sessionId), message);
         Tracing.traceRepair(message);
         int nowInSec = getNowInSeconds();
-        List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
+        List<Future<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
 
         Queue<InetAddressAndPort> requests = new LinkedList<>(endpoints);
         InetAddressAndPort address = requests.poll();
@@ -412,7 +434,7 @@ public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
             final InetAddressAndPort nextAddress = requests.poll();
             final ValidationTask nextTask = newValidationTask(nextAddress, nowInSec);
             tasks.add(nextTask);
-            Futures.addCallback(currentTask, new FutureCallback<TreeResponse>()
+            currentTask.addCallback(new FutureCallback<TreeResponse>()
             {
                 public void onSuccess(TreeResponse result)
                 {
@@ -423,24 +445,24 @@ public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
 
                 // failure is handled at root of job chain
                 public void onFailure(Throwable t) {}
-            }, MoreExecutors.directExecutor());
+            });
             currentTask = nextTask;
         }
         // start running tasks
         taskExecutor.execute(firstTask);
-        return Futures.allAsList(tasks);
+        return FutureCombiner.allOf(tasks);
     }
 
     /**
      * Creates {@link ValidationTask} and submit them to task executor so that tasks run sequentially within each dc.
      */
-    private ListenableFuture<List<TreeResponse>> sendDCAwareValidationRequest(Collection<InetAddressAndPort> endpoints)
+    private Future<List<TreeResponse>> sendDCAwareValidationRequest(Collection<InetAddressAndPort> endpoints)
     {
         String message = String.format("Requesting merkle trees for %s (to %s)", desc.columnFamily, endpoints);
         logger.info("{} {}", session.previewKind.logPrefix(desc.sessionId), message);
         Tracing.traceRepair(message);
         int nowInSec = getNowInSeconds();
-        List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
+        List<Future<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
 
         Map<String, Queue<InetAddressAndPort>> requestsByDatacenter = new HashMap<>();
         for (InetAddressAndPort endpoint : endpoints)
@@ -469,7 +491,7 @@ public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
                 final InetAddressAndPort nextAddress = requests.poll();
                 final ValidationTask nextTask = newValidationTask(nextAddress, nowInSec);
                 tasks.add(nextTask);
-                Futures.addCallback(currentTask, new FutureCallback<TreeResponse>()
+                currentTask.addCallback(new FutureCallback<TreeResponse>()
                 {
                     public void onSuccess(TreeResponse result)
                     {
@@ -480,13 +502,13 @@ public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
 
                     // failure is handled at root of job chain
                     public void onFailure(Throwable t) {}
-                }, MoreExecutors.directExecutor());
+                });
                 currentTask = nextTask;
             }
             // start running tasks
             taskExecutor.execute(firstTask);
         }
-        return Futures.allAsList(tasks);
+        return FutureCombiner.allOf(tasks);
     }
 
     private ValidationTask newValidationTask(InetAddressAndPort endpoint, int nowInSec)
diff --git a/src/java/org/apache/cassandra/repair/RepairRunnable.java b/src/java/org/apache/cassandra/repair/RepairRunnable.java
index 3a9de96..faa0a74 100644
--- a/src/java/org/apache/cassandra/repair/RepairRunnable.java
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@ -19,14 +19,7 @@ package org.apache.cassandra.repair;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
+import java.util.*;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -34,25 +27,14 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.AsyncFunction;
+import com.google.common.collect.*;
 import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
 import org.apache.commons.lang3.time.DurationFormatUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.codahale.metrics.Timer;
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
-import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.RepairException;
 import org.apache.cassandra.metrics.RepairMetrics;
@@ -65,20 +47,21 @@ import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.locator.EndpointsForRange;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.metrics.RepairMetrics;
 import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.repair.consistent.CoordinatorSession;
+import org.apache.cassandra.repair.consistent.SyncStatSummary;
 import org.apache.cassandra.repair.messages.RepairOption;
 import org.apache.cassandra.schema.SchemaConstants;
-import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.service.*;
 import org.apache.cassandra.service.ActiveRepairService.ParentRepairStatus;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.tracing.TraceKeyspace;
 import org.apache.cassandra.tracing.TraceState;
@@ -98,8 +81,11 @@ import org.apache.cassandra.utils.progress.ProgressListener;
 import static org.apache.cassandra.service.QueryState.forInternalCalls;
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
-import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
+import org.apache.cassandra.utils.concurrent.Future;
+import org.apache.cassandra.utils.concurrent.FutureCombiner;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
+
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 
 public class RepairRunnable implements Runnable, ProgressEventNotifier
 {
@@ -238,7 +224,7 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
         {
             for (ProgressListener listener : listeners)
                 localState.removeProgressListener(listener);
-            // Because DebuggableThreadPoolExecutor#afterExecute and this callback
+            // Because ExecutorPlus#afterExecute and this callback
             // run in a nondeterministic order (within the same thread), the
             // TraceState may have been nulled out at this point. The TraceState
             // should be traceState, so just set it without bothering to check if it
@@ -316,7 +302,6 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
             traceState.addProgressListener(listener);
         Thread queryThread = createQueryThread(sessionId);
         queryThread.setName("RepairTracePolling");
-        queryThread.start();
         return traceState;
     }
 
@@ -448,48 +433,40 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
     {
 
         // Set up RepairJob executor for this repair command.
-        ListeningExecutorService executor = createExecutor();
+        ExecutorPlus executor = createExecutor();
 
         // Setting the repairedAt time to UNREPAIRED_SSTABLE causes the repairedAt times to be preserved across streamed sstables
-        final ListenableFuture<List<RepairSessionResult>> allSessions = submitRepairSessions(parentSession, false, executor, commonRanges, cfnames);
+        final Future<List<RepairSessionResult>> allSessions = submitRepairSessions(parentSession, false, executor, commonRanges, cfnames);
 
         // After all repair sessions completes(successful or not),
         // run anticompaction if necessary and send finish notice back to client
         final Collection<Range<Token>> successfulRanges = new ArrayList<>();
         final AtomicBoolean hasFailure = new AtomicBoolean();
-        ListenableFuture repairResult = Futures.transformAsync(allSessions, new AsyncFunction<List<RepairSessionResult>, Object>()
-        {
-            @SuppressWarnings("unchecked")
-            public ListenableFuture apply(List<RepairSessionResult> results)
+        allSessions.andThenAsync(results -> {
+            logger.debug("Repair result: {}", results);
+            // filter out null(=failed) results and get successful ranges
+            for (RepairSessionResult sessionResult : results)
             {
-                logger.debug("Repair result: {}", results);
-                // filter out null(=failed) results and get successful ranges
-                for (RepairSessionResult sessionResult : results)
+                if (sessionResult != null)
                 {
-                    if (sessionResult != null)
+                    // don't record successful repair if we had to skip ranges
+                    if (!sessionResult.skippedReplicas)
                     {
-                        // don't record successful repair if we had to skip ranges
-                        if (!sessionResult.skippedReplicas)
-                        {
-                            successfulRanges.addAll(sessionResult.ranges);
-                        }
-                    }
-                    else
-                    {
-                        hasFailure.compareAndSet(false, true);
+                        successfulRanges.addAll(sessionResult.ranges);
                     }
                 }
-                return Futures.immediateFuture(null);
+                else
+                {
+                    hasFailure.compareAndSet(false, true);
+                }
             }
-        }, MoreExecutors.directExecutor());
-        Futures.addCallback(repairResult,
-                            new RepairCompleteCallback(parentSession,
-                                                       successfulRanges,
-                                                       preparedEndpoints,
-                                                       traceState,
-                                                       hasFailure,
-                                                       executor),
-                            MoreExecutors.directExecutor());
+            return ImmediateFuture.success(null);
+        }).addCallback(new RepairCompleteCallback(parentSession,
+                                                  successfulRanges,
+                                                  preparedEndpoints,
+                                                  traceState,
+                                                  hasFailure,
+                                                  executor));
     }
 
     private void incrementalRepair(UUID parentSession,
@@ -517,18 +494,16 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
             fail(e.getMessage());
             return;
         }
-        ListeningExecutorService executor = createExecutor();
+        ExecutorPlus executor = createExecutor();
         AtomicBoolean hasFailure = new AtomicBoolean(false);
-        ListenableFuture repairResult = coordinatorSession.execute(() -> submitRepairSessions(parentSession, true, executor, allRanges, cfnames),
+        Future<?> repairResult = coordinatorSession.execute(() -> submitRepairSessions(parentSession, true, executor, allRanges, cfnames),
                                                                    hasFailure);
         Collection<Range<Token>> ranges = new HashSet<>();
         for (Collection<Range<Token>> range : Iterables.transform(allRanges, cr -> cr.ranges))
         {
             ranges.addAll(range);
         }
-        Futures.addCallback(repairResult,
-                            new RepairCompleteCallback(parentSession, ranges, preparedEndpoints, traceState, hasFailure, executor),
-                            MoreExecutors.directExecutor());
+        repairResult.addCallback(new RepairCompleteCallback(parentSession, ranges, preparedEndpoints, traceState, hasFailure, executor));
     }
 
     private void previewRepair(UUID parentSession,
@@ -539,11 +514,11 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
 
         logger.debug("Starting preview repair for {}", parentSession);
         // Set up RepairJob executor for this repair command.
-        ListeningExecutorService executor = createExecutor();
+        ExecutorPlus executor = createExecutor();
 
-        final ListenableFuture<List<RepairSessionResult>> allSessions = submitRepairSessions(parentSession, false, executor, commonRanges, cfnames);
+        final Future<List<RepairSessionResult>> allSessions = submitRepairSessions(parentSession, false, executor, commonRanges, cfnames);
 
-        Futures.addCallback(allSessions, new FutureCallback<List<RepairSessionResult>>()
+        allSessions.addCallback(new FutureCallback<List<RepairSessionResult>>()
         {
             public void onSuccess(List<RepairSessionResult> results)
             {
@@ -594,7 +569,7 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
                 fail("Error completing preview repair: " + t.getMessage());
                 executor.shutdownNow();
             }
-        }, MoreExecutors.directExecutor());
+        });
     }
 
     private void maybeSnapshotReplicas(UUID parentSession, String keyspace, List<RepairSessionResult> results)
@@ -653,13 +628,13 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
         return iter;
     }
 
-    private ListenableFuture<List<RepairSessionResult>> submitRepairSessions(UUID parentSession,
-                                                                             boolean isIncremental,
-                                                                             ListeningExecutorService executor,
-                                                                             List<CommonRange> commonRanges,
-                                                                             String... cfnames)
+    private Future<List<RepairSessionResult>> submitRepairSessions(UUID parentSession,
+                                                                   boolean isIncremental,
+                                                                   ExecutorPlus executor,
+                                                                   List<CommonRange> commonRanges,
+                                                                   String... cfnames)
     {
-        List<ListenableFuture<RepairSessionResult>> futures = new ArrayList<>(options.getRanges().size());
+        List<Future<RepairSessionResult>> futures = new ArrayList<>(options.getRanges().size());
 
         for (CommonRange commonRange : commonRanges)
         {
@@ -676,20 +651,18 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
                                                                                      cfnames);
             if (session == null)
                 continue;
-            Futures.addCallback(session, new RepairSessionCallback(session), MoreExecutors.directExecutor());
+            session.addCallback(new RepairSessionCallback(session));
             futures.add(session);
         }
-        return Futures.successfulAsList(futures);
+        return FutureCombiner.successfulOf(futures);
     }
 
-    private ListeningExecutorService createExecutor()
+    private ExecutorPlus createExecutor()
     {
-        return MoreExecutors.listeningDecorator(new JMXEnabledThreadPoolExecutor(options.getJobThreads(),
-                                                                                 Integer.MAX_VALUE,
-                                                                                 SECONDS,
-                                                                                 newBlockingQueue(),
-                                                                                 new NamedThreadFactory("Repair#" + cmd),
-                                                                                 "internal"));
+        return executorFactory()
+                .localAware()
+                .withJmxInternal()
+                .pooled("Repair#" + cmd, options.getJobThreads());
     }
 
     private class RepairSessionCallback implements FutureCallback<RepairSessionResult>
@@ -788,7 +761,7 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
 
     private Thread createQueryThread(final UUID sessionId)
     {
-        return NamedThreadFactory.createThread(new WrappedRunnable()
+        return executorFactory().startThread("Repair-Runnable-" + threadCounter.incrementAndGet(), new WrappedRunnable()
         {
             // Query events within a time interval that overlaps the last by one second. Ignore duplicates. Ignore local traces.
             // Wake up upon local trace activity. Query when notified of trace activity with a timeout that doubles every two timeouts.
@@ -858,7 +831,7 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
                     seen[si].clear();
                 }
             }
-        }, "Repair-Runnable-" + threadCounter.incrementAndGet());
+        });
     }
 
     private static final class SkipRepairException extends RuntimeException
diff --git a/src/java/org/apache/cassandra/repair/RepairSession.java b/src/java/org/apache/cassandra/repair/RepairSession.java
index f44ab35..1c7d6c9 100644
--- a/src/java/org/apache/cassandra/repair/RepairSession.java
+++ b/src/java/org/apache/cassandra/repair/RepairSession.java
@@ -33,11 +33,12 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.*;
+import org.apache.cassandra.concurrent.ExecutorFactory;
+import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.apache.cassandra.utils.concurrent.AsyncFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
@@ -57,6 +58,7 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.MerkleTrees;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Future;
 
 /**
  * Coordinates the (active) repair of a list of non overlapping token ranges.
@@ -120,7 +122,7 @@ public class RepairSession extends AsyncFuture<RepairSessionResult> implements I
     private final ConcurrentMap<Pair<RepairJobDesc, SyncNodePair>, CompletableRemoteSyncTask> syncingTasks = new ConcurrentHashMap<>();
 
     // Tasks(snapshot, validate request, differencing, ...) are run on taskExecutor
-    public final ListeningExecutorService taskExecutor;
+    public final ExecutorPlus taskExecutor;
     public final boolean optimiseStreams;
 
     private volatile boolean terminated = false;
@@ -158,12 +160,12 @@ public class RepairSession extends AsyncFuture<RepairSessionResult> implements I
         this.previewKind = previewKind;
         this.pullRepair = pullRepair;
         this.optimiseStreams = optimiseStreams;
-        this.taskExecutor = MoreExecutors.listeningDecorator(createExecutor());
+        this.taskExecutor = createExecutor();
     }
 
-    protected DebuggableThreadPoolExecutor createExecutor()
+    protected ExecutorPlus createExecutor()
     {
-        return DebuggableThreadPoolExecutor.createCachedThreadpoolWithMaxSize("RepairJobTask");
+        return ExecutorFactory.Global.executorFactory().pooled("RepairJobTask", Integer.MAX_VALUE);
     }
 
     public UUID getId()
@@ -262,7 +264,7 @@ public class RepairSession extends AsyncFuture<RepairSessionResult> implements I
      *
      * @param executor Executor to run validation
      */
-    public void start(ListeningExecutorService executor)
+    public void start(ExecutorPlus executor)
     {
         String message;
         if (terminated)
@@ -306,7 +308,7 @@ public class RepairSession extends AsyncFuture<RepairSessionResult> implements I
         }
 
         // Create and submit RepairJob for each ColumnFamily
-        List<ListenableFuture<RepairResult>> jobs = new ArrayList<>(cfnames.length);
+        List<Future<RepairResult>> jobs = new ArrayList<>(cfnames.length);
         for (String cfname : cfnames)
         {
             RepairJob job = new RepairJob(this, cfname);
@@ -315,7 +317,7 @@ public class RepairSession extends AsyncFuture<RepairSessionResult> implements I
         }
 
         // When all RepairJobs are done without error, cleanup and set the final result
-        Futures.addCallback(Futures.allAsList(jobs), new FutureCallback<List<RepairResult>>()
+        FBUtilities.allOf(jobs).addCallback(new FutureCallback<List<RepairResult>>()
         {
             public void onSuccess(List<RepairResult> results)
             {
@@ -339,7 +341,7 @@ public class RepairSession extends AsyncFuture<RepairSessionResult> implements I
                 Tracing.traceRepair("Session completed with the following error: {}", t);
                 forceShutdown(t);
             }
-        }, MoreExecutors.directExecutor());
+        });
     }
 
     public void terminate()
diff --git a/src/java/org/apache/cassandra/repair/asymmetric/RangeDenormalizer.java b/src/java/org/apache/cassandra/repair/asymmetric/RangeDenormalizer.java
index 2a29871..ee69f51 100644
--- a/src/java/org/apache/cassandra/repair/asymmetric/RangeDenormalizer.java
+++ b/src/java/org/apache/cassandra/repair/asymmetric/RangeDenormalizer.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.repair.asymmetric;
 
 import java.util.Collection;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 
diff --git a/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java b/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java
index 83d15b8..1027d0a 100644
--- a/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java
+++ b/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java
@@ -29,12 +29,11 @@ import javax.annotation.Nullable;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterables;
-import com.google.common.util.concurrent.AsyncFunction;
 import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.MoreExecutors;
-import com.google.common.util.concurrent.SettableFuture;
+import org.apache.cassandra.utils.concurrent.AsyncPromise;
+import org.apache.cassandra.utils.concurrent.Future;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
+import org.apache.cassandra.utils.concurrent.Promise;
 import org.apache.commons.lang3.time.DurationFormatUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -65,8 +64,8 @@ public class CoordinatorSession extends ConsistentSession
     private static final Logger logger = LoggerFactory.getLogger(CoordinatorSession.class);
 
     private final Map<InetAddressAndPort, State> participantStates = new HashMap<>();
-    private final SettableFuture<Boolean> prepareFuture = SettableFuture.create();
-    private final SettableFuture<Boolean> finalizeProposeFuture = SettableFuture.create();
+    private final AsyncPromise<Boolean> prepareFuture = AsyncPromise.uncancellable();
+    private final AsyncPromise<Boolean> finalizeProposeFuture = AsyncPromise.uncancellable();
 
     private volatile long sessionStart = Long.MIN_VALUE;
     private volatile long repairStart = Long.MIN_VALUE;
@@ -149,7 +148,7 @@ public class CoordinatorSession extends ConsistentSession
         MessagingService.instance().send(message, destination);
     }
 
-    public ListenableFuture<Boolean> prepare()
+    public Future<Boolean> prepare()
     {
         Preconditions.checkArgument(allStates(State.PREPARING));
 
@@ -189,12 +188,12 @@ public class CoordinatorSession extends ConsistentSession
         if (getState() == State.PREPARED)
         {
             logger.info("Incremental repair session {} successfully prepared.", sessionID);
-            prepareFuture.set(true);
+            prepareFuture.trySuccess(true);
         }
         else
         {
             fail();
-            prepareFuture.set(false);
+            prepareFuture.trySuccess(false);
         }
     }
 
@@ -203,7 +202,7 @@ public class CoordinatorSession extends ConsistentSession
         setAll(State.REPAIRING);
     }
 
-    public synchronized ListenableFuture<Boolean> finalizePropose()
+    public synchronized Future<Boolean> finalizePropose()
     {
         Preconditions.checkArgument(allStates(State.REPAIRING));
         logger.info("Proposing finalization of repair session {}", sessionID);
@@ -225,7 +224,7 @@ public class CoordinatorSession extends ConsistentSession
         {
             logger.warn("Finalization proposal of session {} rejected by {}. Aborting session", sessionID, participant);
             fail();
-            finalizeProposeFuture.set(false);
+            finalizeProposeFuture.trySuccess(false);
         }
         else
         {
@@ -234,7 +233,7 @@ public class CoordinatorSession extends ConsistentSession
             if (getState() == State.FINALIZE_PROMISED)
             {
                 logger.info("Finalization proposal for repair session {} accepted by all participants.", sessionID);
-                finalizeProposeFuture.set(true);
+                finalizeProposeFuture.trySuccess(true);
             }
         }
     }
@@ -271,8 +270,8 @@ public class CoordinatorSession extends ConsistentSession
         setAll(State.FAILED);
 
         String exceptionMsg = String.format("Incremental repair session %s has failed", sessionID);
-        finalizeProposeFuture.setException(RepairException.warn(exceptionMsg));
-        prepareFuture.setException(RepairException.warn(exceptionMsg));
+        finalizeProposeFuture.tryFailure(RepairException.warn(exceptionMsg));
+        prepareFuture.tryFailure(RepairException.warn(exceptionMsg));
     }
 
     private static String formatDuration(long then, long now)
@@ -288,63 +287,55 @@ public class CoordinatorSession extends ConsistentSession
     /**
      * Runs the asynchronous consistent repair session. Actual repair sessions are scheduled via a submitter to make unit testing easier
      */
-    public ListenableFuture execute(Supplier<ListenableFuture<List<RepairSessionResult>>> sessionSubmitter, AtomicBoolean hasFailure)
+    public Future execute(Supplier<Future<List<RepairSessionResult>>> sessionSubmitter, AtomicBoolean hasFailure)
     {
         logger.info("Beginning coordination of incremental repair session {}", sessionID);
 
         sessionStart = currentTimeMillis();
... 7962 lines suppressed ...

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org