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 13:30:44 UTC

[cassandra] branch trunk updated (f5fb1b0 -> 5b82447)

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

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


    omit f5fb1b0  [CASSANDRA-17013] CEP-10 Phase 1: in-jvm-dtest-api changes and version bump
    omit 50278db  [CASSANDRA-16927] CEP-10 Phase 1: Refactor Streaming
    omit e2721b0  [CASSANDRA-16928] CEP-10 Phase 1: InetAddressAndPort extends InetSocketAddress
    omit 767f98e  [CASSANDRA-16926] CEP-10 Phase 1: Mockable Filesystem
    omit e215d2a  [CASSANDRA-16925] CEP-10 Phase 1: Mockable Task Execution
    omit 5c3a05a  [CASSANDRA-16924] CEP-10 Phase 1: Mockable Blocking Concurrency Primitives
     new e5b92e1  [CASSANDRA-16924] CEP-10 Phase 1: Mockable Blocking Concurrency Primitives
     new be1f050  [CASSANDRA-16925] CEP-10 Phase 1: Mockable Task Execution
     new 6a1d9de  [CASSANDRA-16926] CEP-10 Phase 1: Mockable Filesystem
     new aae7e8b  [CASSANDRA-16928] CEP-10 Phase 1: InetAddressAndPort extends InetSocketAddress
     new 6812fdd  [CASSANDRA-16927] CEP-10 Phase 1: Refactor Streaming
     new 5b82447  [CASSANDRA-17013] CEP-10 Phase 1: in-jvm-dtest-api changes and version bump

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (f5fb1b0)
            \
             N -- N -- N   refs/heads/trunk (5b82447)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:

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


[cassandra] 05/06: [CASSANDRA-16927] CEP-10 Phase 1: Refactor Streaming

Posted by be...@apache.org.
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 6812fddb3e6d20e5e8840e57076b97210a9abfe2
Author: Benedict Elliott Smith <be...@apache.org>
AuthorDate: Wed Jul 28 20:01:18 2021 +0100

    [CASSANDRA-16927] CEP-10 Phase 1: Refactor Streaming
    
    patch by Benedict Elliott Smith and Sam Tunnicliffe; reviewed by Aleksey Yeschenko and Aleksei Zotov for CASSANDRA-16927
    
    Co-authored-by: Benedict Elliott Smith <be...@apache.org>
    Co-authored-by: Aleksey Yeschenko  <al...@apache.org>
    Co-authored-by: Sam Tunnicliffe  <sa...@apache.org>
---
 .../streaming/CassandraCompressedStreamReader.java |   2 +-
 .../streaming/CassandraCompressedStreamWriter.java |   6 +-
 .../CassandraEntireSSTableStreamWriter.java        |   4 +-
 .../db/streaming/CassandraOutgoingFile.java        |  10 +-
 .../db/streaming/CassandraStreamReader.java        |   1 -
 .../db/streaming/CassandraStreamWriter.java        |  10 +-
 .../apache/cassandra/io/sstable/SSTableLoader.java |   6 +-
 .../cassandra/net/AsyncStreamingInputPlus.java     |   8 +-
 .../cassandra/net/AsyncStreamingOutputPlus.java    |  34 +--
 .../cassandra/net/GlobalBufferPoolAllocator.java   |   2 +-
 .../cassandra/net/InboundConnectionInitiator.java  |  10 +-
 .../repair/consistent/SyncStatSummary.java         |  13 +-
 .../apache/cassandra/streaming/OutgoingStream.java |   3 +-
 .../apache/cassandra/streaming/SessionInfo.java    |  10 +-
 .../apache/cassandra/streaming/SessionSummary.java |   7 +-
 .../cassandra/streaming/StreamCoordinator.java     |  35 +--
 .../streaming/StreamDeserializingTask.java         | 120 ++++++++
 .../apache/cassandra/streaming/StreamManager.java  |   2 +-
 .../org/apache/cassandra/streaming/StreamPlan.java |   9 +-
 .../cassandra/streaming/StreamResultFuture.java    |  23 +-
 .../apache/cassandra/streaming/StreamSession.java  | 140 +++++----
 .../cassandra/streaming/StreamTransferTask.java    |   2 +-
 .../cassandra/streaming/StreamingChannel.java      |  73 +++++
 ...sageSender.java => StreamingDataInputPlus.java} |  13 +-
 .../streaming/StreamingDataOutputPlus.java         |  80 ++++++
 ...nder.java => StreamingDataOutputPlusFixed.java} |  30 +-
 .../streaming/async/NettyStreamingChannel.java     | 260 +++++++++++++++++
 .../NettyStreamingConnectionFactory.java}          |  28 +-
 .../async/StreamCompressionSerializer.java         |   3 +-
 .../streaming/async/StreamingInboundHandler.java   | 256 -----------------
 ...ender.java => StreamingMultiplexedChannel.java} | 313 ++++++---------------
 .../streaming/messages/CompleteMessage.java        |   4 +-
 .../streaming/messages/IncomingStreamMessage.java  |  11 +-
 .../streaming/messages/KeepAliveMessage.java       |   4 +-
 .../streaming/messages/OutgoingStreamMessage.java  |   6 +-
 .../streaming/messages/PrepareAckMessage.java      |   4 +-
 .../streaming/messages/PrepareSynAckMessage.java   |   4 +-
 .../streaming/messages/PrepareSynMessage.java      |   5 +-
 .../streaming/messages/ReceivedMessage.java        |   5 +-
 .../streaming/messages/SessionFailedMessage.java   |   4 +-
 .../streaming/messages/StreamInitMessage.java      |  13 +-
 .../streaming/messages/StreamMessage.java          |  11 +-
 .../cassandra/tools/BulkLoadConnectionFactory.java |  18 +-
 .../org/apache/cassandra/tools/BulkLoader.java     |   7 +-
 .../apache/cassandra/tools/nodetool/NetStats.java  |   5 +-
 .../distributed/impl/AbstractCluster.java          |   1 -
 .../cassandra/distributed/impl/Instance.java       |   6 +-
 .../test/metrics/HintsServiceMetricsTest.java      |   1 -
 .../microbench/ZeroCopyStreamingBenchmark.java     |   6 +-
 .../CassandraEntireSSTableStreamWriterTest.java    |   6 +-
 .../db/streaming/CassandraStreamManagerTest.java   |   9 +-
 ...TableStreamConcurrentComponentMutationTest.java |   6 +-
 .../apache/cassandra/dht/StreamStateStoreTest.java |   7 +-
 .../cassandra/net/AsyncStreamingInputPlusTest.java |   5 +-
 .../unit/org/apache/cassandra/net/TestChannel.java |  17 ++
 .../apache/cassandra/repair/LocalSyncTaskTest.java |   6 +-
 ...ntireSSTableStreamingCorrectFilesCountTest.java |   5 +-
 .../streaming/StreamTransferTaskTest.java          |  23 +-
 .../async/StreamingInboundHandlerTest.java         |  39 +--
 ...t.java => StreamingMultiplexedChannelTest.java} |  93 ++----
 60 files changed, 1002 insertions(+), 842 deletions(-)

diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java
index f6af23c..cfa9018 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java
@@ -100,7 +100,7 @@ public class CassandraCompressedStreamReader extends CassandraStreamReader
                 }
                 assert in.getBytesRead() == sectionLength;
             }
-            logger.trace("[Stream #{}] Finished receiving file #{} from {} readBytes = {}, totalSize = {}", session.planId(), fileSeqNum,
+            logger.info("[Stream #{}] Finished receiving file #{} from {} readBytes = {}, totalSize = {}", session.planId(), fileSeqNum,
                          session.peer, FBUtilities.prettyPrintMemory(cis.chunkBytesRead()), FBUtilities.prettyPrintMemory(totalSize));
             return writer;
         }
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java
index 8d0b67f..99908f2 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java
@@ -30,9 +30,8 @@ import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.ChannelProxy;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.net.AsyncStreamingOutputPlus;
 import org.apache.cassandra.streaming.ProgressInfo;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -57,9 +56,8 @@ public class CassandraCompressedStreamWriter extends CassandraStreamWriter
     }
 
     @Override
-    public void write(DataOutputStreamPlus output) throws IOException
+    public void write(StreamingDataOutputPlus out) throws IOException
     {
-        AsyncStreamingOutputPlus out = (AsyncStreamingOutputPlus) output;
         long totalSize = totalSize();
         logger.debug("[Stream #{}] Start streaming file {} to {}, repairedAt = {}, totalSize = {}", session.planId(),
                      sstable.getFilename(), session.peer, sstable.getSSTableMetadata().repairedAt, totalSize);
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java
index ef82eb2..f096e57 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java
@@ -26,8 +26,8 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.net.AsyncStreamingOutputPlus;
 import org.apache.cassandra.streaming.ProgressInfo;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamManager;
 import org.apache.cassandra.streaming.StreamSession;
 
@@ -63,7 +63,7 @@ public class CassandraEntireSSTableStreamWriter
      * @param out where this writes data to
      * @throws IOException on any I/O error
      */
-    public void write(AsyncStreamingOutputPlus out) throws IOException
+    public void write(StreamingDataOutputPlus out) throws IOException
     {
         long totalSize = manifest.totalSize();
         logger.debug("[Stream #{}] Start streaming sstable {} to {}, repairedAt = {}, totalSize = {}",
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
index 0904720..d6dec8a 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
@@ -30,10 +30,9 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.net.AsyncStreamingOutputPlus;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamOperation;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.utils.concurrent.Ref;
@@ -148,11 +147,8 @@ public class CassandraOutgoingFile implements OutgoingStream
     }
 
     @Override
-    public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
+    public void write(StreamSession session, StreamingDataOutputPlus out, int version) throws IOException
     {
-        // FileStreamTask uses AsyncStreamingOutputPlus for streaming.
-        assert out instanceof AsyncStreamingOutputPlus : "Unexpected DataOutputStreamPlus " + out.getClass();
-
         SSTableReader sstable = ref.get();
 
         if (shouldStreamEntireSSTable)
@@ -169,7 +165,7 @@ public class CassandraOutgoingFile implements OutgoingStream
                 out.flush();
 
                 CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, context);
-                writer.write((AsyncStreamingOutputPlus) out);
+                writer.write(out);
             }
         }
         else
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java
index 17c40b8..6835fad 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java
@@ -25,7 +25,6 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
 import com.google.common.collect.UnmodifiableIterator;
 
-import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java
index cb513ab..d69f3ee 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java
@@ -33,9 +33,8 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.ChannelProxy;
 import org.apache.cassandra.io.util.DataIntegrityMetadata;
 import org.apache.cassandra.io.util.DataIntegrityMetadata.ChecksumValidator;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.net.AsyncStreamingOutputPlus;
 import org.apache.cassandra.streaming.ProgressInfo;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamManager;
 import org.apache.cassandra.streaming.StreamManager.StreamRateLimiter;
 import org.apache.cassandra.streaming.StreamSession;
@@ -75,16 +74,15 @@ public class CassandraStreamWriter
      *
      * CassandraStreamWriter uses LZF compression on wire to decrease size to transfer.
      *
-     * @param output where this writes data to
+     * @param out where this writes data to
      * @throws IOException on any I/O error
      */
-    public void write(DataOutputStreamPlus output) throws IOException
+    public void write(StreamingDataOutputPlus out) throws IOException
     {
         long totalSize = totalSize();
         logger.debug("[Stream #{}] Start streaming file {} to {}, repairedAt = {}, totalSize = {}", session.planId(),
                      sstable.getFilename(), session.peer, sstable.getSSTableMetadata().repairedAt, totalSize);
 
-        AsyncStreamingOutputPlus out = (AsyncStreamingOutputPlus) output;
         try(ChannelProxy proxy = sstable.getDataChannel().newChannel();
             ChecksumValidator validator = new File(sstable.descriptor.filenameFor(Component.CRC)).exists()
                                           ? DataIntegrityMetadata.checksumValidator(sstable.descriptor)
@@ -145,7 +143,7 @@ public class CassandraStreamWriter
      *
      * @throws java.io.IOException on any I/O error
      */
-    protected long write(ChannelProxy proxy, ChecksumValidator validator, AsyncStreamingOutputPlus output, long start, int transferOffset, int toTransfer, int bufferSize) throws IOException
+    protected long write(ChannelProxy proxy, ChecksumValidator validator, StreamingDataOutputPlus output, long start, int transferOffset, int toTransfer, int bufferSize) throws IOException
     {
         // the count of bytes to read off disk
         int minReadable = (int) Math.min(bufferSize, proxy.size() - start);
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
index 7df9258..df4a4e2 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
@@ -38,6 +38,8 @@ import org.apache.cassandra.utils.Pair;
 
 import org.apache.cassandra.utils.concurrent.Ref;
 
+import static org.apache.cassandra.streaming.StreamingChannel.Factory.Global.streamingFactory;
+
 /**
  * Cassandra SSTable bulk loader.
  * Load an externally created sstable into a cluster.
@@ -287,9 +289,9 @@ public class SSTableLoader implements StreamEventHandler
          *
          * @return StreamConnectionFactory to use
          */
-        public StreamConnectionFactory getConnectionFactory()
+        public StreamingChannel.Factory getConnectionFactory()
         {
-            return new DefaultConnectionFactory();
+            return streamingFactory();
         }
 
         /**
diff --git a/src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.java b/src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.java
index 7e73355..b964dda 100644
--- a/src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.java
+++ b/src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.net;
 import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.TimeUnit;
 
@@ -31,11 +32,12 @@ import io.netty.buffer.ByteBufAllocator;
 import io.netty.buffer.Unpooled;
 import io.netty.channel.Channel;
 import org.apache.cassandra.io.util.RebufferingInputStream;
+import org.apache.cassandra.streaming.StreamingDataInputPlus;
 
 import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
 // TODO: rewrite
-public class AsyncStreamingInputPlus extends RebufferingInputStream
+public class AsyncStreamingInputPlus extends RebufferingInputStream implements StreamingDataInputPlus
 {
     public static class InputTimeoutException extends IOException
     {
@@ -108,7 +110,7 @@ public class AsyncStreamingInputPlus extends RebufferingInputStream
      * the {@link #rebufferTimeoutNanos} elapses while blocking. It's then not safe to reuse this instance again.
      */
     @Override
-    protected void reBuffer() throws EOFException, InputTimeoutException
+    protected void reBuffer() throws ClosedChannelException, InputTimeoutException
     {
         if (queue.isEmpty())
             channel.read();
@@ -131,7 +133,7 @@ public class AsyncStreamingInputPlus extends RebufferingInputStream
             throw new InputTimeoutException();
 
         if (next == Unpooled.EMPTY_BUFFER) // Unpooled.EMPTY_BUFFER is the indicator that the input is closed
-            throw new EOFException();
+            throw new ClosedChannelException();
 
         currentBuf = next;
         buffer = next.nioBuffer();
diff --git a/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java b/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
index 9c0f265..096b883 100644
--- a/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
+++ b/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
@@ -35,7 +35,7 @@ import io.netty.handler.ssl.SslHandler;
 import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.net.SharedDefaultFileRegion.SharedFileChannel;
-import org.apache.cassandra.streaming.StreamManager.StreamRateLimiter;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.utils.memory.BufferPool;
 import org.apache.cassandra.utils.memory.BufferPools;
 
@@ -50,7 +50,7 @@ import static java.lang.Math.min;
  * The correctness of this class depends on the ChannelPromise we create against a Channel always being completed,
  * which appears to be a guarantee provided by Netty so long as the event loop is running.
  */
-public class AsyncStreamingOutputPlus extends AsyncChannelOutputPlus
+public class AsyncStreamingOutputPlus extends AsyncChannelOutputPlus implements StreamingDataOutputPlus
 {
     private static final Logger logger = LoggerFactory.getLogger(AsyncStreamingOutputPlus.class);
 
@@ -97,28 +97,6 @@ public class AsyncStreamingOutputPlus extends AsyncChannelOutputPlus
         return flushed() + buffer.position();
     }
 
-    public interface BufferSupplier
-    {
-        /**
-         * Request a buffer with at least the given capacity.
-         * This method may only be invoked once, and the lifetime of buffer it returns will be managed
-         * by the AsyncChannelOutputPlus it was created for.
-         */
-        ByteBuffer get(int capacity) throws IOException;
-    }
-
-    public interface Write
-    {
-        /**
-         * Write to a buffer, and flush its contents to the channel.
-         * <p>
-         * The lifetime of the buffer will be managed by the AsyncChannelOutputPlus you issue this Write to.
-         * If the method exits successfully, the contents of the buffer will be written to the channel, otherwise
-         * the buffer will be cleaned and the exception propagated to the caller.
-         */
-        void write(BufferSupplier supplier) throws IOException;
-    }
-
     /**
      * Provide a lambda that can request a buffer of suitable size, then fill the buffer and have
      * that buffer written and flushed to the underlying channel, without having to handle buffer
@@ -126,7 +104,7 @@ public class AsyncStreamingOutputPlus extends AsyncChannelOutputPlus
      * <p>
      * Any exception thrown by the Write will be propagated to the caller, after any buffer is cleaned up.
      */
-    public int writeToChannel(Write write, StreamRateLimiter limiter) throws IOException
+    public int writeToChannel(Write write, RateLimiter limiter) throws IOException
     {
         doFlush(0);
         class Holder
@@ -175,7 +153,7 @@ public class AsyncStreamingOutputPlus extends AsyncChannelOutputPlus
      * WARNING: this method blocks only for permission to write to the netty channel; it exits before
      * the {@link FileRegion}(zero-copy) or {@link ByteBuffer}(ssl) is flushed to the network.
      */
-    public long writeFileToChannel(FileChannel file, StreamRateLimiter limiter) throws IOException
+    public long writeFileToChannel(FileChannel file, RateLimiter limiter) throws IOException
     {
         if (channel.pipeline().get(SslHandler.class) != null)
             // each batch is loaded into ByteBuffer, 64kb is more BufferPool friendly.
@@ -187,7 +165,7 @@ public class AsyncStreamingOutputPlus extends AsyncChannelOutputPlus
     }
 
     @VisibleForTesting
-    long writeFileToChannel(FileChannel fc, StreamRateLimiter limiter, int batchSize) throws IOException
+    long writeFileToChannel(FileChannel fc, RateLimiter limiter, int batchSize) throws IOException
     {
         final long length = fc.size();
         long bytesTransferred = 0;
@@ -224,7 +202,7 @@ public class AsyncStreamingOutputPlus extends AsyncChannelOutputPlus
     }
 
     @VisibleForTesting
-    long writeFileToChannelZeroCopy(FileChannel file, StreamRateLimiter limiter, int batchSize, int lowWaterMark, int highWaterMark) throws IOException
+    long writeFileToChannelZeroCopy(FileChannel file, RateLimiter limiter, int batchSize, int lowWaterMark, int highWaterMark) throws IOException
     {
         final long length = file.size();
         long bytesTransferred = 0;
diff --git a/src/java/org/apache/cassandra/net/GlobalBufferPoolAllocator.java b/src/java/org/apache/cassandra/net/GlobalBufferPoolAllocator.java
index 16fd5c6..f559a63 100644
--- a/src/java/org/apache/cassandra/net/GlobalBufferPoolAllocator.java
+++ b/src/java/org/apache/cassandra/net/GlobalBufferPoolAllocator.java
@@ -34,7 +34,7 @@ public class GlobalBufferPoolAllocator extends BufferPoolAllocator
         super();
     }
 
-    static ByteBuf wrap(ByteBuffer buffer)
+    public static ByteBuf wrap(ByteBuffer buffer)
     {
         return new Wrapped(instance, buffer, buffer.capacity());
     }
diff --git a/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java b/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
index fbf655f..867efd0 100644
--- a/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
+++ b/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
@@ -52,11 +52,14 @@ import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.OutboundConnectionSettings.Framing;
 import org.apache.cassandra.security.ISslContextFactory;
 import org.apache.cassandra.security.SSLFactory;
-import org.apache.cassandra.streaming.async.StreamingInboundHandler;
+import org.apache.cassandra.streaming.StreamDeserializingTask;
+import org.apache.cassandra.streaming.StreamingChannel;
+import org.apache.cassandra.streaming.async.NettyStreamingChannel;
 import org.apache.cassandra.utils.memory.BufferPools;
 
 import static java.lang.Math.*;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.net.MessagingService.*;
 import static org.apache.cassandra.net.SocketFactory.WIRETRACE;
 import static org.apache.cassandra.net.SocketFactory.newSslHandler;
@@ -413,7 +416,10 @@ public class InboundConnectionInitiator
             }
 
             BufferPools.forNetworking().setRecycleWhenFreeForCurrentThread(false);
-            pipeline.replace(this, "streamInbound", new StreamingInboundHandler(from, current_version, null));
+            NettyStreamingChannel streamingChannel = new NettyStreamingChannel(current_version, channel, StreamingChannel.Kind.CONTROL);
+            pipeline.replace(this, "streamInbound", streamingChannel);
+            executorFactory().startThread(String.format("Stream-Deserializer-%s-%s", from, channel.id()),
+                                          new StreamDeserializingTask(null, streamingChannel, current_version));
 
             logger.info("{} streaming connection established, version = {}, framing = {}, encryption = {}",
                         SocketFactory.channelId(from,
diff --git a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java
index f8e1bfb..b145fb6 100644
--- a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java
+++ b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java
@@ -18,6 +18,7 @@
 
 package org.apache.cassandra.repair.consistent;
 
+import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
@@ -44,14 +45,14 @@ public class SyncStatSummary
 
     private static class Session
     {
-        final InetAddressAndPort src;
-        final InetAddressAndPort dst;
+        final InetSocketAddress src;
+        final InetSocketAddress dst;
 
         int files = 0;
         long bytes = 0;
         long ranges = 0;
 
-        Session(InetAddressAndPort src, InetAddressAndPort dst)
+        Session(InetSocketAddress src, InetSocketAddress dst)
         {
             this.src = src;
             this.dst = dst;
@@ -86,7 +87,7 @@ public class SyncStatSummary
         int ranges = -1;
         boolean totalsCalculated = false;
 
-        final Map<Pair<InetAddressAndPort, InetAddressAndPort>, Session> sessions = new HashMap<>();
+        final Map<Pair<InetSocketAddress, InetSocketAddress>, Session> sessions = new HashMap<>();
 
         Table(String keyspace, String table)
         {
@@ -94,9 +95,9 @@ public class SyncStatSummary
             this.table = table;
         }
 
-        Session getOrCreate(InetAddressAndPort from, InetAddressAndPort to)
+        Session getOrCreate(InetSocketAddress from, InetSocketAddress to)
         {
-            Pair<InetAddressAndPort, InetAddressAndPort> k = Pair.create(from, to);
+            Pair<InetSocketAddress, InetSocketAddress> k = Pair.create(from, to);
             if (!sessions.containsKey(k))
             {
                 sessions.put(k, new Session(from, to));
diff --git a/src/java/org/apache/cassandra/streaming/OutgoingStream.java b/src/java/org/apache/cassandra/streaming/OutgoingStream.java
index 546462d..276e5a3 100644
--- a/src/java/org/apache/cassandra/streaming/OutgoingStream.java
+++ b/src/java/org/apache/cassandra/streaming/OutgoingStream.java
@@ -21,7 +21,6 @@ package org.apache.cassandra.streaming;
 import java.io.IOException;
 import java.util.UUID;
 
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.schema.TableId;
 
 /**
@@ -36,7 +35,7 @@ public interface OutgoingStream
     /**
      * Write the streams data into the socket
      */
-    void write(StreamSession session, DataOutputStreamPlus output, int version) throws IOException;
+    void write(StreamSession session, StreamingDataOutputPlus output, int version) throws IOException;
 
     /**
      * Release any resources held by the stream
diff --git a/src/java/org/apache/cassandra/streaming/SessionInfo.java b/src/java/org/apache/cassandra/streaming/SessionInfo.java
index 4b4bbed..31cff9b 100644
--- a/src/java/org/apache/cassandra/streaming/SessionInfo.java
+++ b/src/java/org/apache/cassandra/streaming/SessionInfo.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.streaming;
 
 import java.io.Serializable;
+import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -26,7 +27,6 @@ import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 
-import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -34,9 +34,9 @@ import org.apache.cassandra.utils.FBUtilities;
  */
 public final class SessionInfo implements Serializable
 {
-    public final InetAddressAndPort peer;
+    public final InetSocketAddress peer;
     public final int sessionIndex;
-    public final InetAddressAndPort connecting;
+    public final InetSocketAddress connecting;
     /** Immutable collection of receiving summaries */
     public final Collection<StreamSummary> receivingSummaries;
     /** Immutable collection of sending summaries*/
@@ -47,9 +47,9 @@ public final class SessionInfo implements Serializable
     private final Map<String, ProgressInfo> receivingFiles;
     private final Map<String, ProgressInfo> sendingFiles;
 
-    public SessionInfo(InetAddressAndPort peer,
+    public SessionInfo(InetSocketAddress peer,
                        int sessionIndex,
-                       InetAddressAndPort connecting,
+                       InetSocketAddress connecting,
                        Collection<StreamSummary> receivingSummaries,
                        Collection<StreamSummary> sendingSummaries,
                        StreamSession.State state)
diff --git a/src/java/org/apache/cassandra/streaming/SessionSummary.java b/src/java/org/apache/cassandra/streaming/SessionSummary.java
index 65ecd32..9588e49 100644
--- a/src/java/org/apache/cassandra/streaming/SessionSummary.java
+++ b/src/java/org/apache/cassandra/streaming/SessionSummary.java
@@ -19,6 +19,7 @@
 package org.apache.cassandra.streaming;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -33,14 +34,14 @@ import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAdd
 
 public class SessionSummary
 {
-    public final InetAddressAndPort coordinator;
-    public final InetAddressAndPort peer;
+    public final InetSocketAddress coordinator;
+    public final InetSocketAddress peer;
     /** Immutable collection of receiving summaries */
     public final Collection<StreamSummary> receivingSummaries;
     /** Immutable collection of sending summaries*/
     public final Collection<StreamSummary> sendingSummaries;
 
-    public SessionSummary(InetAddressAndPort coordinator, InetAddressAndPort peer,
+    public SessionSummary(InetSocketAddress coordinator, InetSocketAddress peer,
                           Collection<StreamSummary> receivingSummaries,
                           Collection<StreamSummary> sendingSummaries)
     {
diff --git a/src/java/org/apache/cassandra/streaming/StreamCoordinator.java b/src/java/org/apache/cassandra/streaming/StreamCoordinator.java
index e590e96..6b97416 100644
--- a/src/java/org/apache/cassandra/streaming/StreamCoordinator.java
+++ b/src/java/org/apache/cassandra/streaming/StreamCoordinator.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.streaming;
 
+import java.net.InetSocketAddress;
 import java.util.*;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -25,6 +26,8 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.locator.InetAddressAndPort;
 
+import static org.apache.cassandra.net.MessagingService.current_version;
+
 
 /**
  * {@link StreamCoordinator} is a helper class that abstracts away maintaining multiple
@@ -39,16 +42,16 @@ public class StreamCoordinator
 
     private final boolean connectSequentially;
 
-    private final Map<InetAddressAndPort, HostStreamingData> peerSessions = new HashMap<>();
+    private final Map<InetSocketAddress, HostStreamingData> peerSessions = new HashMap<>();
     private final StreamOperation streamOperation;
     private final int connectionsPerHost;
     private final boolean follower;
-    private StreamConnectionFactory factory;
+    private StreamingChannel.Factory factory;
     private Iterator<StreamSession> sessionsToConnect = null;
     private final UUID pendingRepair;
     private final PreviewKind previewKind;
 
-    public StreamCoordinator(StreamOperation streamOperation, int connectionsPerHost, StreamConnectionFactory factory,
+    public StreamCoordinator(StreamOperation streamOperation, int connectionsPerHost, StreamingChannel.Factory factory,
                              boolean follower, boolean connectSequentially, UUID pendingRepair, PreviewKind previewKind)
     {
         this.streamOperation = streamOperation;
@@ -60,7 +63,7 @@ public class StreamCoordinator
         this.previewKind = previewKind;
     }
 
-    public void setConnectionFactory(StreamConnectionFactory factory)
+    public void setConnectionFactory(StreamingChannel.Factory factory)
     {
         this.factory = factory;
     }
@@ -147,19 +150,19 @@ public class StreamCoordinator
             logger.debug("Finished connecting all sessions");
     }
 
-    public synchronized Set<InetAddressAndPort> getPeers()
+    public synchronized Set<InetSocketAddress> getPeers()
     {
         return new HashSet<>(peerSessions.keySet());
     }
 
-    public synchronized StreamSession getOrCreateNextSession(InetAddressAndPort peer)
+    public synchronized StreamSession getOrCreateOutboundSession(InetAddressAndPort peer)
     {
-        return getOrCreateHostData(peer).getOrCreateNextSession(peer);
+        return getOrCreateHostData(peer).getOrCreateOutboundSession(peer);
     }
 
-    public synchronized StreamSession getOrCreateSessionById(InetAddressAndPort peer, int id)
+    public synchronized StreamSession getOrCreateInboundSession(InetAddressAndPort from, StreamingChannel channel, int messagingVersion, int id)
     {
-        return getOrCreateHostData(peer).getOrCreateSessionById(peer, id);
+        return getOrCreateHostData(from).getOrCreateInboundSession(from, channel, messagingVersion, id);
     }
 
     public StreamSession getSessionById(InetAddressAndPort peer, int id)
@@ -198,13 +201,13 @@ public class StreamCoordinator
 
             for (Collection<OutgoingStream> bucket : buckets)
             {
-                StreamSession session = sessionList.getOrCreateNextSession(to);
+                StreamSession session = sessionList.getOrCreateOutboundSession(to);
                 session.addTransferStreams(bucket);
             }
         }
         else
         {
-            StreamSession session = sessionList.getOrCreateNextSession(to);
+            StreamSession session = sessionList.getOrCreateOutboundSession(to);
             session.addTransferStreams(streams);
         }
     }
@@ -241,7 +244,7 @@ public class StreamCoordinator
         return data;
     }
 
-    private HostStreamingData getOrCreateHostData(InetAddressAndPort peer)
+    private HostStreamingData getOrCreateHostData(InetSocketAddress peer)
     {
         HostStreamingData data = peerSessions.get(peer);
         if (data == null)
@@ -280,12 +283,12 @@ public class StreamCoordinator
             return false;
         }
 
-        public StreamSession getOrCreateNextSession(InetAddressAndPort peer)
+        public StreamSession getOrCreateOutboundSession(InetAddressAndPort peer)
         {
             // create
             if (streamSessions.size() < connectionsPerHost)
             {
-                StreamSession session = new StreamSession(streamOperation, peer, factory, isFollower(), streamSessions.size(),
+                StreamSession session = new StreamSession(streamOperation, peer, factory, null, current_version, isFollower(), streamSessions.size(),
                                                           pendingRepair, previewKind);
                 streamSessions.put(++lastReturned, session);
                 sessionInfos.put(lastReturned, session.getSessionInfo());
@@ -314,12 +317,12 @@ public class StreamCoordinator
             return Collections.unmodifiableCollection(streamSessions.values());
         }
 
-        public StreamSession getOrCreateSessionById(InetAddressAndPort peer, int id)
+        public StreamSession getOrCreateInboundSession(InetAddressAndPort from, StreamingChannel channel, int messagingVersion, int id)
         {
             StreamSession session = streamSessions.get(id);
             if (session == null)
             {
-                session = new StreamSession(streamOperation, peer, factory, isFollower(), id, pendingRepair, previewKind);
+                session = new StreamSession(streamOperation, from, factory, channel, messagingVersion, isFollower(), id, pendingRepair, previewKind);
                 streamSessions.put(id, session);
                 sessionInfos.put(id, session.getSessionInfo());
             }
diff --git a/src/java/org/apache/cassandra/streaming/StreamDeserializingTask.java b/src/java/org/apache/cassandra/streaming/StreamDeserializingTask.java
new file mode 100644
index 0000000..dd2678c
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamDeserializingTask.java
@@ -0,0 +1,120 @@
+/*
+ * 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.streaming;
+
+import java.nio.channels.ClosedChannelException;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.streaming.messages.KeepAliveMessage;
+import org.apache.cassandra.streaming.messages.StreamMessage;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.streaming.StreamSession.createLogTag;
+
+/**
+ * The task that performs the actual deserialization.
+ */
+public class StreamDeserializingTask implements Runnable
+{
+    private static final Logger logger = LoggerFactory.getLogger(StreamDeserializingTask.class);
+
+    private final StreamingChannel channel;
+    private final int messagingVersion;
+    @VisibleForTesting
+    protected StreamSession session;
+
+    public StreamDeserializingTask(StreamSession session, StreamingChannel channel, int messagingVersion)
+    {
+        this.session = session;
+        this.channel = channel;
+        this.messagingVersion = messagingVersion;
+    }
+
+    @Override
+    public void run()
+    {
+        @SuppressWarnings("resource") // closed in finally
+        StreamingDataInputPlus input = channel.in();
+        try
+        {
+            StreamMessage message;
+            while (null != (message = StreamMessage.deserialize(input, messagingVersion)))
+            {
+                // keep-alives don't necessarily need to be tied to a session (they could be arrive before or after
+                // wrt session lifecycle, due to races), just log that we received the message and carry on
+                if (message instanceof KeepAliveMessage)
+                {
+                    if (logger.isDebugEnabled())
+                        logger.debug("{} Received {}", createLogTag(session, channel), message);
+                    continue;
+                }
+
+                if (session == null)
+                    session = deriveSession(message);
+
+                if (logger.isDebugEnabled())
+                    logger.debug("{} Received {}", createLogTag(session, channel), message);
+
+                session.messageReceived(message);
+            }
+        }
+        catch (ClosedChannelException ignore)
+        {
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+            if (session != null)
+            {
+                session.onError(t);
+            }
+            else if (t instanceof StreamReceiveException)
+            {
+                ((StreamReceiveException)t).session.onError(t);
+            }
+            else
+            {
+                logger.error("{} stream operation from {} failed", createLogTag(session, channel), InetAddressAndPort.toString(channel.peer(), true), t);
+            }
+        }
+        finally
+        {
+            channel.close();
+            input.close();
+        }
+    }
+
+    @VisibleForTesting
+    public StreamSession deriveSession(StreamMessage message)
+    {
+        // StreamInitMessage starts a new channel here, but IncomingStreamMessage needs a session
+        // to be established a priori
+        StreamSession streamSession = message.getOrCreateAndAttachInboundSession(channel, messagingVersion);
+
+        // Attach this channel to the session: this only happens upon receiving the first init message as a follower;
+        // in all other cases, no new control channel will be added, as the proper control channel will be already attached.
+        streamSession.attachInbound(channel);
+        return streamSession;
+    }
+}
diff --git a/src/java/org/apache/cassandra/streaming/StreamManager.java b/src/java/org/apache/cassandra/streaming/StreamManager.java
index 7d8f143..9c24c1c 100644
--- a/src/java/org/apache/cassandra/streaming/StreamManager.java
+++ b/src/java/org/apache/cassandra/streaming/StreamManager.java
@@ -60,7 +60,7 @@ public class StreamManager implements StreamManagerMBean
         return new StreamRateLimiter(peer);
     }
 
-    public static class StreamRateLimiter
+    public static class StreamRateLimiter implements StreamingDataOutputPlus.RateLimiter
     {
         public static final double BYTES_PER_MEGABIT = (1024 * 1024) / 8; // from bits
         private static final RateLimiter limiter = RateLimiter.create(calculateRateInBytes());
diff --git a/src/java/org/apache/cassandra/streaming/StreamPlan.java b/src/java/org/apache/cassandra/streaming/StreamPlan.java
index 60845fa..47016ce 100644
--- a/src/java/org/apache/cassandra/streaming/StreamPlan.java
+++ b/src/java/org/apache/cassandra/streaming/StreamPlan.java
@@ -28,6 +28,7 @@ import org.apache.cassandra.utils.UUIDGen;
 
 import static com.google.common.collect.Iterables.all;
 import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.apache.cassandra.streaming.StreamingChannel.Factory.Global.streamingFactory;
 
 /**
  * {@link StreamPlan} is a helper class that builds StreamOperation of given configuration.
@@ -63,7 +64,7 @@ public class StreamPlan
                       boolean connectSequentially, UUID pendingRepair, PreviewKind previewKind)
     {
         this.streamOperation = streamOperation;
-        this.coordinator = new StreamCoordinator(streamOperation, connectionsPerHost, new DefaultConnectionFactory(),
+        this.coordinator = new StreamCoordinator(streamOperation, connectionsPerHost, streamingFactory(),
                                                  false, connectSequentially, pendingRepair, previewKind);
     }
 
@@ -107,7 +108,7 @@ public class StreamPlan
         assert all(fullRanges, Replica::isSelf) || RangesAtEndpoint.isDummyList(fullRanges) : fullRanges.toString();
         assert all(transientRanges, Replica::isSelf) || RangesAtEndpoint.isDummyList(transientRanges) : transientRanges.toString();
 
-        StreamSession session = coordinator.getOrCreateNextSession(from);
+        StreamSession session = coordinator.getOrCreateOutboundSession(from);
         session.addStreamRequest(keyspace, fullRanges, transientRanges, Arrays.asList(columnFamilies));
         return this;
     }
@@ -123,7 +124,7 @@ public class StreamPlan
      */
     public StreamPlan transferRanges(InetAddressAndPort to, String keyspace, RangesAtEndpoint replicas, String... columnFamilies)
     {
-        StreamSession session = coordinator.getOrCreateNextSession(to);
+        StreamSession session = coordinator.getOrCreateOutboundSession(to);
         session.addTransferRanges(keyspace, replicas, Arrays.asList(columnFamilies), flushBeforeTransfer);
         return this;
     }
@@ -155,7 +156,7 @@ public class StreamPlan
      * @param factory StreamConnectionFactory to use
      * @return self
      */
-    public StreamPlan connectionFactory(StreamConnectionFactory factory)
+    public StreamPlan connectionFactory(StreamingChannel.Factory factory)
     {
         this.coordinator.setConnectionFactory(factory);
         return this;
diff --git a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
index 8febff6..6bcd074 100644
--- a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
+++ b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
@@ -25,10 +25,11 @@ import org.apache.cassandra.utils.concurrent.AsyncFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import io.netty.channel.Channel;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.FBUtilities;
 
+import static org.apache.cassandra.streaming.StreamingChannel.Factory.Global.streamingFactory;
+
 /**
  * A future on the result ({@link StreamState}) of a streaming plan.
  *
@@ -72,7 +73,7 @@ public final class StreamResultFuture extends AsyncFuture<StreamState>
 
     private StreamResultFuture(UUID planId, StreamOperation streamOperation, UUID pendingRepair, PreviewKind previewKind)
     {
-        this(planId, streamOperation, new StreamCoordinator(streamOperation, 0, new DefaultConnectionFactory(), true, false, pendingRepair, previewKind));
+        this(planId, streamOperation, new StreamCoordinator(streamOperation, 0, streamingFactory(), true, false, pendingRepair, previewKind));
     }
 
     public static StreamResultFuture createInitiator(UUID planId, StreamOperation streamOperation, Collection<StreamEventHandler> listeners,
@@ -102,24 +103,24 @@ public final class StreamResultFuture extends AsyncFuture<StreamState>
                                                                  UUID planId,
                                                                  StreamOperation streamOperation,
                                                                  InetAddressAndPort from,
-                                                                 Channel channel,
+                                                                 StreamingChannel channel,
+                                                                 int messagingVersion,
                                                                  UUID pendingRepair,
                                                                  PreviewKind previewKind)
     {
         StreamResultFuture future = StreamManager.instance.getReceivingStream(planId);
         if (future == null)
         {
-            logger.info("[Stream #{} ID#{}] Creating new streaming plan for {} from {} channel.remote {} channel.local {}" +
-                        " channel.id {}", planId, sessionIndex, streamOperation.getDescription(),
-                        from, channel.remoteAddress(), channel.localAddress(), channel.id());
+            logger.info("[Stream #{} ID#{}] Creating new streaming plan for {} from {} {}", planId, sessionIndex, streamOperation.getDescription(),
+                        from, channel.description());
 
             // The main reason we create a StreamResultFuture on the receiving side is for JMX exposure.
             future = new StreamResultFuture(planId, streamOperation, pendingRepair, previewKind);
             StreamManager.instance.registerFollower(future);
         }
-        future.attachConnection(from, sessionIndex, channel);
-        logger.info("[Stream #{}, ID#{}] Received streaming plan for {} from {} channel.remote {} channel.local {} channel.id {}",
-                    planId, sessionIndex, streamOperation.getDescription(), from, channel.remoteAddress(), channel.localAddress(), channel.id());
+        future.initInbound(from, channel, messagingVersion, sessionIndex);
+        logger.info("[Stream #{}, ID#{}] Received streaming plan for {} from {} {}",
+                    planId, sessionIndex, streamOperation.getDescription(), from, channel.description());
         return future;
     }
 
@@ -135,9 +136,9 @@ public final class StreamResultFuture extends AsyncFuture<StreamState>
         return coordinator;
     }
 
-    private void attachConnection(InetAddressAndPort from, int sessionIndex, Channel channel)
+    private void initInbound(InetAddressAndPort from, StreamingChannel channel, int messagingVersion, int sessionIndex)
     {
-        StreamSession session = coordinator.getOrCreateSessionById(from, sessionIndex);
+        StreamSession session = coordinator.getOrCreateInboundSession(from, channel, messagingVersion, sessionIndex);
         session.init(this);
     }
 
diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java
index fbb4c61..4499024 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -24,9 +24,12 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 
+import javax.annotation.Nullable;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.*;
 
+import io.netty.channel.Channel;
 import io.netty.util.concurrent.Future;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -36,8 +39,6 @@ import org.apache.cassandra.utils.concurrent.FutureCombiner;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelId;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Range;
@@ -46,9 +47,8 @@ import org.apache.cassandra.gms.*;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.metrics.StreamingMetrics;
-import org.apache.cassandra.net.OutboundConnectionSettings;
 import org.apache.cassandra.schema.TableId;
-import org.apache.cassandra.streaming.async.NettyStreamingMessageSender;
+import org.apache.cassandra.streaming.async.StreamingMultiplexedChannel;
 import org.apache.cassandra.streaming.messages.*;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
@@ -57,6 +57,8 @@ import org.apache.cassandra.utils.NoSpamLogger;
 import static com.google.common.collect.Iterables.all;
 import static org.apache.cassandra.net.MessagingService.current_version;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+import static org.apache.cassandra.locator.InetAddressAndPort.hostAddressAndPort;
+import static org.apache.cassandra.utils.FBUtilities.getBroadcastAddressAndPort;
 
 /**
  * Handles the streaming a one or more streams to and from a specific remote node.
@@ -90,7 +92,7 @@ import static org.apache.cassandra.utils.Clock.Global.nanoTime;
  * 3. Streaming phase
  *
  *   (a) The streaming phase is started at each node by calling {@link StreamSession#startStreamingFiles(boolean)}.
- *       This will send, sequentially on each outbound streaming connection (see {@link NettyStreamingMessageSender}),
+ *       This will send, sequentially on each outbound streaming connection (see {@link StreamingMultiplexedChannel}),
  *       an {@link OutgoingStreamMessage} for each stream in each of the {@link StreamTransferTask}.
  *       Each {@link OutgoingStreamMessage} consists of a {@link StreamMessageHeader} that contains metadata about
  *       the stream, followed by the stream content itself. Once all the files for a {@link StreamTransferTask} are sent,
@@ -131,7 +133,7 @@ import static org.apache.cassandra.utils.Clock.Global.nanoTime;
  *
  * All messages which derive from {@link StreamMessage} are sent by the standard internode messaging
  * (via {@link org.apache.cassandra.net.MessagingService}, while the actual files themselves are sent by a special
- * "streaming" connection type. See {@link NettyStreamingMessageSender} for details. Because of the asynchronous
+ * "streaming" connection type. See {@link StreamingMultiplexedChannel} for details. Because of the asynchronous
  */
 public class StreamSession implements IEndpointStateChangeSubscriber
 {
@@ -148,7 +150,6 @@ public class StreamSession implements IEndpointStateChangeSubscriber
      * Each {@code StreamSession} is identified by this InetAddressAndPort which is broadcast address of the node streaming.
      */
     public final InetAddressAndPort peer;
-    private final OutboundConnectionSettings template;
 
     private final int index;
 
@@ -167,9 +168,10 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     final Map<String, Set<Range<Token>>> transferredRangesPerKeyspace = new HashMap<>();
 
     private final boolean isFollower;
-    private final NettyStreamingMessageSender messageSender;
+    private final StreamingMultiplexedChannel channel;
     // contains both inbound and outbound channels
-    private final ConcurrentMap<ChannelId, Channel> channels = new ConcurrentHashMap<>();
+    private final ConcurrentMap<Object, StreamingChannel> inbound = new ConcurrentHashMap<>();
+    private final ConcurrentMap<Object, StreamingChannel> outbound = new ConcurrentHashMap<>();
 
     // "maybeCompleted()" should be executed at most once. Because it can be executed asynchronously by IO
     // threads(serialization/deserialization) and stream messaging processing thread, causing connection closed before
@@ -230,21 +232,18 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     /**
      * Create new streaming session with the peer.
      */
-    public StreamSession(StreamOperation streamOperation, InetAddressAndPort peer, StreamConnectionFactory factory,
+    public StreamSession(StreamOperation streamOperation, InetAddressAndPort peer, StreamingChannel.Factory factory, @Nullable StreamingChannel controlChannel, int messagingVersion,
                          boolean isFollower, int index, UUID pendingRepair, PreviewKind previewKind)
     {
         this.streamOperation = streamOperation;
         this.peer = peer;
-        this.template = new OutboundConnectionSettings(peer);
         this.isFollower = isFollower;
         this.index = index;
 
-        this.messageSender = new NettyStreamingMessageSender(this, template, factory, current_version, previewKind.isPreview());
+        this.channel = new StreamingMultiplexedChannel(this, factory, peer, controlChannel, messagingVersion);
         this.metrics = StreamingMetrics.get(peer);
         this.pendingRepair = pendingRepair;
         this.previewKind = previewKind;
-
-        logger.debug("Creating stream session to {} as {}", template, isFollower ? "follower" : "initiator");
     }
 
     public boolean isFollower()
@@ -309,18 +308,19 @@ public class StreamSession implements IEndpointStateChangeSubscriber
      * Attach a channel to this session upon receiving the first inbound message.
      *
      * @param channel The channel to attach.
-     * @param isControlChannel If the channel is the one to send control messages to.
      * @return False if the channel was already attached, true otherwise.
      */
-    public synchronized boolean attachInbound(Channel channel, boolean isControlChannel)
+    public synchronized boolean attachInbound(StreamingChannel channel)
     {
         failIfFinished();
 
-        if (!messageSender.hasControlChannel() && isControlChannel)
-            messageSender.injectControlMessageChannel(channel);
-
-        channel.closeFuture().addListener(ignored -> onChannelClose(channel));
-        return channels.putIfAbsent(channel.id(), channel) == null;
+        boolean attached = inbound.putIfAbsent(channel.id(), channel) == null;
+        if (attached)
+            channel.onClose(() -> {
+                if (null != inbound.remove(channel.id()) && inbound.isEmpty())
+                    this.channel.close();
+            });
+        return attached;
     }
 
     /**
@@ -329,22 +329,14 @@ public class StreamSession implements IEndpointStateChangeSubscriber
      * @param channel The channel to attach.
      * @return False if the channel was already attached, true otherwise.
      */
-    public synchronized boolean attachOutbound(Channel channel)
+    public synchronized boolean attachOutbound(StreamingChannel channel)
     {
         failIfFinished();
 
-        channel.closeFuture().addListener(ignored -> onChannelClose(channel));
-        return channels.putIfAbsent(channel.id(), channel) == null;
-    }
-
-    /**
-     * On channel closing, if no channels are left just close the message sender; this must be closed last to ensure
-     * keep alive messages are sent until the very end of the streaming session.
-     */
-    private void onChannelClose(Channel channel)
-    {
-        if (channels.remove(channel.id()) != null && channels.isEmpty())
-            messageSender.close();
+        boolean attached = outbound.putIfAbsent(channel.id(), channel) == null;
+        if (attached)
+            channel.onClose(() -> outbound.remove(channel.id()));
+        return attached;
     }
 
     /**
@@ -362,9 +354,17 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         try
         {
             logger.info("[Stream #{}] Starting streaming to {}{}", planId(),
-                                                                   peer,
-                                                                   template.connectTo == null ? "" : " through " + template.connectTo);
-            messageSender.initialize();
+                        hostAddressAndPort(channel.peer()),
+                        channel.connectedTo().equals(channel.peer()) ? "" : " through " + hostAddressAndPort(channel.connectedTo()));
+
+            StreamInitMessage message = new StreamInitMessage(getBroadcastAddressAndPort(),
+                                                              sessionIndex(),
+                                                              planId(),
+                                                              streamOperation(),
+                                                              getPendingRepair(),
+                                                              getPreviewKind());
+
+            channel.sendControlMessage(message);
             onInitializationComplete();
         }
         catch (Exception e)
@@ -503,8 +503,9 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         // due to failure, channels should be always closed regardless, even if this is not the initator.
         if (!isFollower || state != State.COMPLETE)
         {
-            logger.debug("[Stream #{}] Will close attached channels {}", planId(), channels);
-            channels.values().forEach(channel -> futures.add(channel.close()));
+            logger.debug("[Stream #{}] Will close attached inbound {} and outbound {} channels", planId(), inbound, outbound);
+            inbound.values().forEach(channel -> futures.add(channel.close()));
+            outbound.values().forEach(channel -> futures.add(channel.close()));
         }
 
         sink.onClose(peer);
@@ -549,9 +550,9 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         return state;
     }
 
-    public NettyStreamingMessageSender getMessageSender()
+    public StreamingMultiplexedChannel getChannel()
     {
-        return messageSender;
+        return channel;
     }
 
     /**
@@ -625,7 +626,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
             prepare.summaries.add(task.getSummary());
         }
 
-        messageSender.sendMessage(prepare);
+        channel.sendControlMessage(prepare);
     }
 
     /**
@@ -657,8 +658,8 @@ public class StreamSession implements IEndpointStateChangeSubscriber
 
         logError(e);
         // send session failure message
-        if (messageSender.connected())
-            messageSender.sendMessage(new SessionFailedMessage());
+        if (channel.connected())
+            channel.sendControlMessage(new SessionFailedMessage());
         // fail session
         return closeSession(State.FAILED);
     }
@@ -669,16 +670,16 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         {
             logger.error("[Stream #{}] Did not receive response from peer {}{} for {} secs. Is peer down? " +
                          "If not, maybe try increasing streaming_keep_alive_period_in_secs.", planId(),
-                         peer.getHostAddressAndPort(),
-                         template.connectTo == null ? "" : " through " + template.connectTo.getHostAddressAndPort(),
+                         hostAddressAndPort(channel.peer()),
+                         channel.peer().equals(channel.connectedTo()) ? "" : " through " + hostAddressAndPort(channel.connectedTo()),
                          2 * DatabaseDescriptor.getStreamingKeepAlivePeriod(),
                          e);
         }
         else
         {
             logger.error("[Stream #{}] Streaming error occurred on session with peer {}{}", planId(),
-                         peer.getHostAddressAndPort(),
-                         template.connectTo == null ? "" : " through " + template.connectTo.getHostAddressAndPort(),
+                         hostAddressAndPort(channel.peer()),
+                         channel.peer().equals(channel.connectedTo()) ? "" : " through " + hostAddressAndPort(channel.connectedTo()),
                          e);
         }
     }
@@ -717,7 +718,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         if (!peer.equals(FBUtilities.getBroadcastAddressAndPort()))
             for (StreamTransferTask task : transfers.values())
                 prepareSynAck.summaries.add(task.getSummary());
-        messageSender.sendMessage(prepareSynAck);
+        channel.sendControlMessage(prepareSynAck);
 
         streamResult.handleSessionPrepared(this);
 
@@ -736,7 +737,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
 
             // only send the (final) ACK if we are expecting the peer to send this node (the initiator) some files
             if (!isPreview())
-                messageSender.sendMessage(new PrepareAckMessage());
+                channel.sendControlMessage(new PrepareAckMessage());
         }
 
         if (isPreview())
@@ -793,7 +794,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         StreamingMetrics.totalIncomingBytes.inc(headerSize);
         metrics.incomingBytes.inc(headerSize);
         // send back file received message
-        messageSender.sendMessage(new ReceivedMessage(message.header.tableId, message.header.sequenceNumber));
+        channel.sendControlMessage(new ReceivedMessage(message.header.tableId, message.header.sequenceNumber));
         StreamHook.instance.reportIncomingStream(message.header.tableId, message.stream, this, message.header.sequenceNumber);
         long receivedStartNanos = nanoTime();
         try
@@ -872,7 +873,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         }
         else
         {
-            messageSender.sendMessage(new CompleteMessage());
+            channel.sendControlMessage(new CompleteMessage());
             closeSession(State.COMPLETE);
         }
 
@@ -899,8 +900,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         List<StreamSummary> transferSummaries = Lists.newArrayList();
         for (StreamTask transfer : transfers.values())
             transferSummaries.add(transfer.getSummary());
-        // TODO: the connectTo treatment here is peculiar, and needs thinking about - since the connection factory can change it
-        return new SessionInfo(peer, index, template.connectTo == null ? peer : template.connectTo, receivingSummaries, transferSummaries, state);
+        return new SessionInfo(channel.peer(), index, channel.connectedTo(), receivingSummaries, transferSummaries, state);
     }
 
     public synchronized void taskCompleted(StreamReceiveTask completedTask)
@@ -979,7 +979,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
                 {
                     // pass the session planId/index to the OFM (which is only set at init(), after the transfers have already been created)
                     ofm.header.addSessionInfo(this);
-                    messageSender.sendMessage(ofm);
+                    channel.sendControlMessage(ofm);
                 }
             }
             else
@@ -1040,4 +1040,34 @@ public class StreamSession implements IEndpointStateChangeSubscriber
          */
         public void onClose(InetAddressAndPort from);
     }
+
+    public static String createLogTag(StreamSession session)
+    {
+        return createLogTag(session, (Object) null);
+    }
+
+    public static String createLogTag(StreamSession session, StreamingChannel channel)
+    {
+        return createLogTag(session, channel == null ? null : channel.id());
+    }
+
+    public static String createLogTag(StreamSession session, Channel channel)
+    {
+        return createLogTag(session, channel == null ? null : channel.id());
+    }
+
+    public static String createLogTag(StreamSession session, Object channelId)
+    {
+        StringBuilder sb = new StringBuilder(64);
+        sb.append("[Stream");
+
+        if (session != null)
+            sb.append(" #").append(session.planId());
+
+        if (channelId != null)
+            sb.append(" channel: ").append(channelId);
+
+        sb.append(']');
+        return sb.toString();
+    }
 }
diff --git a/src/java/org/apache/cassandra/streaming/StreamTransferTask.java b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
index 2fb1213..70ad7d8 100644
--- a/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
@@ -91,7 +91,7 @@ public class StreamTransferTask extends StreamTask
             if (stream != null)
                 stream.complete();
 
-            logger.debug("recevied sequenceNumber {}, remaining files {}", sequenceNumber, streams.keySet());
+            logger.debug("received sequenceNumber {}, remaining files {}", sequenceNumber, streams.keySet());
             signalComplete = streams.isEmpty();
         }
 
diff --git a/src/java/org/apache/cassandra/streaming/StreamingChannel.java b/src/java/org/apache/cassandra/streaming/StreamingChannel.java
new file mode 100644
index 0000000..af6e68e
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamingChannel.java
@@ -0,0 +1,73 @@
+/*
+ * 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.streaming;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.function.IntFunction;
+
+import io.netty.util.concurrent.Future;
+import org.apache.cassandra.streaming.async.NettyStreamingConnectionFactory;
+
+public interface StreamingChannel
+{
+    public interface Factory
+    {
+        public static class Global
+        {
+            private static StreamingChannel.Factory FACTORY = new NettyStreamingConnectionFactory();
+            public static StreamingChannel.Factory streamingFactory()
+            {
+                return FACTORY;
+            }
+
+            public static void unsafeSet(StreamingChannel.Factory factory)
+            {
+                FACTORY = factory;
+            }
+        }
+
+        StreamingChannel create(InetSocketAddress to, int messagingVersion, Kind kind) throws IOException;
+    }
+
+    public enum Kind { CONTROL, FILE }
+
+    public interface Send
+    {
+        void send(IntFunction<StreamingDataOutputPlus> outSupplier) throws IOException;
+    }
+
+    Object id();
+    String description();
+
+    InetSocketAddress peer();
+    InetSocketAddress connectedTo();
+    boolean connected();
+
+    StreamingDataInputPlus in();
+
+    /**
+     * until closed, cannot invoke {@link #send(Send)}
+     */
+    StreamingDataOutputPlus acquireOut();
+    Future<?> send(Send send) throws IOException;
+
+    Future<?> close();
+    void onClose(Runnable runOnClose);
+}
diff --git a/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java b/src/java/org/apache/cassandra/streaming/StreamingDataInputPlus.java
similarity index 76%
copy from src/java/org/apache/cassandra/streaming/StreamingMessageSender.java
copy to src/java/org/apache/cassandra/streaming/StreamingDataInputPlus.java
index 9562981..f3e6111 100644
--- a/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java
+++ b/src/java/org/apache/cassandra/streaming/StreamingDataInputPlus.java
@@ -18,17 +18,12 @@
 
 package org.apache.cassandra.streaming;
 
-import java.io.IOException;
+import java.io.Closeable;
 
-import org.apache.cassandra.streaming.messages.StreamMessage;
+import org.apache.cassandra.io.util.DataInputPlus;
 
-public interface StreamingMessageSender
+public interface StreamingDataInputPlus extends DataInputPlus, Closeable
 {
-    void initialize() throws IOException;
-
-    void sendMessage(StreamMessage message) throws IOException;
-
-    boolean connected();
-
+    @Override
     void close();
 }
diff --git a/src/java/org/apache/cassandra/streaming/StreamingDataOutputPlus.java b/src/java/org/apache/cassandra/streaming/StreamingDataOutputPlus.java
new file mode 100644
index 0000000..d4a514b
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamingDataOutputPlus.java
@@ -0,0 +1,80 @@
+/*
+ * 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.streaming;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+import io.netty.channel.FileRegion;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+public interface StreamingDataOutputPlus extends DataOutputPlus, Closeable
+{
+    interface BufferSupplier
+    {
+        /**
+         * Request a buffer with at least the given capacity.
+         * This method may only be invoked once, and the lifetime of buffer it returns will be managed
+         * by the AsyncChannelOutputPlus it was created for.
+         */
+        ByteBuffer get(int capacity) throws IOException;
+    }
+
+    interface Write
+    {
+        /**
+         * Write to a buffer, and flush its contents to the channel.
+         * <p>
+         * The lifetime of the buffer will be managed by the AsyncChannelOutputPlus you issue this Write to.
+         * If the method exits successfully, the contents of the buffer will be written to the channel, otherwise
+         * the buffer will be cleaned and the exception propagated to the caller.
+         */
+        void write(BufferSupplier supplier) throws IOException;
+    }
+
+    interface RateLimiter
+    {
+        void acquire(int bytes);
+    }
+
+    /**
+     * Provide a lambda that can request a buffer of suitable size, then fill the buffer and have
+     * that buffer written and flushed to the underlying channel, without having to handle buffer
+     * allocation, lifetime or cleanup, including in case of exceptions.
+     * <p>
+     * Any exception thrown by the Write will be propagated to the caller, after any buffer is cleaned up.
+     */
+    int writeToChannel(Write write, RateLimiter limiter) throws IOException;
+
+    /**
+     * Writes all data in file channel to stream: <br>
+     * * For zero-copy-streaming, 1MiB at a time, with at most 2MiB in flight at once. <br>
+     * * For streaming with SSL, 64kb at a time, with at most 32+64kb (default low water mark + batch size) in flight. <br>
+     * <p>
+     * This method takes ownership of the provided {@link FileChannel}.
+     * <p>
+     * WARNING: this method blocks only for permission to write to the netty channel; it exits before
+     * the {@link FileRegion}(zero-copy) or {@link ByteBuffer}(ssl) is flushed to the network.
+     */
+    long writeFileToChannel(FileChannel file, RateLimiter limiter) throws IOException;
+
+    default void flush() throws IOException {}
+}
diff --git a/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java b/src/java/org/apache/cassandra/streaming/StreamingDataOutputPlusFixed.java
similarity index 52%
rename from src/java/org/apache/cassandra/streaming/StreamingMessageSender.java
rename to src/java/org/apache/cassandra/streaming/StreamingDataOutputPlusFixed.java
index 9562981..2159090 100644
--- a/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java
+++ b/src/java/org/apache/cassandra/streaming/StreamingDataOutputPlusFixed.java
@@ -19,16 +19,32 @@
 package org.apache.cassandra.streaming;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
 
-import org.apache.cassandra.streaming.messages.StreamMessage;
+import org.apache.cassandra.io.util.DataOutputBufferFixed;
 
-public interface StreamingMessageSender
+public class StreamingDataOutputPlusFixed extends DataOutputBufferFixed implements StreamingDataOutputPlus
 {
-    void initialize() throws IOException;
+    public StreamingDataOutputPlusFixed(ByteBuffer buffer)
+    {
+        super(buffer);
+    }
 
-    void sendMessage(StreamMessage message) throws IOException;
+    @Override
+    public int writeToChannel(Write write, RateLimiter limiter) throws IOException
+    {
+        int position = buffer.position();
+        write.write(size -> buffer);
+        return buffer.position() - position;
+    }
 
-    boolean connected();
-
-    void close();
+    @Override
+    public long writeFileToChannel(FileChannel file, RateLimiter limiter) throws IOException
+    {
+        long count = 0;
+        long tmp;
+        while (0 <= (tmp = file.read(buffer))) count += tmp;
+        return count;
+    }
 }
diff --git a/src/java/org/apache/cassandra/streaming/async/NettyStreamingChannel.java b/src/java/org/apache/cassandra/streaming/async/NettyStreamingChannel.java
new file mode 100644
index 0000000..fb5d37a
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/async/NettyStreamingChannel.java
@@ -0,0 +1,260 @@
+/*
+ * 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.streaming.async;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.IntFunction;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.util.AttributeKey;
+import io.netty.util.ReferenceCountUtil;
+import org.apache.cassandra.net.AsyncChannelPromise;
+import org.apache.cassandra.net.AsyncStreamingInputPlus;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.net.GlobalBufferPoolAllocator;
+import org.apache.cassandra.streaming.StreamingChannel;
+import org.apache.cassandra.streaming.StreamingDataInputPlus;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
+import org.apache.cassandra.streaming.StreamingDataOutputPlusFixed;
+import org.apache.cassandra.utils.concurrent.Future;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
+
+import static io.netty.util.AttributeKey.valueOf;
+import static java.lang.Boolean.FALSE;
+
+public class NettyStreamingChannel extends ChannelInboundHandlerAdapter implements StreamingChannel
+{
+    private static final Logger logger = LoggerFactory.getLogger(NettyStreamingChannel.class);
+    private static volatile boolean trackInboundHandlers = false;
+    private static Collection<NettyStreamingChannel> inboundHandlers;
+
+    @VisibleForTesting
+    static final AttributeKey<Boolean> TRANSFERRING_FILE_ATTR = valueOf("transferringFile");
+    final int messagingVersion;
+    final Channel channel;
+
+    /**
+     * A collection of {@link ByteBuf}s that are yet to be processed. Incoming buffers are first dropped into this
+     * structure, and then consumed.
+     * <p>
+     * For thread safety, this structure's resources are released on the consuming thread
+     * (via {@link AsyncStreamingInputPlus#close()},
+     * but the producing side calls {@link AsyncStreamingInputPlus#requestClosure()} to notify the input that is should close.
+     */
+    @VisibleForTesting
+    final AsyncStreamingInputPlus in;
+
+    private volatile boolean closed;
+
+    public NettyStreamingChannel(int messagingVersion, Channel channel, Kind kind)
+    {
+        this.messagingVersion = messagingVersion;
+        this.channel = channel;
+        channel.attr(TRANSFERRING_FILE_ATTR).set(FALSE);
+        if (kind == Kind.CONTROL)
+        {
+            if (trackInboundHandlers)
+                inboundHandlers.add(this);
+            in = new AsyncStreamingInputPlus(channel);
+        }
+        else in = null;
+    }
+
+    @Override
+    public Object id()
+    {
+        return channel.id();
+    }
+
+    @Override
+    public String description()
+    {
+        return "channel.remote " + channel.remoteAddress() +
+               " channel.local " + channel.localAddress() +
+               " channel.id " + channel.id();
+    }
+
+    @Override
+    public InetSocketAddress peer()
+    {
+        return (InetSocketAddress) channel.remoteAddress();
+    }
+
+    @Override
+    public InetSocketAddress connectedTo()
+    {
+        return peer();
+    }
+
+    @Override
+    public boolean connected()
+    {
+        return channel.isOpen();
+    }
+
+    public StreamingDataInputPlus in()
+    {
+        return in;
+    }
+
+    public StreamingDataOutputPlus acquireOut()
+    {
+        if (!channel.attr(TRANSFERRING_FILE_ATTR).compareAndSet(false, true))
+            throw new IllegalStateException("channel's transferring state is currently set to true. refusing to start new stream");
+
+        return new AsyncStreamingOutputPlus(channel)
+        {
+            @Override
+            public void close() throws IOException
+            {
+                try
+                {
+                    super.close();
+                }
+                finally
+                {
+                    NettyStreamingChannel.this.channel.attr(TRANSFERRING_FILE_ATTR).set(FALSE);
+                }
+            }
+        };
+    }
+
+    public Future<?> send(Send send)
+    {
+        class Factory implements IntFunction<StreamingDataOutputPlus>
+        {
+            ByteBuf buf;
+            ByteBuffer buffer;
+
+            @Override
+            public StreamingDataOutputPlus apply(int size)
+            {
+                buf = GlobalBufferPoolAllocator.instance.buffer(size);
+                buffer = buf.nioBuffer(buf.writerIndex(), size);
+                return new StreamingDataOutputPlusFixed(buffer);
+            }
+        }
+
+        Factory factory = new Factory();
+        try
+        {
+            send.send(factory);
+            ByteBuf buf = factory.buf;
+            ByteBuffer buffer = factory.buffer;
+            try
+            {
+                assert buffer.position() == buffer.limit();
+                buf.writerIndex(buffer.position());
+                AsyncChannelPromise promise = new AsyncChannelPromise(channel);
+                channel.writeAndFlush(buf, promise);
+                return promise;
+            }
+            catch (Throwable t)
+            {
+                buf.release();
+                throw t;
+            }
+        }
+        catch (Throwable t)
+        {
+            return ImmediateFuture.failure(t);
+        }
+    }
+
+    @Override
+    public synchronized io.netty.util.concurrent.Future<?> close()
+    {
+        if (closed)
+            return channel.closeFuture();
+
+        closed = true;
+        if (in != null)
+        {
+            in.requestClosure();
+            if (trackInboundHandlers)
+                inboundHandlers.remove(this);
+        }
+
+        return channel.close();
+    }
+
+    @Override
+    public void onClose(Runnable runOnClose)
+    {
+        channel.closeFuture().addListener(ignore -> runOnClose.run());
+    }
+
+    @Override
+    public void channelRead(ChannelHandlerContext ctx, Object message)
+    {
+        if (closed || !(message instanceof ByteBuf) || !in.append((ByteBuf) message))
+            ReferenceCountUtil.release(message);
+    }
+
+    @Override
+    public void channelInactive(ChannelHandlerContext ctx)
+    {
+        close();
+        ctx.fireChannelInactive();
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
+    {
+        if (cause instanceof IOException)
+            logger.trace("connection problem while streaming", cause);
+        else
+            logger.warn("exception occurred while in processing streaming data", cause);
+        close();
+    }
+
+    /** Shutdown for in-JVM tests. For any other usage, tracking of active inbound streaming handlers
+     *  should be revisted first and in-JVM shutdown refactored with it.
+     *  This does not prevent new inbound handlers being added after shutdown, nor is not thread-safe
+     *  around new inbound handlers being opened during shutdown.
+     */
+    @VisibleForTesting
+    public static void shutdown()
+    {
+        assert trackInboundHandlers : "in-JVM tests required tracking of inbound streaming handlers";
+
+        inboundHandlers.forEach(NettyStreamingChannel::close);
+        inboundHandlers.clear();
+    }
+
+    public static void trackInboundHandlers()
+    {
+        inboundHandlers = Collections.newSetFromMap(new ConcurrentHashMap<>());
+        trackInboundHandlers = true;
+    }
+
+}
diff --git a/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java b/src/java/org/apache/cassandra/streaming/async/NettyStreamingConnectionFactory.java
similarity index 64%
rename from src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
rename to src/java/org/apache/cassandra/streaming/async/NettyStreamingConnectionFactory.java
index 5f2163f..a6261bf 100644
--- a/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
+++ b/src/java/org/apache/cassandra/streaming/async/NettyStreamingConnectionFactory.java
@@ -16,13 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.cassandra.streaming;
+package org.apache.cassandra.streaming.async;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 
 import com.google.common.annotations.VisibleForTesting;
 
 import io.netty.channel.Channel;
+import io.netty.channel.ChannelPipeline;
 import io.netty.channel.EventLoop;
 import io.netty.util.concurrent.Future;
 import org.apache.cassandra.net.ConnectionCategory;
@@ -30,16 +32,17 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.OutboundConnectionInitiator.Result;
 import org.apache.cassandra.net.OutboundConnectionInitiator.Result.StreamingSuccess;
 import org.apache.cassandra.net.OutboundConnectionSettings;
+import org.apache.cassandra.streaming.StreamingChannel;
 
+import static org.apache.cassandra.locator.InetAddressAndPort.getByAddress;
 import static org.apache.cassandra.net.OutboundConnectionInitiator.initiateStreaming;
 
-public class DefaultConnectionFactory implements StreamConnectionFactory
+public class NettyStreamingConnectionFactory implements StreamingChannel.Factory
 {
     @VisibleForTesting
     public static int MAX_CONNECT_ATTEMPTS = 3;
 
-    @Override
-    public Channel createConnection(OutboundConnectionSettings template, int messagingVersion) throws IOException
+    public static NettyStreamingChannel connect(OutboundConnectionSettings template, int messagingVersion, StreamingChannel.Kind kind) throws IOException
     {
         EventLoop eventLoop = MessagingService.instance().socketFactory.outboundStreamingGroup().next();
 
@@ -49,10 +52,25 @@ public class DefaultConnectionFactory implements StreamConnectionFactory
             Future<Result<StreamingSuccess>> result = initiateStreaming(eventLoop, template.withDefaults(ConnectionCategory.STREAMING), messagingVersion);
             result.awaitUninterruptibly(); // initiate has its own timeout, so this is "guaranteed" to return relatively promptly
             if (result.isSuccess())
-                return result.getNow().success().channel;
+            {
+                Channel channel = result.getNow().success().channel;
+                NettyStreamingChannel streamingChannel = new NettyStreamingChannel(messagingVersion, channel, kind);
+                if (kind == StreamingChannel.Kind.CONTROL)
+                {
+                    ChannelPipeline pipeline = channel.pipeline();
+                    pipeline.addLast("stream", streamingChannel);
+                }
+                return streamingChannel;
+            }
 
             if (++attempts == MAX_CONNECT_ATTEMPTS)
                 throw new IOException("failed to connect to " + template.to + " for streaming data", result.cause());
         }
     }
+
+    @Override
+    public StreamingChannel create(InetSocketAddress to, int messagingVersion, StreamingChannel.Kind kind) throws IOException
+    {
+        return connect(new OutboundConnectionSettings(getByAddress(to)), messagingVersion, kind);
+    }
 }
diff --git a/src/java/org/apache/cassandra/streaming/async/StreamCompressionSerializer.java b/src/java/org/apache/cassandra/streaming/async/StreamCompressionSerializer.java
index fc1bde2..e34b2bd 100644
--- a/src/java/org/apache/cassandra/streaming/async/StreamCompressionSerializer.java
+++ b/src/java/org/apache/cassandra/streaming/async/StreamCompressionSerializer.java
@@ -28,6 +28,7 @@ import net.jpountz.lz4.LZ4Compressor;
 import net.jpountz.lz4.LZ4SafeDecompressor;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 
 import static org.apache.cassandra.net.MessagingService.current_version;
 
@@ -54,7 +55,7 @@ public class StreamCompressionSerializer
      */
     private static final int HEADER_LENGTH = 8;
 
-    public static AsyncStreamingOutputPlus.Write serialize(LZ4Compressor compressor, ByteBuffer in, int version)
+    public static StreamingDataOutputPlus.Write serialize(LZ4Compressor compressor, ByteBuffer in, int version)
     {
         assert version == current_version;
         return bufferSupplier -> {
diff --git a/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java b/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java
deleted file mode 100644
index 3b9c172..0000000
--- a/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java
+++ /dev/null
@@ -1,256 +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.streaming.async;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import javax.annotation.Nullable;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.Uninterruptibles;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-import io.netty.util.ReferenceCountUtil;
-import io.netty.util.concurrent.FastThreadLocalThread;
-import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.net.AsyncStreamingInputPlus;
-import org.apache.cassandra.streaming.StreamReceiveException;
-import org.apache.cassandra.streaming.StreamSession;
-import org.apache.cassandra.streaming.messages.KeepAliveMessage;
-import org.apache.cassandra.streaming.messages.StreamInitMessage;
-import org.apache.cassandra.streaming.messages.StreamMessage;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-
-import static org.apache.cassandra.streaming.async.NettyStreamingMessageSender.createLogTag;
-
-/**
- * Handles the inbound side of streaming messages and stream data. From the incoming data, we derserialize the message
- * including the actual stream data itself. Because the reading and deserialization of streams is a blocking affair,
- * we can't block the netty event loop. Thus we have a background thread perform all the blocking deserialization.
- */
-public class StreamingInboundHandler extends ChannelInboundHandlerAdapter
-{
-    private static final Logger logger = LoggerFactory.getLogger(StreamingInboundHandler.class);
-    private static volatile boolean trackInboundHandlers = false;
-    private static Collection<StreamingInboundHandler> inboundHandlers;
-    private final InetAddressAndPort remoteAddress;
-    private final int protocolVersion;
-
-    private final StreamSession session;
-
-    /**
-     * A collection of {@link ByteBuf}s that are yet to be processed. Incoming buffers are first dropped into this
-     * structure, and then consumed.
-     * <p>
-     * For thread safety, this structure's resources are released on the consuming thread
-     * (via {@link AsyncStreamingInputPlus#close()},
-     * but the producing side calls {@link AsyncStreamingInputPlus#requestClosure()} to notify the input that is should close.
-     */
-    private AsyncStreamingInputPlus buffers;
-
-    private volatile boolean closed;
-
-    public StreamingInboundHandler(InetAddressAndPort remoteAddress, int protocolVersion, @Nullable StreamSession session)
-    {
-        this.remoteAddress = remoteAddress;
-        this.protocolVersion = protocolVersion;
-        this.session = session;
-        if (trackInboundHandlers)
-            inboundHandlers.add(this);
-    }
-
-    @Override
-    @SuppressWarnings("resource")
-    public void handlerAdded(ChannelHandlerContext ctx)
-    {
-        buffers = new AsyncStreamingInputPlus(ctx.channel());
-        Thread blockingIOThread = new FastThreadLocalThread(new StreamDeserializingTask(session, ctx.channel()),
-                                                            String.format("Stream-Deserializer-%s-%s", remoteAddress.toString(), ctx.channel().id()));
-        blockingIOThread.setDaemon(true);
-        blockingIOThread.start();
-    }
-
-    @Override
-    public void channelRead(ChannelHandlerContext ctx, Object message)
-    {
-        if (closed || !(message instanceof ByteBuf) || !buffers.append((ByteBuf) message))
-            ReferenceCountUtil.release(message);
-    }
-
-    @Override
-    public void channelInactive(ChannelHandlerContext ctx)
-    {
-        close();
-        ctx.fireChannelInactive();
-    }
-
-    void close()
-    {
-        closed = true;
-        buffers.requestClosure();
-        if (trackInboundHandlers)
-            inboundHandlers.remove(this);
-    }
-
-    @Override
-    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
-    {
-        if (cause instanceof IOException)
-            logger.trace("connection problem while streaming", cause);
-        else
-            logger.warn("exception occurred while in processing streaming data", cause);
-        close();
-    }
-
-    /**
-     * For testing only!!
-     */
-    void setPendingBuffers(AsyncStreamingInputPlus bufChannel)
-    {
-        this.buffers = bufChannel;
-    }
-
-    /**
-     * The task that performs the actual deserialization.
-     */
-    class StreamDeserializingTask implements Runnable
-    {
-        private final Channel channel;
-
-        @VisibleForTesting
-        StreamSession session;
-
-        StreamDeserializingTask(StreamSession session, Channel channel)
-        {
-            this.session = session;
-            this.channel = channel;
-        }
-
-        @Override
-        public void run()
-        {
-            try
-            {
-                while (true)
-                {
-                    buffers.maybeIssueRead();
-
-                    // do a check of available bytes and possibly sleep some amount of time (then continue).
-                    // this way we can break out of run() sanely or we end up blocking indefintely in StreamMessage.deserialize()
-                    while (buffers.isEmpty())
-                    {
-                        if (closed)
-                            return;
-
-                        Uninterruptibles.sleepUninterruptibly(400, TimeUnit.MILLISECONDS);
-                    }
-
-                    StreamMessage message = StreamMessage.deserialize(buffers, protocolVersion);
-
-                    // keep-alives don't necessarily need to be tied to a session (they could be arrive before or after
-                    // wrt session lifecycle, due to races), just log that we received the message and carry on
-                    if (message instanceof KeepAliveMessage)
-                    {
-                        if (logger.isDebugEnabled())
-                            logger.debug("{} Received {}", createLogTag(session, channel), message);
-                        continue;
-                    }
-
-                    if (session == null)
-                        session = deriveSession(message);
-
-                    if (logger.isDebugEnabled())
-                        logger.debug("{} Received {}", createLogTag(session, channel), message);
-
-                    session.messageReceived(message);
-                }
-            }
-            catch (Throwable t)
-            {
-                JVMStabilityInspector.inspectThrowable(t);
-                if (session != null)
-                {
-                    session.onError(t);
-                }
-                else if (t instanceof StreamReceiveException)
-                {
-                    ((StreamReceiveException)t).session.onError(t);
-                }
-                else
-                {
-                    logger.error("{} stream operation from {} failed", createLogTag(session, channel), remoteAddress, t);
-                }
-            }
-            finally
-            {
-                channel.close();
-                closed = true;
-
-                if (buffers != null)
-                {
-                    // request closure again as the original request could have raced with receiving a
-                    // message and been consumed in the message receive loop above.  Otherweise
-                    // buffers could hang indefinitely on the queue.poll.
-                    buffers.requestClosure();
-                    buffers.close();
-                }
-            }
-        }
-
-        StreamSession deriveSession(StreamMessage message)
-        {
-            // StreamInitMessage starts a new channel here, but IncomingStreamMessage needs a session
-            // to be established a priori
-            StreamSession streamSession = message.getOrCreateSession(channel);
-
-            // Attach this channel to the session: this only happens upon receiving the first init message as a follower;
-            // in all other cases, no new control channel will be added, as the proper control channel will be already attached.
-            streamSession.attachInbound(channel, message instanceof StreamInitMessage);
-            return streamSession;
-        }
-    }
-
-    /** Shutdown for in-JVM tests. For any other usage, tracking of active inbound streaming handlers
-     *  should be revisted first and in-JVM shutdown refactored with it.
-     *  This does not prevent new inbound handlers being added after shutdown, nor is not thread-safe
-     *  around new inbound handlers being opened during shutdown.
-      */
-    @VisibleForTesting
-    public static void shutdown()
-    {
-        assert trackInboundHandlers : "in-JVM tests required tracking of inbound streaming handlers";
-
-        inboundHandlers.forEach(StreamingInboundHandler::close);
-        inboundHandlers.clear();
-    }
-
-    public static void trackInboundHandlers()
-    {
-        inboundHandlers = Collections.newSetFromMap(new ConcurrentHashMap<>());
-        trackInboundHandlers = true;
-    }
-}
diff --git a/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java b/src/java/org/apache/cassandra/streaming/async/StreamingMultiplexedChannel.java
similarity index 53%
rename from src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java
rename to src/java/org/apache/cassandra/streaming/async/StreamingMultiplexedChannel.java
index e3b2fef..711a752 100644
--- a/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java
+++ b/src/java/org/apache/cassandra/streaming/async/StreamingMultiplexedChannel.java
@@ -20,42 +20,36 @@ package org.apache.cassandra.streaming.async;
 
 import java.io.IOError;
 import java.io.IOException;
-import java.nio.ByteBuffer;
+import java.net.InetSocketAddress;
 import java.nio.channels.ClosedByInterruptException;
 import java.util.Collection;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ScheduledFuture;
 
-import com.google.common.annotations.VisibleForTesting;
+import javax.annotation.Nullable;
+
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.streaming.StreamDeserializingTask;
+import org.apache.cassandra.streaming.StreamingChannel;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
 import org.apache.cassandra.utils.concurrent.Semaphore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import io.netty.buffer.ByteBuf;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelPipeline;
-import io.netty.util.AttributeKey;
 import io.netty.util.concurrent.Future;
-import io.netty.util.concurrent.GenericFutureListener;
 import org.apache.cassandra.concurrent.ExecutorPlus;
-import org.apache.cassandra.io.util.DataOutputBufferFixed;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.net.OutboundConnectionSettings;
-import org.apache.cassandra.net.AsyncStreamingOutputPlus;
-import org.apache.cassandra.streaming.StreamConnectionFactory;
 import org.apache.cassandra.streaming.StreamSession;
-import org.apache.cassandra.streaming.StreamingMessageSender;
 import org.apache.cassandra.streaming.messages.IncomingStreamMessage;
 import org.apache.cassandra.streaming.messages.KeepAliveMessage;
 import org.apache.cassandra.streaming.messages.OutgoingStreamMessage;
-import org.apache.cassandra.streaming.messages.StreamInitMessage;
 import org.apache.cassandra.streaming.messages.StreamMessage;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 import static com.google.common.base.Throwables.getRootCause;
-import static io.netty.util.AttributeKey.valueOf;
-import static java.lang.Boolean.FALSE;
 import static java.lang.Integer.parseInt;
 import static java.lang.String.format;
 import static java.lang.System.getProperty;
@@ -64,11 +58,10 @@ import static java.util.concurrent.TimeUnit.*;
 import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.config.Config.PROPERTY_PREFIX;
 import static org.apache.cassandra.config.DatabaseDescriptor.getStreamingKeepAlivePeriod;
-import static org.apache.cassandra.net.AsyncChannelPromise.writeAndFlush;
+import static org.apache.cassandra.streaming.StreamSession.createLogTag;
 import static org.apache.cassandra.streaming.messages.StreamMessage.serialize;
 import static org.apache.cassandra.streaming.messages.StreamMessage.serializedSize;
 import static org.apache.cassandra.utils.FBUtilities.getAvailableProcessors;
-import static org.apache.cassandra.utils.FBUtilities.getBroadcastAddressAndPort;
 import static org.apache.cassandra.utils.JVMStabilityInspector.inspectThrowable;
 import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 import static org.apache.cassandra.utils.concurrent.Semaphore.newFairSemaphore;
@@ -91,23 +84,20 @@ import static org.apache.cassandra.utils.Clock.Global.nanoTime;
  * (we acquire the permits from the rate limiter, or the socket drains). However, we need to ensure that
  * no other messages are submitted to the same channel while the current stream is still being processed.
  */
-public class NettyStreamingMessageSender implements StreamingMessageSender
+public class StreamingMultiplexedChannel
 {
-    private static final Logger logger = LoggerFactory.getLogger(NettyStreamingMessageSender.class);
+    private static final Logger logger = LoggerFactory.getLogger(StreamingMultiplexedChannel.class);
 
     private static final int DEFAULT_MAX_PARALLEL_TRANSFERS = getAvailableProcessors();
     private static final int MAX_PARALLEL_TRANSFERS = parseInt(getProperty(PROPERTY_PREFIX + "streaming.session.parallelTransfers", Integer.toString(DEFAULT_MAX_PARALLEL_TRANSFERS)));
 
-    private static final long DEFAULT_CLOSE_WAIT_IN_MILLIS = MINUTES.toMillis(5);
-
     // a simple mechansim for allowing a degree of fairness across multiple sessions
     private static final Semaphore fileTransferSemaphore = newFairSemaphore(DEFAULT_MAX_PARALLEL_TRANSFERS);
 
+    private final StreamingChannel.Factory factory;
+    private final InetAddressAndPort to;
     private final StreamSession session;
-    private final boolean isPreview;
-    private final int streamingVersion;
-    private final OutboundConnectionSettings template;
-    private final StreamConnectionFactory factory;
+    private final int messagingVersion;
 
     private volatile boolean closed;
 
@@ -115,7 +105,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
      * A special {@link Channel} for sending non-stream streaming messages, basically anything that isn't an
      * {@link OutgoingStreamMessage} (or an {@link IncomingStreamMessage}, but a node doesn't send that, it's only received).
      */
-    private volatile Channel controlMessageChannel;
+    private volatile StreamingChannel controlChannel;
 
     // note: this really doesn't need to be a LBQ, just something that's thread safe
     private final Collection<ScheduledFuture<?>> channelKeepAlives = newBlockingQueue();
@@ -125,23 +115,15 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
     /**
      * A mapping of each {@link #fileTransferExecutor} thread to a channel that can be written to (on that thread).
      */
-    private final ConcurrentMap<Thread, Channel> threadToChannelMap = new ConcurrentHashMap<>();
-
-    /**
-     * A netty channel attribute used to indicate if a channel is currently transferring a stream. This is primarily used
-     * to indicate to the {@link KeepAliveTask} if it is safe to send a {@link KeepAliveMessage}, as sending the
-     * (application level) keep-alive in the middle of a stream would be bad news.
-     */
-    @VisibleForTesting
-    static final AttributeKey<Boolean> TRANSFERRING_FILE_ATTR = valueOf("transferringFile");
+    private final ConcurrentMap<Thread, StreamingChannel> threadToChannelMap = new ConcurrentHashMap<>();
 
-    public NettyStreamingMessageSender(StreamSession session, OutboundConnectionSettings template, StreamConnectionFactory factory, int streamingVersion, boolean isPreview)
+    public StreamingMultiplexedChannel(StreamSession session, StreamingChannel.Factory factory, InetAddressAndPort to, @Nullable StreamingChannel controlChannel, int messagingVersion)
     {
         this.session = session;
-        this.streamingVersion = streamingVersion;
-        this.template = template;
         this.factory = factory;
-        this.isPreview = isPreview;
+        this.to = to;
+        this.messagingVersion = messagingVersion;
+        this.controlChannel = controlChannel;
 
         String name = session.peer.toString().replace(':', '.');
         fileTransferExecutor = executorFactory()
@@ -149,31 +131,16 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
                 .withKeepAlive(1L, SECONDS).build();
     }
 
-    @Override
-    public void initialize()
-    {
-        StreamInitMessage message = new StreamInitMessage(getBroadcastAddressAndPort(),
-                                                          session.sessionIndex(),
-                                                          session.planId(),
-                                                          session.streamOperation(),
-                                                          session.getPendingRepair(),
-                                                          session.getPreviewKind());
-        sendMessage(message);
-    }
 
-    public boolean hasControlChannel()
+
+    public InetAddressAndPort peer()
     {
-        return controlMessageChannel != null;
+        return to;
     }
 
-    /**
-     * Used by follower to setup control message channel created by initiator
-     */
-    public void injectControlMessageChannel(Channel channel)
+    public InetSocketAddress connectedTo()
     {
-        this.controlMessageChannel = channel;
-        channel.attr(TRANSFERRING_FILE_ATTR).set(FALSE);
-        scheduleKeepAliveTask(channel);
+        return controlChannel == null ? to : controlChannel.connectedTo();
     }
 
     /**
@@ -181,7 +148,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
      */
     private void setupControlMessageChannel() throws IOException
     {
-        if (controlMessageChannel == null)
+        if (controlChannel == null)
         {
             /*
              * Inbound handlers are needed:
@@ -189,106 +156,82 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
              *  b) for streaming receiver (note: both initiator and follower can receive streaming files) to reveive files,
              *     in {@link Handler#setupStreamingPipeline}
              */
-            controlMessageChannel = createChannel(true);
-            scheduleKeepAliveTask(controlMessageChannel);
+            controlChannel = createChannel(StreamingChannel.Kind.CONTROL);
         }
     }
 
-    private void scheduleKeepAliveTask(Channel channel)
-    {
-        int keepAlivePeriod = getStreamingKeepAlivePeriod();
-        if (logger.isDebugEnabled())
-            logger.debug("{} Scheduling keep-alive task with {}s period.", createLogTag(session, channel), keepAlivePeriod);
-
-        KeepAliveTask task = new KeepAliveTask(channel, session);
-        ScheduledFuture<?> scheduledFuture = channel.eventLoop().scheduleAtFixedRate(task, 0, keepAlivePeriod, SECONDS);
-        channelKeepAlives.add(scheduledFuture);
-        task.future = scheduledFuture;
-    }
-    
-    private Channel createChannel(boolean isInboundHandlerNeeded) throws IOException
+    private StreamingChannel createChannel(StreamingChannel.Kind kind) throws IOException
     {
-        Channel channel = factory.createConnection(template, streamingVersion);
-        session.attachOutbound(channel);
+        logger.debug("Creating stream session to {} as {}", to, session.isFollower() ? "follower" : "initiator");
 
-        if (isInboundHandlerNeeded)
+        StreamingChannel channel = factory.create(to, messagingVersion, kind);
+        if (kind == StreamingChannel.Kind.CONTROL)
         {
-            ChannelPipeline pipeline = channel.pipeline();
-            pipeline.addLast("stream", new StreamingInboundHandler(template.to, streamingVersion, session));
+            executorFactory().startThread(String.format("Stream-Deserializer-%s-%s", to.toString(), channel.id()),
+                                          new StreamDeserializingTask(session, channel, messagingVersion));
+            session.attachInbound(channel);
         }
-        channel.attr(TRANSFERRING_FILE_ATTR).set(FALSE);
-        logger.debug("Creating channel id {} local {} remote {}", channel.id(), channel.localAddress(), channel.remoteAddress());
+        session.attachOutbound(channel);
+
+        logger.debug("Creating {}", channel.description());
         return channel;
     }
 
-    static String createLogTag(StreamSession session, Channel channel)
+    public Future<?> sendControlMessage(StreamMessage message)
     {
-        StringBuilder sb = new StringBuilder(64);
-        sb.append("[Stream");
-
-        if (session != null)
-                sb.append(" #").append(session.planId());
-
-        if (channel != null)
-                sb.append(" channel: ").append(channel.id());
+        try
+        {
+            setupControlMessageChannel();
+            return sendMessage(controlChannel, message);
+        }
+        catch (Exception e)
+        {
+            close();
+            session.onError(e);
+            return ImmediateFuture.failure(e);
+        }
 
-        sb.append(']');
-        return sb.toString();
     }
-
-    @Override
-    public void sendMessage(StreamMessage message)
+    public Future<?> sendMessage(StreamingChannel channel, StreamMessage message)
     {
         if (closed)
             throw new RuntimeException("stream has been closed, cannot send " + message);
 
         if (message instanceof OutgoingStreamMessage)
         {
-            if (isPreview)
+            if (session.isPreview())
                 throw new RuntimeException("Cannot send stream data messages for preview streaming sessions");
             if (logger.isDebugEnabled())
-                logger.debug("{} Sending {}", createLogTag(session, null), message);
-            fileTransferExecutor.submit(new FileStreamTask((OutgoingStreamMessage)message));
-            return;
+                logger.debug("{} Sending {}", createLogTag(session), message);
+            return fileTransferExecutor.submit(new FileStreamTask((OutgoingStreamMessage)message));
         }
 
         try
         {
-            setupControlMessageChannel();
-            sendControlMessage(controlMessageChannel, message, future -> onControlMessageComplete(future, message));
+            Future<?> promise = channel.send(outSupplier -> {
+                // we anticipate that the control messages are rather small, so allocating a ByteBuf shouldn't  blow out of memory.
+                long messageSize = serializedSize(message, messagingVersion);
+                if (messageSize > 1 << 30)
+                {
+                    throw new IllegalStateException(format("%s something is seriously wrong with the calculated stream control message's size: %d bytes, type is %s",
+                                                           createLogTag(session, controlChannel.id()), messageSize, message.type));
+                }
+                try (StreamingDataOutputPlus out = outSupplier.apply((int) messageSize))
+                {
+                    StreamMessage.serialize(message, out, messagingVersion, session);
+                }
+            });
+            promise.addListener(future -> onMessageComplete(future, message));
+            return promise;
         }
         catch (Exception e)
         {
             close();
             session.onError(e);
+            return ImmediateFuture.failure(e);
         }
     }
 
-    private void sendControlMessage(Channel channel, StreamMessage message, GenericFutureListener listener) throws IOException
-    {
-        if (logger.isDebugEnabled())
-            logger.debug("{} Sending {}", createLogTag(session, channel), message);
-
-        // we anticipate that the control messages are rather small, so allocating a ByteBuf shouldn't  blow out of memory.
-        long messageSize = serializedSize(message, streamingVersion);
-        if (messageSize > 1 << 30)
-        {
-            throw new IllegalStateException(format("%s something is seriously wrong with the calculated stream control message's size: %d bytes, type is %s",
-                                                          createLogTag(session, channel), messageSize, message.type));
-        }
-
-        // as control messages are (expected to be) small, we can simply allocate a ByteBuf here, wrap it, and send via the channel
-        ByteBuf buf = channel.alloc().directBuffer((int) messageSize, (int) messageSize);
-        ByteBuffer nioBuf = buf.nioBuffer(0, (int) messageSize);
-        @SuppressWarnings("resource")
-        DataOutputBufferFixed out = new DataOutputBufferFixed(nioBuf);
-        serialize(message, out, streamingVersion, session);
-        assert nioBuf.position() == nioBuf.limit();
-        buf.writerIndex(nioBuf.position());
-
-        writeAndFlush(channel, buf, listener);
-    }
-
     /**
      * Decides what to do after a {@link StreamMessage} is processed.
      *
@@ -297,16 +240,15 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
      * @return null if the message was processed sucessfully; else, a {@link java.util.concurrent.Future} to indicate
      * the status of aborting any remaining tasks in the session.
      */
-    java.util.concurrent.Future onControlMessageComplete(Future<?> future, StreamMessage msg)
+    Future<?> onMessageComplete(Future<?> future, StreamMessage msg)
     {
-        ChannelFuture channelFuture = (ChannelFuture)future;
         Throwable cause = future.cause();
         if (cause == null)
             return null;
 
-        Channel channel = channelFuture.channel();
+        Channel channel = future instanceof ChannelFuture ? ((ChannelFuture)future).channel() : null;
         logger.error("{} failed to send a stream message/data to peer {}: msg = {}",
-                     createLogTag(session, channel), template.to, msg, future.cause());
+                     createLogTag(session, channel), to, msg, future.cause());
 
         // StreamSession will invoke close(), but we have to mark this sender as closed so the session doesn't try
         // to send any failure messages
@@ -346,21 +288,15 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
             if (!acquirePermit(SEMAPHORE_UNAVAILABLE_LOG_INTERVAL))
                 return;
 
-            Channel channel = null;
+            StreamingChannel channel = null;
             try
             {
                 channel = getOrCreateChannel();
-                if (!channel.attr(TRANSFERRING_FILE_ATTR).compareAndSet(false, true))
-                    throw new IllegalStateException("channel's transferring state is currently set to true. refusing to start new stream");
 
                 // close the DataOutputStreamPlus as we're done with it - but don't close the channel
-                try (DataOutputStreamPlus outPlus = new AsyncStreamingOutputPlus(channel))
+                try (StreamingDataOutputPlus out = channel.acquireOut())
                 {
-                    serialize(msg, outPlus, streamingVersion, session);
-                }
-                finally
-                {
-                    channel.attr(TRANSFERRING_FILE_ATTR).set(FALSE);
+                    serialize(msg, out, messagingVersion, session);
                 }
             }
             catch (Exception e)
@@ -408,26 +344,26 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
 
                         if (logger.isInfoEnabled())
                             logger.info("{} waiting to acquire a permit to begin streaming {}. This message logs every {} minutes",
-                                        createLogTag(session, null), ofm.getName(), logInterval);
+                                        createLogTag(session), ofm.getName(), logInterval);
                     }
                 }
-                catch (InterruptedException ie)
+                catch (InterruptedException e)
                 {
-                    //ignore
+                    throw new UncheckedInterruptedException(e);
                 }
             }
         }
 
-        private Channel getOrCreateChannel()
+        private StreamingChannel getOrCreateChannel()
         {
             Thread currentThread = currentThread();
             try
             {
-                Channel channel = threadToChannelMap.get(currentThread);
+                StreamingChannel channel = threadToChannelMap.get(currentThread);
                 if (channel != null)
                     return channel;
 
-                channel = createChannel(false);
+                channel = createChannel(StreamingChannel.Kind.FILE);
                 threadToChannelMap.put(currentThread, channel);
                 return channel;
             }
@@ -437,22 +373,10 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
             }
         }
 
-        private void onError(Throwable t)
-        {
-            try
-            {
-                session.onError(t).get(DEFAULT_CLOSE_WAIT_IN_MILLIS, MILLISECONDS);
-            }
-            catch (Exception e)
-            {
-                // nop - let the Throwable param be the main failure point here, and let session handle it
-            }
-        }
-
         /**
          * For testing purposes
          */
-        void injectChannel(Channel channel)
+        void injectChannel(StreamingChannel channel)
         {
             Thread currentThread = currentThread();
             if (threadToChannelMap.get(currentThread) != null)
@@ -471,66 +395,6 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
     }
 
     /**
-     * Periodically sends the {@link KeepAliveMessage}.
-     *
-     * NOTE: this task, and the callback function {@link #keepAliveListener(Future)} is executed in the netty event loop.
-     */
-    class KeepAliveTask implements Runnable
-    {
-        private final Channel channel;
-        private final StreamSession session;
-
-        /**
-         * A reference to the scheduled task for this instance so that it may be cancelled.
-         */
-        ScheduledFuture<?> future;
-
-        KeepAliveTask(Channel channel, StreamSession session)
-        {
-            this.channel = channel;
-            this.session = session;
-        }
-
-        public void run()
-        {
-            // if the channel has been closed, cancel the scheduled task and return
-            if (!channel.isOpen() || closed)
-            {
-                future.cancel(false);
-                return;
-            }
-
-            // if the channel is currently processing streaming, skip this execution. As this task executes
-            // on the event loop, even if there is a race with a FileStreamTask which changes the channel attribute
-            // after we check it, the FileStreamTask cannot send out any bytes as this KeepAliveTask is executing
-            // on the event loop (and FileStreamTask publishes it's buffer to the channel, consumed after we're done here).
-            if (channel.attr(TRANSFERRING_FILE_ATTR).get())
-                return;
-
-            try
-            {
-                if (logger.isTraceEnabled())
-                    logger.trace("{} Sending keep-alive to {}.", createLogTag(session, channel), session.peer);
-                sendControlMessage(channel, new KeepAliveMessage(), this::keepAliveListener);
-            }
-            catch (IOException ioe)
-            {
-                future.cancel(false);
-            }
-        }
-
-        private void keepAliveListener(Future<? super Void> future)
-        {
-            if (future.isSuccess() || future.isCancelled())
-                return;
-
-            if (logger.isDebugEnabled())
-                logger.debug("{} Could not send keep-alive message (perhaps stream session is finished?).",
-                             createLogTag(session, channel), future.cause());
-        }
-    }
-
-    /**
      * For testing purposes only.
      */
     public void setClosed()
@@ -538,9 +402,9 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
         closed = true;
     }
 
-    void setControlMessageChannel(Channel channel)
+    void setControlChannel(NettyStreamingChannel channel)
     {
-        controlMessageChannel = channel;
+        controlChannel = channel;
     }
 
     int semaphoreAvailablePermits()
@@ -548,13 +412,11 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
         return fileTransferSemaphore.permits();
     }
 
-    @Override
     public boolean connected()
     {
-        return !closed && (controlMessageChannel == null || controlMessageChannel.isOpen());
+        return !closed && (controlChannel == null || controlChannel.connected());
     }
 
-    @Override
     public void close()
     {
         if (closed)
@@ -562,11 +424,12 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
 
         closed = true;
         if (logger.isDebugEnabled())
-            logger.debug("{} Closing stream connection channels on {}", createLogTag(session, null), template.to);
+            logger.debug("{} Closing stream connection channels on {}", createLogTag(session), to);
         for (ScheduledFuture<?> future : channelKeepAlives)
             future.cancel(false);
         channelKeepAlives.clear();
 
+        threadToChannelMap.values().forEach(StreamingChannel::close);
         threadToChannelMap.clear();
         fileTransferExecutor.shutdownNow();
     }
diff --git a/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java b/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java
index 83d95e0..bf35266 100644
--- a/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java
@@ -18,7 +18,7 @@
 package org.apache.cassandra.streaming.messages;
 
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamSession;
 
 public class CompleteMessage extends StreamMessage
@@ -30,7 +30,7 @@ public class CompleteMessage extends StreamMessage
             return new CompleteMessage();
         }
 
-        public void serialize(CompleteMessage message, DataOutputStreamPlus out, int version, StreamSession session) {}
+        public void serialize(CompleteMessage message, StreamingDataOutputPlus out, int version, StreamSession session) {}
 
         public long serializedSize(CompleteMessage message, int version)
         {
diff --git a/src/java/org/apache/cassandra/streaming/messages/IncomingStreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/IncomingStreamMessage.java
index e268747..3403a24 100644
--- a/src/java/org/apache/cassandra/streaming/messages/IncomingStreamMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/IncomingStreamMessage.java
@@ -20,13 +20,12 @@ package org.apache.cassandra.streaming.messages;
 import java.io.IOException;
 import java.util.Objects;
 
-import io.netty.channel.Channel;
-
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.io.util.DataInputPlus;
 
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.streaming.IncomingStream;
+import org.apache.cassandra.streaming.StreamingChannel;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamManager;
 import org.apache.cassandra.streaming.StreamReceiveException;
 import org.apache.cassandra.streaming.StreamSession;
@@ -36,7 +35,6 @@ public class IncomingStreamMessage extends StreamMessage
 {
     public static Serializer<IncomingStreamMessage> serializer = new Serializer<IncomingStreamMessage>()
     {
-        @SuppressWarnings("resource")
         public IncomingStreamMessage deserialize(DataInputPlus input, int version) throws IOException
         {
             StreamMessageHeader header = StreamMessageHeader.serializer.deserialize(input, version);
@@ -61,7 +59,7 @@ public class IncomingStreamMessage extends StreamMessage
             }
         }
 
-        public void serialize(IncomingStreamMessage message, DataOutputStreamPlus out, int version, StreamSession session)
+        public void serialize(IncomingStreamMessage message, StreamingDataOutputPlus out, int version, StreamSession session)
         {
             throw new UnsupportedOperationException("Not allowed to call serialize on an incoming stream");
         }
@@ -83,8 +81,9 @@ public class IncomingStreamMessage extends StreamMessage
     }
 
     @Override
-    public StreamSession getOrCreateSession(Channel channel)
+    public StreamSession getOrCreateAndAttachInboundSession(StreamingChannel channel, int messagingVersion)
     {
+        stream.session().attachInbound(channel);
         return stream.session();
     }
 
diff --git a/src/java/org/apache/cassandra/streaming/messages/KeepAliveMessage.java b/src/java/org/apache/cassandra/streaming/messages/KeepAliveMessage.java
index 5352b3b..7333f25 100644
--- a/src/java/org/apache/cassandra/streaming/messages/KeepAliveMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/KeepAliveMessage.java
@@ -21,7 +21,7 @@ package org.apache.cassandra.streaming.messages;
 import java.io.IOException;
 
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamSession;
 
 public class KeepAliveMessage extends StreamMessage
@@ -33,7 +33,7 @@ public class KeepAliveMessage extends StreamMessage
             return new KeepAliveMessage();
         }
 
-        public void serialize(KeepAliveMessage message, DataOutputStreamPlus out, int version, StreamSession session)
+        public void serialize(KeepAliveMessage message, StreamingDataOutputPlus out, int version, StreamSession session)
         {}
 
         public long serializedSize(KeepAliveMessage message, int version)
diff --git a/src/java/org/apache/cassandra/streaming/messages/OutgoingStreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/OutgoingStreamMessage.java
index 702e806..4128ddb 100644
--- a/src/java/org/apache/cassandra/streaming/messages/OutgoingStreamMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/OutgoingStreamMessage.java
@@ -22,9 +22,9 @@ import java.io.IOException;
 import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -37,7 +37,7 @@ public class OutgoingStreamMessage extends StreamMessage
             throw new UnsupportedOperationException("Not allowed to call deserialize on an outgoing stream");
         }
 
-        public void serialize(OutgoingStreamMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
+        public void serialize(OutgoingStreamMessage message, StreamingDataOutputPlus out, int version, StreamSession session) throws IOException
         {
             message.startTransfer();
             try
@@ -77,7 +77,7 @@ public class OutgoingStreamMessage extends StreamMessage
                                               stream.getPendingRepair());
     }
 
-    public synchronized void serialize(DataOutputStreamPlus out, int version, StreamSession session) throws IOException
+    public synchronized void serialize(StreamingDataOutputPlus out, int version, StreamSession session) throws IOException
     {
         if (completed)
         {
diff --git a/src/java/org/apache/cassandra/streaming/messages/PrepareAckMessage.java b/src/java/org/apache/cassandra/streaming/messages/PrepareAckMessage.java
index 97fdff7..479ef34 100644
--- a/src/java/org/apache/cassandra/streaming/messages/PrepareAckMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/PrepareAckMessage.java
@@ -21,14 +21,14 @@ package org.apache.cassandra.streaming.messages;
 import java.io.IOException;
 
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamSession;
 
 public class PrepareAckMessage extends StreamMessage
 {
     public static Serializer<PrepareAckMessage> serializer = new Serializer<PrepareAckMessage>()
     {
-        public void serialize(PrepareAckMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
+        public void serialize(PrepareAckMessage message, StreamingDataOutputPlus out, int version, StreamSession session) throws IOException
         {
             //nop
         }
diff --git a/src/java/org/apache/cassandra/streaming/messages/PrepareSynAckMessage.java b/src/java/org/apache/cassandra/streaming/messages/PrepareSynAckMessage.java
index 4e5e8fb..9d97de6 100644
--- a/src/java/org/apache/cassandra/streaming/messages/PrepareSynAckMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/PrepareSynAckMessage.java
@@ -23,7 +23,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.streaming.StreamSummary;
 
@@ -31,7 +31,7 @@ public class PrepareSynAckMessage extends StreamMessage
 {
     public static Serializer<PrepareSynAckMessage> serializer = new Serializer<PrepareSynAckMessage>()
     {
-        public void serialize(PrepareSynAckMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
+        public void serialize(PrepareSynAckMessage message, StreamingDataOutputPlus out, int version, StreamSession session) throws IOException
         {
             out.writeInt(message.summaries.size());
             for (StreamSummary summary : message.summaries)
diff --git a/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java b/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java
index bce47a1..1160033 100644
--- a/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java
@@ -22,8 +22,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamRequest;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.streaming.StreamSummary;
@@ -56,7 +55,7 @@ public class PrepareSynMessage extends StreamMessage
             return size;
         }
 
-        public void serialize(PrepareSynMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
+        public void serialize(PrepareSynMessage message, StreamingDataOutputPlus out, int version, StreamSession session) throws IOException
         {
             // requests
             out.writeInt(message.requests.size());
diff --git a/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java b/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
index a1dd03c..134e2cf 100644
--- a/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
@@ -20,9 +20,8 @@ package org.apache.cassandra.streaming.messages;
 import java.io.*;
 
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamSession;
 
 public class ReceivedMessage extends StreamMessage
@@ -35,7 +34,7 @@ public class ReceivedMessage extends StreamMessage
             return new ReceivedMessage(TableId.deserialize(input), input.readInt());
         }
 
-        public void serialize(ReceivedMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
+        public void serialize(ReceivedMessage message, StreamingDataOutputPlus out, int version, StreamSession session) throws IOException
         {
             message.tableId.serialize(out);
             out.writeInt(message.sequenceNumber);
diff --git a/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java b/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java
index ca10bcc..f09b643 100644
--- a/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java
@@ -18,7 +18,7 @@
 package org.apache.cassandra.streaming.messages;
 
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamSession;
 
 public class SessionFailedMessage extends StreamMessage
@@ -30,7 +30,7 @@ public class SessionFailedMessage extends StreamMessage
             return new SessionFailedMessage();
         }
 
-        public void serialize(SessionFailedMessage message, DataOutputStreamPlus out, int version, StreamSession session) {}
+        public void serialize(SessionFailedMessage message, StreamingDataOutputPlus out, int version, StreamSession session) {}
 
         public long serializedSize(SessionFailedMessage message, int version)
         {
diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
index 0d6ef47..64e3b71 100644
--- a/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
@@ -20,13 +20,12 @@ package org.apache.cassandra.streaming.messages;
 import java.io.IOException;
 import java.util.UUID;
 
-import io.netty.channel.Channel;
-
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.streaming.StreamingChannel;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamOperation;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.streaming.StreamResultFuture;
@@ -64,10 +63,12 @@ public class StreamInitMessage extends StreamMessage
     }
 
     @Override
-    public StreamSession getOrCreateSession(Channel channel)
+    public StreamSession getOrCreateAndAttachInboundSession(StreamingChannel channel, int messagingVersion)
     {
-        return StreamResultFuture.createFollower(sessionIndex, planId, streamOperation, from, channel, pendingRepair, previewKind)
+        StreamSession session = StreamResultFuture.createFollower(sessionIndex, planId, streamOperation, from, channel, messagingVersion, pendingRepair, previewKind)
                                  .getSession(from, sessionIndex);
+        session.attachInbound(channel);
+        return session;
     }
 
     @Override
@@ -81,7 +82,7 @@ public class StreamInitMessage extends StreamMessage
 
     private static class StreamInitMessageSerializer implements Serializer<StreamInitMessage>
     {
-        public void serialize(StreamInitMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
+        public void serialize(StreamInitMessage message, StreamingDataOutputPlus out, int version, StreamSession session) throws IOException
         {
             inetAddressAndPortSerializer.serialize(message.from, out, version);
             out.writeInt(message.sessionIndex);
diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
index 8c6f71b..db393a5 100644
--- a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
@@ -21,10 +21,9 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import io.netty.channel.Channel;
-
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.streaming.StreamingChannel;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
 import org.apache.cassandra.streaming.StreamSession;
 
 /**
@@ -34,7 +33,7 @@ import org.apache.cassandra.streaming.StreamSession;
  */
 public abstract class StreamMessage
 {
-    public static void serialize(StreamMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
+    public static void serialize(StreamMessage message, StreamingDataOutputPlus out, int version, StreamSession session) throws IOException
     {
         out.writeByte(message.type.id);
         message.type.outSerializer.serialize(message, out, version, session);
@@ -55,7 +54,7 @@ public abstract class StreamMessage
     public static interface Serializer<V extends StreamMessage>
     {
         V deserialize(DataInputPlus in, int version) throws IOException;
-        void serialize(V message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException;
+        void serialize(V message, StreamingDataOutputPlus out, int version, StreamSession session) throws IOException;
         long serializedSize(V message, int version) throws IOException;
     }
 
@@ -136,7 +135,7 @@ public abstract class StreamMessage
      * Get or create a {@link StreamSession} based on this stream message data: not all stream messages support this,
      * so the default implementation just throws an exception.
      */
-    public StreamSession getOrCreateSession(Channel channel)
+    public StreamSession getOrCreateAndAttachInboundSession(StreamingChannel channel, int messagingVersion)
     {
         throw new UnsupportedOperationException("Not supported by streaming messages of type: " + this.getClass());
     }
diff --git a/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java b/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
index 177f811..079c08b 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
@@ -19,14 +19,18 @@
 package org.apache.cassandra.tools;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 
-import io.netty.channel.Channel;
 import org.apache.cassandra.config.EncryptionOptions;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.OutboundConnectionSettings;
-import org.apache.cassandra.streaming.DefaultConnectionFactory;
-import org.apache.cassandra.streaming.StreamConnectionFactory;
+import org.apache.cassandra.streaming.StreamingChannel;
+import org.apache.cassandra.streaming.async.NettyStreamingConnectionFactory;
+import org.apache.cassandra.streaming.async.NettyStreamingChannel;
 
-public class BulkLoadConnectionFactory extends DefaultConnectionFactory implements StreamConnectionFactory
+import static org.apache.cassandra.locator.InetAddressAndPort.getByAddress;
+
+public class BulkLoadConnectionFactory extends NettyStreamingConnectionFactory
 {
     // TODO: what is this unused variable for?
     private final boolean outboundBindAny;
@@ -40,15 +44,15 @@ public class BulkLoadConnectionFactory extends DefaultConnectionFactory implemen
         this.outboundBindAny = outboundBindAny;
     }
 
-    public Channel createConnection(OutboundConnectionSettings template, int messagingVersion) throws IOException
+    public NettyStreamingChannel connect(InetSocketAddress to, int messagingVersion, StreamingChannel.Kind kind) throws IOException
     {
         // Connect to secure port for all peers if ServerEncryptionOptions is configured other than 'none'
         // When 'all', 'dc' and 'rack', server nodes always have SSL port open, and since thin client like sstableloader
         // does not know which node is in which dc/rack, connecting to SSL port is always the option.
-
+        OutboundConnectionSettings template = new OutboundConnectionSettings(getByAddress(to));
         if (encryptionOptions != null && encryptionOptions.internode_encryption != EncryptionOptions.ServerEncryptionOptions.InternodeEncryption.none)
             template = template.withConnectTo(template.to.withPort(secureStoragePort));
 
-        return super.createConnection(template, messagingVersion);
+        return connect(template, messagingVersion, kind);
     }
 }
diff --git a/src/java/org/apache/cassandra/tools/BulkLoader.java b/src/java/org/apache/cassandra/tools/BulkLoader.java
index 43df49c..d1a4b8b 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoader.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoader.java
@@ -35,7 +35,6 @@ import com.datastax.shaded.netty.channel.socket.SocketChannel;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.io.sstable.SSTableLoader;
-import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.security.SSLFactory;
 import org.apache.cassandra.streaming.*;
 import org.apache.cassandra.utils.FBUtilities;
@@ -129,7 +128,7 @@ public class BulkLoader
         private long peak = 0;
         private int totalFiles = 0;
 
-        private final Multimap<InetAddressAndPort, SessionInfo> sessionsByHost = HashMultimap.create();
+        private final Multimap<InetSocketAddress, SessionInfo> sessionsByHost = HashMultimap.create();
 
         public ProgressIndicator()
         {
@@ -170,7 +169,7 @@ public class BulkLoader
 
                 boolean updateTotalFiles = totalFiles == 0;
                 // recalculate progress across all sessions in all hosts and display
-                for (InetAddressAndPort peer : sessionsByHost.keySet())
+                for (InetSocketAddress peer : sessionsByHost.keySet())
                 {
                     sb.append("[").append(peer).append("]");
 
@@ -301,7 +300,7 @@ public class BulkLoader
         }
 
         @Override
-        public StreamConnectionFactory getConnectionFactory()
+        public StreamingChannel.Factory getConnectionFactory()
         {
             return new BulkLoadConnectionFactory(sslStoragePort, serverEncOptions, false);
         }
diff --git a/src/java/org/apache/cassandra/tools/nodetool/NetStats.java b/src/java/org/apache/cassandra/tools/nodetool/NetStats.java
index cc6b7b1..aacc071 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/NetStats.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/NetStats.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingServiceMBean;
 import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.SessionInfo;
@@ -54,11 +55,11 @@ public class NetStats extends NodeToolCmd
             out.printf("%s %s%n", status.streamOperation.getDescription(), status.planId.toString());
             for (SessionInfo info : status.sessions)
             {
-                out.printf("    %s", info.peer.toString(printPort));
+                out.printf("    %s", InetAddressAndPort.toString(info.peer, printPort));
                 // print private IP when it is used
                 if (!info.peer.equals(info.connecting))
                 {
-                    out.printf(" (using %s)", info.connecting.toString(printPort));
+                    out.printf(" (using %s)", InetAddressAndPort.toString(info.connecting, printPort));
                 }
                 out.printf("%n");
                 if (!info.receivingSummaries.isEmpty())
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
index 217dcd5..eca9088 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
@@ -86,7 +86,6 @@ import org.reflections.util.ConfigurationBuilder;
 
 import static org.apache.cassandra.distributed.shared.NetworkTopology.addressAndPort;
 import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
-import static org.reflections.ReflectionUtils.forNames;
 
 /**
  * AbstractCluster creates, initializes and manages Cassandra instances ({@link Instance}.
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
index 2d441f2..3f67b2f 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
@@ -122,7 +122,7 @@ import org.apache.cassandra.service.reads.trackwarnings.CoordinatorWarnings;
 import org.apache.cassandra.service.snapshot.SnapshotManager;
 import org.apache.cassandra.streaming.StreamReceiveTask;
 import org.apache.cassandra.streaming.StreamTransferTask;
-import org.apache.cassandra.streaming.async.StreamingInboundHandler;
+import org.apache.cassandra.streaming.async.NettyStreamingChannel;
 import org.apache.cassandra.tools.NodeTool;
 import org.apache.cassandra.tools.Output;
 import org.apache.cassandra.tools.SystemExitException;
@@ -180,7 +180,7 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
 
         // Enable streaming inbound handler tracking so they can be closed properly without leaking
         // the blocking IO thread.
-        StreamingInboundHandler.trackInboundHandlers();
+        NettyStreamingChannel.trackInboundHandlers();
     }
 
     @Override
@@ -760,10 +760,10 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
                                 CompactionManager.instance::forceShutdown,
                                 () -> BatchlogManager.instance.shutdownAndWait(1L, MINUTES),
                                 HintsService.instance::shutdownBlocking,
-                                StreamingInboundHandler::shutdown,
                                 () -> CompactionLogger.shutdownNowAndWait(1L, MINUTES),
                                 () -> AuthCache.shutdownAllAndWait(1L, MINUTES),
                                 () -> Sampler.shutdownNowAndWait(1L, MINUTES),
+                                NettyStreamingChannel::shutdown,
                                 () -> StreamReceiveTask.shutdownAndWait(1L, MINUTES),
                                 () -> StreamTransferTask.shutdownAndWait(1L, MINUTES),
                                 () -> SecondaryIndexManager.shutdownAndWait(1L, MINUTES),
diff --git a/test/distributed/org/apache/cassandra/distributed/test/metrics/HintsServiceMetricsTest.java b/test/distributed/org/apache/cassandra/distributed/test/metrics/HintsServiceMetricsTest.java
index cf7e9f9..730cc45 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/metrics/HintsServiceMetricsTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/metrics/HintsServiceMetricsTest.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.distributed.test.metrics;
 
 import java.util.Arrays;
 import java.util.concurrent.Callable;
-import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
diff --git a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java
index 9e84e0a..cc24274 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java
+++ b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java
@@ -55,7 +55,7 @@ import org.apache.cassandra.net.AsyncStreamingInputPlus;
 import org.apache.cassandra.net.AsyncStreamingOutputPlus;
 import org.apache.cassandra.schema.CachingParams;
 import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.async.NettyStreamingConnectionFactory;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.streaming.SessionInfo;
 import org.apache.cassandra.streaming.StreamCoordinator;
@@ -216,13 +216,13 @@ public class ZeroCopyStreamingBenchmark
 
         private StreamSession setupStreamingSessionForTest()
         {
-            StreamCoordinator streamCoordinator = new StreamCoordinator(StreamOperation.BOOTSTRAP, 1, new DefaultConnectionFactory(), false, false, null, PreviewKind.NONE);
+            StreamCoordinator streamCoordinator = new StreamCoordinator(StreamOperation.BOOTSTRAP, 1, new NettyStreamingConnectionFactory(), false, false, null, PreviewKind.NONE);
             StreamResultFuture future = StreamResultFuture.createInitiator(UUID.randomUUID(), StreamOperation.BOOTSTRAP, Collections.<StreamEventHandler>emptyList(), streamCoordinator);
 
             InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
             streamCoordinator.addSessionInfo(new SessionInfo(peer, 0, peer, Collections.emptyList(), Collections.emptyList(), StreamSession.State.INITIALIZED));
 
-            StreamSession session = streamCoordinator.getOrCreateNextSession(peer);
+            StreamSession session = streamCoordinator.getOrCreateOutboundSession(peer);
             session.init(future);
             return session;
         }
diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
index 58d26c1..bf7c148 100644
--- a/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
+++ b/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
@@ -49,7 +49,7 @@ import org.apache.cassandra.net.SharedDefaultFileRegion;
 import org.apache.cassandra.net.AsyncStreamingOutputPlus;
 import org.apache.cassandra.schema.CachingParams;
 import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.async.NettyStreamingConnectionFactory;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.streaming.SessionInfo;
 import org.apache.cassandra.streaming.StreamCoordinator;
@@ -204,13 +204,13 @@ public class CassandraEntireSSTableStreamWriterTest
 
     private StreamSession setupStreamingSessionForTest()
     {
-        StreamCoordinator streamCoordinator = new StreamCoordinator(StreamOperation.BOOTSTRAP, 1, new DefaultConnectionFactory(), false, false, null, PreviewKind.NONE);
+        StreamCoordinator streamCoordinator = new StreamCoordinator(StreamOperation.BOOTSTRAP, 1, new NettyStreamingConnectionFactory(), false, false, null, PreviewKind.NONE);
         StreamResultFuture future = StreamResultFuture.createInitiator(UUID.randomUUID(), StreamOperation.BOOTSTRAP, Collections.<StreamEventHandler>emptyList(), streamCoordinator);
 
         InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
         streamCoordinator.addSessionInfo(new SessionInfo(peer, 0, peer, Collections.emptyList(), Collections.emptyList(), StreamSession.State.INITIALIZED));
 
-        StreamSession session = streamCoordinator.getOrCreateNextSession(peer);
+        StreamSession session = streamCoordinator.getOrCreateOutboundSession(peer);
         session.init(future);
         return session;
     }
diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
index 4281278..7c59628 100644
--- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
@@ -52,14 +52,15 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.ActiveRepairService;
-import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.StreamingChannel;
+import org.apache.cassandra.streaming.async.NettyStreamingConnectionFactory;
 import org.apache.cassandra.streaming.OutgoingStream;
 import org.apache.cassandra.streaming.PreviewKind;
-import org.apache.cassandra.streaming.StreamConnectionFactory;
 import org.apache.cassandra.streaming.StreamOperation;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.utils.UUIDGen;
@@ -72,7 +73,7 @@ public class CassandraStreamManagerTest
     private static final String KEYSPACE = null;
     private String keyspace = null;
     private static final String table = "tbl";
-    private static final StreamConnectionFactory connectionFactory = new DefaultConnectionFactory();
+    private static final StreamingChannel.Factory connectionFactory = new NettyStreamingConnectionFactory();
 
     private TableMetadata tbm;
     private ColumnFamilyStore cfs;
@@ -99,6 +100,8 @@ public class CassandraStreamManagerTest
             return new StreamSession(StreamOperation.REPAIR,
                                      InetAddressAndPort.getByName("127.0.0.1"),
                                      connectionFactory,
+                                     null,
+                                     MessagingService.current_version,
                                      false,
                                      0,
                                      pendingRepair,
diff --git a/test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java b/test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
index 3cc8943..4d2242b 100644
--- a/test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
+++ b/test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
@@ -71,7 +71,7 @@ import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.ActiveRepairService;
-import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.async.NettyStreamingConnectionFactory;
 import org.apache.cassandra.streaming.OutgoingStream;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.streaming.SessionInfo;
@@ -319,13 +319,13 @@ public class EntireSSTableStreamConcurrentComponentMutationTest
 
     private StreamSession setupStreamingSessionForTest()
     {
-        StreamCoordinator streamCoordinator = new StreamCoordinator(StreamOperation.BOOTSTRAP, 1, new DefaultConnectionFactory(), false, false, null, PreviewKind.NONE);
+        StreamCoordinator streamCoordinator = new StreamCoordinator(StreamOperation.BOOTSTRAP, 1, new NettyStreamingConnectionFactory(), false, false, null, PreviewKind.NONE);
         StreamResultFuture future = StreamResultFuture.createInitiator(UUID.randomUUID(), StreamOperation.BOOTSTRAP, Collections.emptyList(), streamCoordinator);
 
         InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
         streamCoordinator.addSessionInfo(new SessionInfo(peer, 0, peer, Collections.emptyList(), Collections.emptyList(), StreamSession.State.INITIALIZED));
 
-        StreamSession session = streamCoordinator.getOrCreateNextSession(peer);
+        StreamSession session = streamCoordinator.getOrCreateOutboundSession(peer);
         session.init(future);
         return session;
     }
diff --git a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
index b18d249..d731385 100644
--- a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
+++ b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
@@ -26,13 +26,14 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.async.NettyStreamingConnectionFactory;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.streaming.StreamEvent;
 import org.apache.cassandra.streaming.StreamOperation;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.utils.FBUtilities;
 
+import static org.apache.cassandra.net.MessagingService.current_version;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
@@ -55,7 +56,7 @@ public class StreamStateStoreTest
         Range<Token> range = new Range<>(factory.fromString("0"), factory.fromString("100"));
 
         InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
-        StreamSession session = new StreamSession(StreamOperation.BOOTSTRAP, local, new DefaultConnectionFactory(), false, 0, null, PreviewKind.NONE);
+        StreamSession session = new StreamSession(StreamOperation.BOOTSTRAP, local, new NettyStreamingConnectionFactory(), null, current_version, false, 0, null, PreviewKind.NONE);
         session.addStreamRequest("keyspace1", RangesAtEndpoint.toDummyList(Collections.singleton(range)), RangesAtEndpoint.toDummyList(Collections.emptyList()), Collections.singleton("cf"));
 
         StreamStateStore store = new StreamStateStore();
@@ -76,7 +77,7 @@ public class StreamStateStoreTest
 
         // add different range within the same keyspace
         Range<Token> range2 = new Range<>(factory.fromString("100"), factory.fromString("200"));
-        session = new StreamSession(StreamOperation.BOOTSTRAP, local, new DefaultConnectionFactory(), false, 0, null, PreviewKind.NONE);
+        session = new StreamSession(StreamOperation.BOOTSTRAP, local, new NettyStreamingConnectionFactory(), null, current_version,false, 0, null, PreviewKind.NONE);
         session.addStreamRequest("keyspace1", RangesAtEndpoint.toDummyList(Collections.singleton(range2)), RangesAtEndpoint.toDummyList(Collections.emptyList()), Collections.singleton("cf"));
         session.state(StreamSession.State.COMPLETE);
         store.handleStreamEvent(new StreamEvent.SessionCompleteEvent(session));
diff --git a/test/unit/org/apache/cassandra/net/AsyncStreamingInputPlusTest.java b/test/unit/org/apache/cassandra/net/AsyncStreamingInputPlusTest.java
index bc41992..417a51c 100644
--- a/test/unit/org/apache/cassandra/net/AsyncStreamingInputPlusTest.java
+++ b/test/unit/org/apache/cassandra/net/AsyncStreamingInputPlusTest.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.net;
 import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
 import java.nio.channels.WritableByteChannel;
 import java.util.concurrent.TimeUnit;
 
@@ -184,13 +185,13 @@ public class AsyncStreamingInputPlusTest
         consumeUntilTestCycle(2, 8, 0, 16);
     }
 
-    @Test(expected = EOFException.class)
+    @Test(expected = ClosedChannelException.class)
     public void consumeUntil_SingleBuffer_Fails() throws IOException
     {
         consumeUntilTestCycle(1, 8, 0, 9);
     }
 
-    @Test(expected = EOFException.class)
+    @Test(expected = ClosedChannelException.class)
     public void consumeUntil_MultipleBuffer_Fails() throws IOException
     {
         consumeUntilTestCycle(2, 8, 0, 17);
diff --git a/test/unit/org/apache/cassandra/net/TestChannel.java b/test/unit/org/apache/cassandra/net/TestChannel.java
index 17da6fa..d76e045 100644
--- a/test/unit/org/apache/cassandra/net/TestChannel.java
+++ b/test/unit/org/apache/cassandra/net/TestChannel.java
@@ -19,28 +19,45 @@
 package org.apache.cassandra.net;
 
 import java.io.IOException;
+import java.net.SocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.WritableByteChannel;
 
+import com.google.common.net.InetAddresses;
+
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
 import io.netty.channel.ChannelOutboundBuffer;
 import io.netty.channel.FileRegion;
 import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.locator.InetAddressAndPort;
 
 public class TestChannel extends EmbeddedChannel
 {
+    public static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 0);
+
     final int inFlightLimit;
     int inFlight;
 
     ChannelOutboundBuffer flush;
     long flushBytes;
 
+    public TestChannel()
+    {
+        this(Integer.MAX_VALUE);
+    }
+
     public TestChannel(int inFlightLimit)
     {
         this.inFlightLimit = inFlightLimit;
     }
 
+    @Override
+    public SocketAddress remoteAddress()
+    {
+        return REMOTE_ADDR;
+    }
+
     // we override ByteBuf to prevent retain() from working, to avoid release() since it is not needed in our usage
     // since the lifetime must live longer, we simply copy any outbound ByteBuf here for our tests
     protected void doWrite(ChannelOutboundBuffer in)
diff --git a/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java b/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
index 68804d0..57a65cf 100644
--- a/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
+++ b/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
@@ -40,7 +40,7 @@ import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.streaming.StreamCoordinator;
-import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.async.NettyStreamingConnectionFactory;
 import org.apache.cassandra.streaming.StreamPlan;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.streaming.StreamSession;
@@ -134,14 +134,14 @@ public class LocalSyncTaskTest extends AbstractRepairTest
         TreeResponse r2 = new TreeResponse(InetAddressAndPort.getByName("127.0.0.2"), tree2);
         LocalSyncTask task = new LocalSyncTask(desc, r1.endpoint, r2.endpoint, MerkleTrees.difference(r1.trees, r2.trees),
                                                NO_PENDING_REPAIR, true, true, PreviewKind.NONE);
-        DefaultConnectionFactory.MAX_CONNECT_ATTEMPTS = 1;
+        NettyStreamingConnectionFactory.MAX_CONNECT_ATTEMPTS = 1;
         try
         {
             task.run();
         }
         finally
         {
-            DefaultConnectionFactory.MAX_CONNECT_ATTEMPTS = 3;
+            NettyStreamingConnectionFactory.MAX_CONNECT_ATTEMPTS = 3;
         }
 
         // ensure that the changed range was recorded
diff --git a/test/unit/org/apache/cassandra/streaming/EntireSSTableStreamingCorrectFilesCountTest.java b/test/unit/org/apache/cassandra/streaming/EntireSSTableStreamingCorrectFilesCountTest.java
index 9aaf85c..0612968 100644
--- a/test/unit/org/apache/cassandra/streaming/EntireSSTableStreamingCorrectFilesCountTest.java
+++ b/test/unit/org/apache/cassandra/streaming/EntireSSTableStreamingCorrectFilesCountTest.java
@@ -56,6 +56,7 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.SharedDefaultFileRegion;
 import org.apache.cassandra.schema.CompactionParams;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.streaming.async.NettyStreamingConnectionFactory;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -193,7 +194,7 @@ public class EntireSSTableStreamingCorrectFilesCountTest
     {
         StreamCoordinator streamCoordinator = new StreamCoordinator(StreamOperation.BOOTSTRAP,
                                                                     1,
-                                                                    new DefaultConnectionFactory(),
+                                                                    new NettyStreamingConnectionFactory(),
                                                                     false,
                                                                     false,
                                                                     null,
@@ -212,7 +213,7 @@ public class EntireSSTableStreamingCorrectFilesCountTest
                                                          Collections.emptyList(),
                                                          StreamSession.State.INITIALIZED));
 
-        StreamSession session = streamCoordinator.getOrCreateNextSession(peer);
+        StreamSession session = streamCoordinator.getOrCreateOutboundSession(peer);
         session.init(future);
 
         return session;
diff --git a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
index ffab5bf..f061e51 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
@@ -17,6 +17,8 @@
  */
 package org.apache.cassandra.streaming;
 
+import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -43,12 +45,18 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.OutboundConnectionSettings;
+import org.apache.cassandra.net.TestChannel;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.streaming.async.NettyStreamingChannel;
+import org.apache.cassandra.streaming.async.NettyStreamingConnectionFactory;
 import org.apache.cassandra.streaming.messages.OutgoingStreamMessage;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.Ref;
 
+import static org.apache.cassandra.net.MessagingService.current_version;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
@@ -57,6 +65,15 @@ public class StreamTransferTaskTest
     public static final String KEYSPACE1 = "StreamTransferTaskTest";
     public static final String CF_STANDARD = "Standard1";
 
+    static final StreamingChannel.Factory FACTORY = new NettyStreamingConnectionFactory()
+    {
+        @Override
+        public NettyStreamingChannel create(InetSocketAddress to, int messagingVersion, StreamingChannel.Kind kind)
+        {
+            return new NettyStreamingChannel(messagingVersion, new TestChannel(), kind);
+        }
+    };
+
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
     {
@@ -77,7 +94,7 @@ public class StreamTransferTaskTest
     public void testScheduleTimeout() throws Exception
     {
         InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
-        StreamSession session = new StreamSession(StreamOperation.BOOTSTRAP, peer, (template, messagingVersion) -> new EmbeddedChannel(), false, 0, UUID.randomUUID(), PreviewKind.ALL);
+        StreamSession session = new StreamSession(StreamOperation.BOOTSTRAP, peer, FACTORY, null, current_version, false, 0, UUID.randomUUID(), PreviewKind.ALL);
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD);
 
         // create two sstables
@@ -125,9 +142,9 @@ public class StreamTransferTaskTest
     public void testFailSessionDuringTransferShouldNotReleaseReferences() throws Exception
     {
         InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
-        StreamCoordinator streamCoordinator = new StreamCoordinator(StreamOperation.BOOTSTRAP, 1, new DefaultConnectionFactory(), false, false, null, PreviewKind.NONE);
+        StreamCoordinator streamCoordinator = new StreamCoordinator(StreamOperation.BOOTSTRAP, 1, new NettyStreamingConnectionFactory(), false, false, null, PreviewKind.NONE);
         StreamResultFuture future = StreamResultFuture.createInitiator(UUID.randomUUID(), StreamOperation.OTHER, Collections.<StreamEventHandler>emptyList(), streamCoordinator);
-        StreamSession session = new StreamSession(StreamOperation.BOOTSTRAP, peer, null, false, 0, null, PreviewKind.NONE);
+        StreamSession session = new StreamSession(StreamOperation.BOOTSTRAP, peer, FACTORY, null, current_version, false, 0, null, PreviewKind.NONE);
         session.init(future);
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD);
 
diff --git a/test/unit/org/apache/cassandra/streaming/async/StreamingInboundHandlerTest.java b/test/unit/org/apache/cassandra/streaming/async/StreamingInboundHandlerTest.java
index a43b116..aacd5d4 100644
--- a/test/unit/org/apache/cassandra/streaming/async/StreamingInboundHandlerTest.java
+++ b/test/unit/org/apache/cassandra/streaming/async/StreamingInboundHandlerTest.java
@@ -19,6 +19,7 @@
 package org.apache.cassandra.streaming.async;
 
 import java.io.IOException;
+import java.net.SocketAddress;
 import java.nio.ByteBuffer;
 import java.util.UUID;
 
@@ -39,25 +40,28 @@ import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.AsyncStreamingInputPlus;
+import org.apache.cassandra.net.TestChannel;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.StreamDeserializingTask;
 import org.apache.cassandra.streaming.StreamManager;
 import org.apache.cassandra.streaming.StreamOperation;
 import org.apache.cassandra.streaming.StreamResultFuture;
 import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamingChannel;
 import org.apache.cassandra.streaming.messages.CompleteMessage;
 import org.apache.cassandra.streaming.messages.IncomingStreamMessage;
 import org.apache.cassandra.streaming.messages.StreamInitMessage;
 import org.apache.cassandra.streaming.messages.StreamMessageHeader;
 
+import static org.apache.cassandra.net.TestChannel.REMOTE_ADDR;
+
 public class StreamingInboundHandlerTest
 {
     private static final int VERSION = MessagingService.current_version;
-    private static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 0);
 
-    private StreamingInboundHandler handler;
+    private NettyStreamingChannel streamingChannel;
     private EmbeddedChannel channel;
-    private AsyncStreamingInputPlus buffers;
     private ByteBuf buf;
 
     @BeforeClass
@@ -69,10 +73,9 @@ public class StreamingInboundHandlerTest
     @Before
     public void setup()
     {
-        handler = new StreamingInboundHandler(REMOTE_ADDR, VERSION, null);
-        channel = new EmbeddedChannel(handler);
-        buffers = new AsyncStreamingInputPlus(channel);
-        handler.setPendingBuffers(buffers);
+        channel = new TestChannel();
+        streamingChannel = new NettyStreamingChannel(VERSION, channel, StreamingChannel.Kind.CONTROL);
+        channel.pipeline().addLast("stream", streamingChannel);
     }
 
     @After
@@ -88,24 +91,10 @@ public class StreamingInboundHandlerTest
     }
 
     @Test
-    public void channelRead_Closed()
-    {
-        int size = 8;
-        buf = channel.alloc().buffer(size);
-        Assert.assertEquals(1, buf.refCnt());
-        buf.writerIndex(size);
-        handler.close();
-        channel.writeInbound(buf);
-        Assert.assertEquals(0, buffers.unsafeAvailable());
-        Assert.assertEquals(0, buf.refCnt());
-        Assert.assertFalse(channel.releaseInbound());
-    }
-
-    @Test
     public void channelRead_WrongObject()
     {
         channel.writeInbound("homer");
-        Assert.assertEquals(0, buffers.unsafeAvailable());
+        Assert.assertEquals(0, streamingChannel.in.unsafeAvailable());
         Assert.assertFalse(channel.releaseInbound());
     }
 
@@ -113,7 +102,7 @@ public class StreamingInboundHandlerTest
     public void StreamDeserializingTask_deriveSession_StreamInitMessage()
     {
         StreamInitMessage msg = new StreamInitMessage(REMOTE_ADDR, 0, UUID.randomUUID(), StreamOperation.REPAIR, UUID.randomUUID(), PreviewKind.ALL);
-        StreamingInboundHandler.StreamDeserializingTask task = handler.new StreamDeserializingTask(null, channel);
+        StreamDeserializingTask task = new StreamDeserializingTask(null, streamingChannel, streamingChannel.messagingVersion);
         StreamSession session = task.deriveSession(msg);
         Assert.assertNotNull(session);
     }
@@ -122,7 +111,7 @@ public class StreamingInboundHandlerTest
     public void StreamDeserializingTask_deriveSession_NoSession()
     {
         CompleteMessage msg = new CompleteMessage();
-        StreamingInboundHandler.StreamDeserializingTask task = handler.new StreamDeserializingTask(null, channel);
+        StreamDeserializingTask task = new StreamDeserializingTask(null, streamingChannel, streamingChannel.messagingVersion);
         task.deriveSession(msg);
     }
 
@@ -146,7 +135,7 @@ public class StreamingInboundHandlerTest
     public void StreamDeserializingTask_deserialize_ISM_HasSession()
     {
         UUID planId = UUID.randomUUID();
-        StreamResultFuture future = StreamResultFuture.createFollower(0, planId, StreamOperation.REPAIR, REMOTE_ADDR, channel, UUID.randomUUID(), PreviewKind.ALL);
+        StreamResultFuture future = StreamResultFuture.createFollower(0, planId, StreamOperation.REPAIR, REMOTE_ADDR, streamingChannel, streamingChannel.messagingVersion, UUID.randomUUID(), PreviewKind.ALL);
         StreamManager.instance.registerFollower(future);
         StreamMessageHeader header = new StreamMessageHeader(TableId.generate(), REMOTE_ADDR, planId, false,
                                                              0, 0, 0, UUID.randomUUID());
diff --git a/test/unit/org/apache/cassandra/streaming/async/NettyStreamingMessageSenderTest.java b/test/unit/org/apache/cassandra/streaming/async/StreamingMultiplexedChannelTest.java
similarity index 59%
rename from test/unit/org/apache/cassandra/streaming/async/NettyStreamingMessageSenderTest.java
rename to test/unit/org/apache/cassandra/streaming/async/StreamingMultiplexedChannelTest.java
index 76bfa76..fdd9dec 100644
--- a/test/unit/org/apache/cassandra/streaming/async/NettyStreamingMessageSenderTest.java
+++ b/test/unit/org/apache/cassandra/streaming/async/StreamingMultiplexedChannelTest.java
@@ -24,7 +24,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-import com.google.common.net.InetAddresses;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -33,23 +32,24 @@ import org.junit.Test;
 
 import io.netty.channel.ChannelPromise;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.TestChannel;
-import org.apache.cassandra.net.TestScheduledFuture;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.streaming.StreamOperation;
 import org.apache.cassandra.streaming.StreamResultFuture;
 import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamingChannel;
 import org.apache.cassandra.streaming.messages.CompleteMessage;
 
-public class NettyStreamingMessageSenderTest
-{
-    private static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 0);
+import static org.apache.cassandra.net.MessagingService.current_version;
+import static org.apache.cassandra.net.TestChannel.REMOTE_ADDR;
 
+public class StreamingMultiplexedChannelTest
+{
+    private NettyStreamingChannel streamingChannel;
     private TestChannel channel;
     private StreamSession session;
-    private NettyStreamingMessageSender sender;
-    private NettyStreamingMessageSender.FileStreamTask fileStreamTask;
+    private StreamingMultiplexedChannel sender;
+    private StreamingMultiplexedChannel.FileStreamTask fileStreamTask;
 
     @BeforeClass
     public static void before()
@@ -60,16 +60,16 @@ public class NettyStreamingMessageSenderTest
     @Before
     public void setUp()
     {
-        channel = new TestChannel(Integer.MAX_VALUE);
-        channel.attr(NettyStreamingMessageSender.TRANSFERRING_FILE_ATTR).set(Boolean.FALSE);
+        channel = new TestChannel();
+        streamingChannel = new NettyStreamingChannel(current_version, channel, StreamingChannel.Kind.CONTROL);
         UUID pendingRepair = UUID.randomUUID();
-        session = new StreamSession(StreamOperation.BOOTSTRAP, REMOTE_ADDR, (template, messagingVersion) -> null, true, 0, pendingRepair, PreviewKind.ALL);
-        StreamResultFuture future = StreamResultFuture.createFollower(0, UUID.randomUUID(), StreamOperation.REPAIR, REMOTE_ADDR, channel, pendingRepair, session.getPreviewKind());
+        session = new StreamSession(StreamOperation.BOOTSTRAP, REMOTE_ADDR, new NettyStreamingConnectionFactory(), streamingChannel, current_version, true, 0, pendingRepair, PreviewKind.ALL);
+        StreamResultFuture future = StreamResultFuture.createFollower(0, UUID.randomUUID(), StreamOperation.REPAIR, REMOTE_ADDR, streamingChannel, current_version, pendingRepair, session.getPreviewKind());
         session.init(future);
-        session.attachOutbound(channel);
+        session.attachOutbound(streamingChannel);
 
-        sender = session.getMessageSender();
-        sender.setControlMessageChannel(channel);
+        sender = session.getChannel();
+        sender.setControlChannel(streamingChannel);
     }
 
     @After
@@ -80,59 +80,6 @@ public class NettyStreamingMessageSenderTest
     }
 
     @Test
-    public void KeepAliveTask_normalSend()
-    {
-        Assert.assertTrue(channel.isOpen());
-        NettyStreamingMessageSender.KeepAliveTask task = sender.new KeepAliveTask(channel, session);
-        task.run();
-        Assert.assertTrue(channel.releaseOutbound());
-    }
-
-    @Test
-    public void KeepAliveTask_channelClosed()
-    {
-        channel.close();
-        Assert.assertFalse(channel.isOpen());
-        channel.releaseOutbound();
-        NettyStreamingMessageSender.KeepAliveTask task = sender.new KeepAliveTask(channel, session);
-        task.future = new TestScheduledFuture();
-        Assert.assertFalse(task.future.isCancelled());
-        task.run();
-        Assert.assertTrue(task.future.isCancelled());
-        Assert.assertFalse(channel.releaseOutbound());
-    }
-
-    @Test
-    public void KeepAliveTask_closed()
-    {
-        Assert.assertTrue(channel.isOpen());
-        NettyStreamingMessageSender.KeepAliveTask task = sender.new KeepAliveTask(channel, session);
-        task.future = new TestScheduledFuture();
-        Assert.assertFalse(task.future.isCancelled());
-
-        sender.setClosed();
-        Assert.assertFalse(sender.connected());
-        task.run();
-        Assert.assertTrue(task.future.isCancelled());
-        Assert.assertFalse(channel.releaseOutbound());
-    }
-
-    @Test
-    public void KeepAliveTask_CurrentlyStreaming()
-    {
-        Assert.assertTrue(channel.isOpen());
-        channel.attr(NettyStreamingMessageSender.TRANSFERRING_FILE_ATTR).set(Boolean.TRUE);
-        NettyStreamingMessageSender.KeepAliveTask task = sender.new KeepAliveTask(channel, session);
-        task.future = new TestScheduledFuture();
-        Assert.assertFalse(task.future.isCancelled());
-
-        Assert.assertTrue(sender.connected());
-        task.run();
-        Assert.assertFalse(task.future.isCancelled());
-        Assert.assertFalse(channel.releaseOutbound());
-    }
-
-    @Test
     public void FileStreamTask_acquirePermit_closed()
     {
         fileStreamTask = sender.new FileStreamTask(null);
@@ -153,9 +100,9 @@ public class NettyStreamingMessageSenderTest
     public void FileStreamTask_BadChannelAttr()
     {
         int permits = sender.semaphoreAvailablePermits();
-        channel.attr(NettyStreamingMessageSender.TRANSFERRING_FILE_ATTR).set(Boolean.TRUE);
+        channel.attr(NettyStreamingChannel.TRANSFERRING_FILE_ATTR).set(Boolean.TRUE);
         fileStreamTask = sender.new FileStreamTask(null);
-        fileStreamTask.injectChannel(channel);
+        fileStreamTask.injectChannel(streamingChannel);
         fileStreamTask.run();
         Assert.assertEquals(StreamSession.State.FAILED, session.state());
         Assert.assertTrue(channel.releaseOutbound()); // when the session fails, it will send a SessionFailed msg
@@ -167,7 +114,7 @@ public class NettyStreamingMessageSenderTest
     {
         int permits = sender.semaphoreAvailablePermits();
         fileStreamTask = sender.new FileStreamTask(new CompleteMessage());
-        fileStreamTask.injectChannel(channel);
+        fileStreamTask.injectChannel(streamingChannel);
         fileStreamTask.run();
         Assert.assertNotEquals(StreamSession.State.FAILED, session.state());
         Assert.assertTrue(channel.releaseOutbound());
@@ -181,7 +128,7 @@ public class NettyStreamingMessageSenderTest
         Assert.assertTrue(sender.connected());
         ChannelPromise promise = channel.newPromise();
         promise.setSuccess();
-        Assert.assertNull(sender.onControlMessageComplete(promise, new CompleteMessage()));
+        Assert.assertNull(sender.onMessageComplete(promise, new CompleteMessage()));
         Assert.assertTrue(channel.isOpen());
         Assert.assertTrue(sender.connected());
         Assert.assertNotEquals(StreamSession.State.FAILED, session.state());
@@ -194,7 +141,7 @@ public class NettyStreamingMessageSenderTest
         Assert.assertTrue(sender.connected());
         ChannelPromise promise = channel.newPromise();
         promise.setFailure(new RuntimeException("this is just a testing exception"));
-        Future f = sender.onControlMessageComplete(promise, new CompleteMessage());
+        Future f = sender.onMessageComplete(promise, new CompleteMessage());
 
         f.get(5, TimeUnit.SECONDS);
 

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


[cassandra] 03/06: [CASSANDRA-16926] CEP-10 Phase 1: Mockable Filesystem

Posted by be...@apache.org.
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 6a1d9de3d938d24baa2ba98b5379b0536177c786
Author: Benedict Elliott Smith <be...@apache.org>
AuthorDate: Thu Jul 29 17:20:18 2021 +0100

    [CASSANDRA-16926] CEP-10 Phase 1: Mockable Filesystem
    
    patch by Benedict; reviewed by Aleksey Yeschenko, Sam Tunnicliffe and Aleksei Zotov for CASSANDRA-16926
    
    Co-authored-by: Benedict Elliott Smith <be...@apache.org>
    Co-authored-by: Aleksey Yeschenko  <al...@apache.org>
---
 checkstyle.xml                                     |   4 +-
 .../cassandra/audit/AuditLogEntryCategory.java     |   2 +-
 .../apache/cassandra/cache/AutoSavingCache.java    |  33 +-
 .../config/CassandraRelevantProperties.java        |   2 +
 .../cassandra/config/DatabaseDescriptor.java       | 101 +--
 .../apache/cassandra/config/EncryptionOptions.java |   1 +
 .../cassandra/config/YamlConfigurationLoader.java  |   8 +-
 .../cassandra/cql3/conditions/ColumnCondition.java |   2 +-
 .../cassandra/cql3/selection/ResultSetBuilder.java |   2 +-
 .../cassandra/cql3/statements/StatementType.java   |   2 +-
 .../org/apache/cassandra/db/ColumnFamilyStore.java |  43 +-
 src/java/org/apache/cassandra/db/Directories.java  |  94 +--
 .../apache/cassandra/db/DisallowedDirectories.java |   8 +-
 .../cassandra/db/DisallowedDirectoriesMBean.java   |   2 +-
 src/java/org/apache/cassandra/db/Keyspace.java     |   7 +-
 src/java/org/apache/cassandra/db/ReadResponse.java |   1 +
 .../org/apache/cassandra/db/SSTableImporter.java   |   2 +-
 .../org/apache/cassandra/db/SystemKeyspace.java    |   5 +-
 .../db/VirtualTableSinglePartitionReadQuery.java   |   2 +-
 .../cassandra/db/WindowsFailedSnapshotTracker.java |  16 +-
 .../cassandra/db/aggregation/GroupMaker.java       |   2 +-
 .../commitlog/AbstractCommitLogSegmentManager.java |   2 +-
 .../apache/cassandra/db/commitlog/CommitLog.java   |  17 +-
 .../cassandra/db/commitlog/CommitLogArchiver.java  |  20 +-
 .../db/commitlog/CommitLogDescriptor.java          |   7 +-
 .../cassandra/db/commitlog/CommitLogReader.java    |   7 +-
 .../cassandra/db/commitlog/CommitLogReplayer.java  |   4 +-
 .../cassandra/db/commitlog/CommitLogSegment.java   |  20 +-
 .../db/commitlog/CommitLogSegmentManagerCDC.java   |   6 +-
 .../commitlog/CommitLogSegmentManagerStandard.java |   1 +
 .../apache/cassandra/db/commitlog/IntervalSet.java |   2 +-
 .../db/commitlog/MemoryMappedSegment.java          |   2 +-
 .../db/commitlog/PeriodicCommitLogService.java     |   2 +-
 .../cassandra/db/compaction/CompactionLogger.java  |   5 +-
 .../cassandra/db/compaction/CompactionManager.java |   6 +-
 .../db/compaction/CompactionStrategyManager.java   |   6 +-
 .../apache/cassandra/db/compaction/Scrubber.java   |   5 +-
 .../SizeTieredCompactionStrategyOptions.java       |   2 +-
 .../apache/cassandra/db/compaction/Upgrader.java   |   4 +-
 .../apache/cassandra/db/compaction/Verifier.java   |   8 +-
 .../compaction/writers/CompactionAwareWriter.java  |   1 +
 .../db/lifecycle/LifecycleTransaction.java         |   2 +-
 .../cassandra/db/lifecycle/LogAwareFileLister.java |   4 +-
 .../org/apache/cassandra/db/lifecycle/LogFile.java |  14 +-
 .../apache/cassandra/db/lifecycle/LogRecord.java   |  26 +-
 .../apache/cassandra/db/lifecycle/LogReplica.java  |  16 +-
 .../cassandra/db/lifecycle/LogReplicaSet.java      |   6 +-
 .../cassandra/db/lifecycle/LogTransaction.java     |   8 +-
 .../org/apache/cassandra/db/lifecycle/Tracker.java |   2 +-
 .../CassandraEntireSSTableStreamReader.java        |   2 +-
 .../db/streaming/CassandraStreamReader.java        |   1 +
 .../db/streaming/CassandraStreamWriter.java        |   2 +-
 .../cassandra/db/streaming/ComponentContext.java   |   8 +-
 .../cassandra/db/streaming/ComponentManifest.java  |   3 +-
 .../dht/tokenallocator/OfflineTokenAllocator.java  |   2 +-
 .../dht/tokenallocator/ReplicationStrategy.java    |   2 +-
 .../dht/tokenallocator/TokenAllocator.java         |   2 +-
 .../tokenallocator/TokenAllocatorDiagnostics.java  |   2 +-
 .../dht/tokenallocator/TokenAllocatorEvent.java    |   2 +-
 .../cassandra/diag/DiagnosticEventService.java     |   2 +-
 .../org/apache/cassandra/fql/FullQueryLogger.java  |   4 +-
 .../org/apache/cassandra/gms/EndpointState.java    |   1 +
 .../org/apache/cassandra/gms/FailureDetector.java  |   3 +-
 .../org/apache/cassandra/gms/GossipDigest.java     |   1 +
 .../org/apache/cassandra/gms/GossipDigestAck2.java |   1 +
 .../org/apache/cassandra/gms/GossipDigestSyn.java  |   2 +
 .../cassandra/gms/GossipShutdownVerbHandler.java   |   2 +-
 .../apache/cassandra/gms/GossiperDiagnostics.java  |   2 +-
 .../org/apache/cassandra/gms/GossiperEvent.java    |   2 +-
 .../org/apache/cassandra/gms/GossiperMBean.java    |   2 +-
 .../org/apache/cassandra/gms/HeartBeatState.java   |   2 +
 .../org/apache/cassandra/gms/TokenSerializer.java  |   4 +-
 .../org/apache/cassandra/gms/VersionedValue.java   |   1 +
 .../cassandra/hadoop/cql3/CqlBulkRecordWriter.java |   6 +-
 .../cassandra/hints/ChecksummedDataInput.java      |   2 +-
 .../cassandra/hints/CompressedHintsWriter.java     |   3 +-
 .../cassandra/hints/EncryptedHintsWriter.java      |   2 +-
 .../apache/cassandra/hints/HintDiagnostics.java    |   2 +-
 src/java/org/apache/cassandra/hints/HintEvent.java |   2 +-
 .../org/apache/cassandra/hints/HintsCatalog.java   |  10 +-
 .../apache/cassandra/hints/HintsDescriptor.java    |  14 +-
 .../cassandra/hints/HintsDispatchExecutor.java     |   3 +-
 .../apache/cassandra/hints/HintsDispatcher.java    |   4 +-
 .../org/apache/cassandra/hints/HintsReader.java    |   2 +-
 .../org/apache/cassandra/hints/HintsService.java   |   2 +-
 .../org/apache/cassandra/hints/HintsStore.java     |   6 +-
 .../org/apache/cassandra/hints/HintsWriter.java    |   4 +-
 .../org/apache/cassandra/hints/package-info.java   |   2 +-
 .../cassandra/index/internal/keys/KeysIndex.java   |   2 +-
 .../index/internal/keys/KeysSearcher.java          |   2 +-
 .../cassandra/index/sasi/SASIIndexBuilder.java     |   2 +-
 .../apache/cassandra/index/sasi/SSTableIndex.java  |   2 +-
 .../index/sasi/analyzer/DelimiterAnalyzer.java     |   2 +-
 .../sasi/analyzer/filter/StopWordFactory.java      |   6 +-
 .../cassandra/index/sasi/conf/DataTracker.java     |   4 +-
 .../cassandra/index/sasi/disk/OnDiskIndex.java     |  46 +-
 .../index/sasi/disk/OnDiskIndexBuilder.java        |  12 +-
 .../index/sasi/disk/PerSSTableIndexWriter.java     |   2 +-
 .../cassandra/index/sasi/disk/TokenTree.java       |   2 +-
 .../index/sasi/disk/TokenTreeBuilder.java          |   2 +-
 .../index/sasi/memory/KeyRangeIterator.java        |   2 +-
 .../cassandra/index/sasi/utils/CombinedTerm.java   |   2 +-
 .../cassandra/index/sasi/utils/trie/Cursor.java    |   2 +-
 .../index/sasi/utils/trie/PatriciaTrie.java        |   2 +-
 src/java/org/apache/cassandra/io/FSError.java      |  14 +-
 src/java/org/apache/cassandra/io/FSReadError.java  |  10 +-
 src/java/org/apache/cassandra/io/FSWriteError.java |  10 +-
 .../io/compress/CompressedSequentialWriter.java    |   2 +-
 .../cassandra/io/compress/CompressionMetadata.java |  23 +-
 .../io/sstable/AbstractSSTableSimpleWriter.java    |  22 +-
 .../cassandra/io/sstable/CQLSSTableWriter.java     |   4 +-
 .../io/sstable/CorruptSSTableException.java        |   3 +-
 .../apache/cassandra/io/sstable/Descriptor.java    |  37 +-
 .../io/sstable/IndexSummaryManagerMBean.java       |   2 +-
 .../apache/cassandra/io/sstable/KeyIterator.java   |   2 +-
 .../org/apache/cassandra/io/sstable/SSTable.java   |  18 +-
 .../cassandra/io/sstable/SSTableHeaderFix.java     |  14 +-
 .../io/sstable/SSTableIdentityIterator.java        |   1 +
 .../apache/cassandra/io/sstable/SSTableLoader.java |  18 +-
 .../io/sstable/SSTableSimpleUnsortedWriter.java    |   2 +-
 .../cassandra/io/sstable/SSTableSimpleWriter.java  |   2 +-
 .../cassandra/io/sstable/SnapshotDeletingTask.java |   2 +-
 .../io/sstable/format/SSTableFlushObserver.java    |   2 +-
 .../cassandra/io/sstable/format/SSTableReader.java |  13 +-
 .../io/sstable/format/SSTableReaderBuilder.java    |   9 +-
 .../io/sstable/format/big/BigTableWriter.java      |  11 +-
 .../sstable/format/big/BigTableZeroCopyWriter.java |   4 +-
 .../io/sstable/metadata/MetadataCollector.java     |   0
 .../io/sstable/metadata/MetadataSerializer.java    |  16 +-
 .../io/sstable/metadata/StatsMetadata.java         |   0
 .../cassandra/io/util/AbstractReaderFileProxy.java |   2 +-
 .../io/util/BufferedDataOutputStreamPlus.java      |  22 -
 .../org/apache/cassandra/io/util/ChannelProxy.java |   3 +-
 .../apache/cassandra/io/util/ChecksumWriter.java   |   2 +
 .../io/util/ChecksummedRandomAccessReader.java     |   3 +-
 .../io/util/ChecksummedSequentialWriter.java       |   3 +-
 .../org/apache/cassandra/io/util/ChunkReader.java  |   2 +-
 .../apache/cassandra/io/util/DataInputPlus.java    |   2 +
 .../cassandra/io/util/DataIntegrityMetadata.java   |   2 +-
 src/java/org/apache/cassandra/io/util/File.java    | 608 +++++++++++++++++
 .../cassandra/io/util/FileInputStreamPlus.java     |  95 +++
 .../cassandra/io/util/FileOutputStreamPlus.java    |  69 ++
 .../{RewindableDataInput.java => FileReader.java}  |  20 +-
 .../org/apache/cassandra/io/util/FileUtils.java    | 592 ++++-------------
 .../{RewindableDataInput.java => FileWriter.java}  |  21 +-
 .../org/apache/cassandra/io/util/PathUtils.java    | 731 +++++++++++++++++++++
 .../cassandra/io/util/RandomAccessReader.java      |   2 +-
 .../apache/cassandra/io/util/ReaderFileProxy.java  |   2 +-
 .../org/apache/cassandra/io/util/Rebufferer.java   |   2 +-
 .../cassandra/io/util/RewindableDataInput.java     |   2 +-
 .../io/util/RewindableDataInputStreamPlus.java     | 571 ----------------
 .../apache/cassandra/io/util/SequentialWriter.java |   5 +-
 .../cassandra/io/util/SimpleChunkReader.java       |   2 +-
 .../apache/cassandra/locator/CloudstackSnitch.java |   4 +-
 .../apache/cassandra/locator/RangesByEndpoint.java |   2 +-
 .../cassandra/net/AsyncChannelOutputPlus.java      |   2 +-
 .../cassandra/net/AsyncMessageOutputPlus.java      |   2 +-
 .../cassandra/net/AsyncStreamingOutputPlus.java    |   2 +-
 src/java/org/apache/cassandra/net/Verb.java        |   2 +-
 .../repair/asymmetric/HostDifferences.java         |   2 +-
 .../repair/asymmetric/PreferedNodeFilter.java      |   2 +-
 .../cassandra/schema/SchemaPullVerbHandler.java    |   2 +-
 .../apache/cassandra/schema/TableMetadataRef.java  |   2 +-
 src/java/org/apache/cassandra/schema/Views.java    |   2 +-
 .../security/FileBasedSslContextFactory.java       |   3 +-
 .../apache/cassandra/service/CassandraDaemon.java  |   2 +-
 .../cassandra/service/DefaultFSErrorHandler.java   |   6 +-
 .../service/FileSystemOwnershipCheck.java          |   6 +-
 .../apache/cassandra/service/StartupChecks.java    |  14 +-
 .../apache/cassandra/service/StorageService.java   |   9 +-
 .../service/pager/MultiPartitionPager.java         |   2 +-
 .../service/snapshot/SnapshotManager.java          |   2 +-
 .../service/snapshot/SnapshotManifest.java         |   8 +-
 .../cassandra/service/snapshot/TableSnapshot.java  |   2 +-
 .../streaming/messages/PrepareSynMessage.java      |   1 +
 .../streaming/messages/ReceivedMessage.java        |   1 +
 .../org/apache/cassandra/tools/AuditLogViewer.java |   4 +-
 .../org/apache/cassandra/tools/BulkLoader.java     |   2 +-
 src/java/org/apache/cassandra/tools/JMXTool.java   |   8 +-
 .../org/apache/cassandra/tools/LoaderOptions.java  |   4 +-
 src/java/org/apache/cassandra/tools/NodeTool.java  |  13 +-
 .../org/apache/cassandra/tools/SSTableExport.java  |   4 +-
 .../cassandra/tools/SSTableMetadataViewer.java     |   4 +-
 .../cassandra/tools/SSTableOfflineRelevel.java     |   2 +-
 .../cassandra/tools/SSTableRepairedAtSetter.java   |   6 +-
 .../cassandra/tools/StandaloneSSTableUtil.java     |   4 +-
 .../apache/cassandra/tools/StandaloneScrubber.java |   4 +-
 .../apache/cassandra/tools/StandaloneSplitter.java |   4 +-
 .../cassandra/tools/nodetool/Assassinate.java      |   2 +-
 .../cassandra/tools/nodetool/CfHistograms.java     |   2 +-
 .../apache/cassandra/tools/nodetool/CfStats.java   |   2 +-
 .../apache/cassandra/tools/nodetool/Compact.java   |   2 +-
 .../cassandra/tools/nodetool/Decommission.java     |   2 +-
 .../cassandra/tools/nodetool/DisableAuditLog.java  |   2 +-
 .../tools/nodetool/DisableAutoCompaction.java      |   2 +-
 .../cassandra/tools/nodetool/DisableBackup.java    |   2 +-
 .../cassandra/tools/nodetool/DisableBinary.java    |   2 +-
 .../tools/nodetool/DisableFullQueryLog.java        |   2 +-
 .../cassandra/tools/nodetool/DisableGossip.java    |   2 +-
 .../cassandra/tools/nodetool/DisableHandoff.java   |   2 +-
 .../tools/nodetool/DisableHintsForDC.java          |   2 +-
 .../tools/nodetool/DisableOldProtocolVersions.java |   2 +-
 .../org/apache/cassandra/tools/nodetool/Drain.java |   2 +-
 .../cassandra/tools/nodetool/EnableAuditLog.java   |   2 +-
 .../tools/nodetool/EnableAutoCompaction.java       |   2 +-
 .../cassandra/tools/nodetool/EnableBackup.java     |   2 +-
 .../cassandra/tools/nodetool/EnableBinary.java     |   2 +-
 .../tools/nodetool/EnableFullQueryLog.java         |   2 +-
 .../cassandra/tools/nodetool/EnableGossip.java     |   2 +-
 .../cassandra/tools/nodetool/EnableHandoff.java    |   2 +-
 .../tools/nodetool/EnableOldProtocolVersions.java  |   2 +-
 .../org/apache/cassandra/tools/nodetool/Flush.java |   2 +-
 .../tools/nodetool/GetSnapshotThrottle.java        |   2 +-
 .../apache/cassandra/tools/nodetool/HostStat.java  |   2 +-
 .../tools/nodetool/InvalidateCounterCache.java     |   2 +-
 .../tools/nodetool/InvalidateKeyCache.java         |   2 +-
 .../tools/nodetool/InvalidateRowCache.java         |   2 +-
 .../org/apache/cassandra/tools/nodetool/Move.java  |   2 +-
 .../cassandra/tools/nodetool/PauseHandoff.java     |   2 +-
 .../apache/cassandra/tools/nodetool/Rebuild.java   |   2 +-
 .../cassandra/tools/nodetool/RebuildIndex.java     |   2 +-
 .../cassandra/tools/nodetool/ReloadTriggers.java   |   2 +-
 .../tools/nodetool/ResetFullQueryLog.java          |   2 +-
 .../cassandra/tools/nodetool/ResetLocalSchema.java |   2 +-
 .../cassandra/tools/nodetool/ResumeHandoff.java    |   2 +-
 .../tools/nodetool/SetBatchlogReplayThrottle.java  |   2 +-
 .../cassandra/tools/nodetool/SetCacheCapacity.java |   2 +-
 .../tools/nodetool/SetCacheKeysToSave.java         |   2 +-
 .../tools/nodetool/SetCompactionThreshold.java     |   2 +-
 .../tools/nodetool/SetCompactionThroughput.java    |   2 +-
 .../nodetool/SetHintedHandoffThrottleInKB.java     |   2 +-
 .../cassandra/tools/nodetool/SetHostStat.java      |   2 +-
 .../cassandra/tools/nodetool/SetLoggingLevel.java  |   2 +-
 .../tools/nodetool/SetSnapshotThrottle.java        |   2 +-
 .../tools/nodetool/SetStreamThroughput.java        |   2 +-
 .../tools/nodetool/SetTraceProbability.java        |   2 +-
 .../org/apache/cassandra/tools/nodetool/Sjk.java   |  14 +-
 .../cassandra/tools/nodetool/TruncateHints.java    |   2 +-
 .../tools/nodetool/stats/StatsHolder.java          |   2 +-
 .../tools/nodetool/stats/StatsKeyspace.java        |   2 +-
 .../tools/nodetool/stats/StatsPrinter.java         |   2 +-
 .../cassandra/triggers/CustomClassLoader.java      |  26 +-
 .../apache/cassandra/triggers/TriggerExecutor.java |   2 +-
 .../cassandra/utils/BloomFilterSerializer.java     |   5 +-
 .../org/apache/cassandra/utils/ByteArrayUtil.java  |   2 +-
 .../cassandra/utils/DirectorySizeCalculator.java   |   3 +-
 .../org/apache/cassandra/utils/ExecutorUtils.java  |   2 +-
 .../org/apache/cassandra/utils/FBUtilities.java    |  12 +-
 src/java/org/apache/cassandra/utils/HeapUtils.java |  16 +-
 .../cassandra/utils/IndexedSearchIterator.java     |   2 +-
 .../cassandra/utils/JVMStabilityInspector.java     |   3 +-
 .../org/apache/cassandra/utils/NativeLibrary.java  |   6 +-
 .../apache/cassandra/utils/ResourceWatcher.java    |   2 +-
 src/java/org/apache/cassandra/utils/SyncUtil.java  |   9 +-
 .../org/apache/cassandra/utils/Throwables.java     |   4 +-
 .../org/apache/cassandra/utils/binlog/BinLog.java  |  14 +-
 .../utils/btree/LeafBTreeSearchIterator.java       |   2 +-
 .../apache/cassandra/utils/btree/NodeCursor.java   |   2 +-
 .../apache/cassandra/utils/obs/OffHeapBitSet.java  |   4 +-
 .../utils/streamhist/HistogramDataConsumer.java    |   0
 .../StreamingTombstoneHistogramBuilder.java        |   0
 .../utils/streamhist/TombstoneHistogram.java       |   0
 .../cassandra/net/GlobalInboundSettings.java       |   2 +-
 .../apache/cassandra/transport/DriverBurnTest.java |   2 +-
 .../cassandra/transport/SimpleClientBurnTest.java  |   2 +-
 .../distributed/impl/AbstractCluster.java          |   8 +-
 .../cassandra/distributed/impl/FileLogAction.java  |   4 +-
 .../cassandra/distributed/impl/Instance.java       |  12 +-
 .../cassandra/distributed/impl/InstanceConfig.java |   4 +-
 .../cassandra/distributed/shared/Byteman.java      |  14 +-
 .../cassandra/distributed/shared/ClusterUtils.java |   2 +-
 .../distributed/shared/ShutdownException.java      |   2 +-
 .../cassandra/distributed/test/FrozenUDTTest.java  |   2 +-
 .../distributed/test/LargeColumnTest.java          |   2 +-
 .../distributed/test/MessageFiltersTest.java       |   2 +-
 .../distributed/test/MessageForwardingTest.java    |   2 +-
 .../distributed/test/NativeProtocolTest.java       |   2 +-
 .../test/NetstatsRepairStreamingTest.java          |   2 +-
 .../test/ReadRepairEmptyRangeTombstonesTest.java   |   2 +-
 .../distributed/test/ReadRepairTester.java         |   2 +-
 .../test/ReplicaFilteringProtectionTest.java       |   2 +-
 .../distributed/test/ResourceLeakTest.java         |   4 +-
 .../distributed/test/ShortReadProtectionTest.java  |   2 +-
 .../distributed/test/ring/BootstrapTest.java       |   2 +-
 .../ring/CommunicationDuringDecommissionTest.java  |   2 +-
 .../distributed/test/ring/PendingWritesTest.java   |   2 +-
 .../upgrade/MixedModeReadRepairWriteTest.java      |   2 +-
 .../org/apache/cassandra/cql3/CorruptionTest.java  |   6 +-
 .../db/commitlog/CommitLogStressTest.java          |  16 +-
 ...AbstractReplicationAwareTokenAllocatorTest.java |   2 +-
 .../cassandra/hints/HintsWriteThenReadTest.java    |   4 +-
 .../io/compress/CompressorPerformance.java         |   4 +-
 .../io/sstable/CQLSSTableWriterLongTest.java       |   8 +-
 .../cassandra/streaming/LongStreamingTest.java     |  12 +-
 .../microbench/BloomFilterSerializerBench.java     |  16 +-
 .../cassandra/test/microbench/CompactionBench.java |   6 +-
 .../test/microbench/DirectorySizerBench.java       |   6 +-
 .../test/microbench/OutputStreamBench.java         |   2 +-
 .../StreamingTombstoneHistogramBuilderBench.java   |   0
 .../cassandra/AbstractSerializationsTester.java    |  23 +-
 .../CassandraBriefJUnitResultFormatter.java        |   2 +-
 .../CassandraIsolatedJunit4ClassRunner.java        |   2 +-
 .../CassandraXMLJUnitResultFormatter.java          |   2 +-
 test/unit/org/apache/cassandra/SchemaLoader.java   |   7 +-
 .../unit/org/apache/cassandra/ServerTestUtils.java |   5 +-
 test/unit/org/apache/cassandra/Util.java           |   8 +-
 .../apache/cassandra/audit/AuditLogFilterTest.java |   2 +-
 .../cassandra/audit/AuditLoggerAuthTest.java       |   2 +-
 .../apache/cassandra/audit/BinAuditLoggerTest.java |   1 +
 .../org/apache/cassandra/auth/AuthTestUtils.java   |   4 +-
 .../cassandra/auth/PasswordAuthenticatorTest.java  |   2 +-
 .../config/CassandraRelevantPropertiesTest.java    |   2 +-
 .../config/DatabaseDescriptorRefTest.java          |   4 +
 .../cassandra/config/EncryptionOptionsTest.java    |   2 +-
 test/unit/org/apache/cassandra/cql3/CQLTester.java |   4 +-
 .../entities/SecondaryIndexOnStaticColumnTest.java |   2 +-
 .../validation/miscellaneous/OverflowTest.java     |   2 +-
 .../validation/operations/CompactStorageTest.java  |   2 +-
 .../operations/DropRecreateAndRestoreTest.java     |   2 +-
 .../cql3/validation/operations/TTLTest.java        |  16 +-
 .../apache/cassandra/db/ColumnFamilyStoreTest.java |   6 +-
 .../org/apache/cassandra/db/DirectoriesTest.java   | 120 ++--
 .../cassandra/db/DiskBoundaryManagerTest.java      |   2 +-
 test/unit/org/apache/cassandra/db/ImportTest.java  |  31 +-
 .../unit/org/apache/cassandra/db/MmapFileTest.java |  20 +-
 .../db/MutationExceededMaxSizeExceptionTest.java   |   2 +-
 .../org/apache/cassandra/db/ReadMessageTest.java   |   1 +
 .../db/RecoveryManagerMissingHeaderTest.java       |   6 +-
 .../org/apache/cassandra/db/RowIndexEntryTest.java |   2 +-
 .../apache/cassandra/db/SchemaCQLHelperTest.java   |   4 +-
 test/unit/org/apache/cassandra/db/ScrubTest.java   |  10 +-
 .../cassandra/db/SerializationHeaderTest.java      |   5 +-
 test/unit/org/apache/cassandra/db/VerifyTest.java  |   4 +-
 .../cassandra/db/commitlog/CDCTestReplayer.java    |   4 +-
 .../db/commitlog/CommitLogChainedMarkersTest.java  |   2 +-
 .../db/commitlog/CommitLogReaderTest.java          |   4 +-
 .../commitlog/CommitLogSegmentManagerCDCTest.java  |  32 +-
 .../cassandra/db/commitlog/CommitLogTest.java      |  32 +-
 .../db/commitlog/CommitLogTestReplayer.java        |   4 +-
 .../db/commitlog/CommitLogUpgradeTest.java         |   6 +-
 .../db/commitlog/CommitLogUpgradeTestMaker.java    |  13 +-
 .../db/commitlog/CommitlogShutdownTest.java        |   4 +-
 .../cassandra/db/commitlog/SegmentReaderTest.java  |  11 +-
 .../db/compaction/AntiCompactionBytemanTest.java   |   2 +-
 .../db/compaction/AntiCompactionTest.java          |   2 +-
 .../compaction/CompactionStrategyManagerTest.java  |   4 +-
 .../cassandra/db/compaction/CompactionsTest.java   |   4 +-
 .../cassandra/db/lifecycle/LogTransactionTest.java | 112 ++--
 .../db/lifecycle/RealTransactionsTest.java         |  14 +-
 .../cassandra/db/marshal/TimestampTypeTest.java    |   2 +-
 .../cassandra/db/rows/UnfilteredRowsGenerator.java |   2 +-
 .../apache/cassandra/dht/LengthPartitioner.java    |   2 +-
 .../apache/cassandra/fql/FullQueryLoggerTest.java  |  30 +-
 .../apache/cassandra/gms/SerializationsTest.java   |   5 +-
 .../org/apache/cassandra/hints/AlteredHints.java   |   4 +-
 .../cassandra/hints/ChecksummedDataInputTest.java  |   2 +-
 .../apache/cassandra/hints/HintWriteTTLTest.java   |   4 +-
 .../apache/cassandra/hints/HintsCatalogTest.java   |   8 +-
 .../cassandra/hints/HintsDescriptorTest.java       |  12 +-
 .../apache/cassandra/hints/HintsReaderTest.java    |  14 +-
 .../org/apache/cassandra/hints/HintsStoreTest.java |   4 +-
 .../apache/cassandra/index/CustomIndexTest.java    |   2 +-
 .../apache/cassandra/index/sasi/SASIIndexTest.java |   3 +-
 .../cassandra/index/sasi/disk/OnDiskIndexTest.java |   2 +-
 .../index/sasi/disk/PerSSTableIndexWriterTest.java |   2 +-
 .../cassandra/index/sasi/disk/TokenTreeTest.java   |   2 +-
 .../index/sasi/utils/LongIteratorTest.java         |   2 +-
 .../index/sasi/utils/MappedBufferTest.java         |   9 +-
 .../index/sasi/utils/RangeUnionIteratorTest.java   |   2 +-
 .../compress/CompressedRandomAccessReaderTest.java |  40 +-
 .../compress/CompressedSequentialWriterTest.java   |  32 +-
 .../cassandra/io/compress/CompressorTest.java      |   3 +-
 .../cassandra/io/sstable/BigTableWriterTest.java   |   2 +-
 .../io/sstable/CQLSSTableWriterClientTest.java     |  18 +-
 .../cassandra/io/sstable/CQLSSTableWriterTest.java |  19 +-
 .../cassandra/io/sstable/DescriptorTest.java       |  18 +-
 .../cassandra/io/sstable/LegacySSTableTest.java    |  23 +-
 .../io/sstable/SSTableCorruptionDetectionTest.java |   2 +-
 .../cassandra/io/sstable/SSTableHeaderFixTest.java |   8 +-
 .../cassandra/io/sstable/SSTableLoaderTest.java    |  16 +-
 .../cassandra/io/sstable/SSTableReaderTest.java    |  10 +-
 .../cassandra/io/sstable/SSTableRewriterTest.java  |  33 +-
 .../apache/cassandra/io/sstable/SSTableUtils.java  |  12 +-
 .../cassandra/io/sstable/SSTableWriterTest.java    |  30 +-
 .../io/sstable/SSTableWriterTestBase.java          |   8 +-
 .../sstable/format/SSTableFlushObserverTest.java   |   8 +-
 .../format/big/BigTableZeroCopyWriterTest.java     |   2 +-
 .../sstable/metadata/MetadataSerializerTest.java   |  14 +-
 .../io/util/BufferedRandomAccessFileTest.java      |  17 +-
 .../io/util/ChecksummedRandomAccessReaderTest.java |   3 +-
 .../io/util/ChecksummedSequentialWriterTest.java   |   3 +-
 .../apache/cassandra/io/util/DataOutputTest.java   |  25 +-
 .../org/apache/cassandra/io/util/FileTest.java     | 326 +++++++++
 .../apache/cassandra/io/util/FileUtilsTest.java    |  11 +-
 .../cassandra/io/util/MmappedRegionsTest.java      |   7 +-
 .../cassandra/io/util/NIODataInputStreamTest.java  |   3 +-
 .../cassandra/io/util/RandomAccessReaderTest.java  |  19 +-
 .../io/util/RewindableDataInputStreamPlusTest.java | 539 ---------------
 .../cassandra/io/util/SequentialWriterTest.java    |  13 +-
 .../cassandra/locator/PendingRangesTest.java       |   2 +-
 .../metrics/HintedHandOffMetricsTest.java          |   2 +-
 .../cassandra/metrics/LatencyMetricsTest.java      |   2 +-
 .../net/AsyncStreamingOutputPlusTest.java          |   6 +-
 .../apache/cassandra/net/ChunkedInputPlusTest.java |   2 +-
 .../net/ManyToOneConcurrentLinkedQueueTest.java    |   2 +-
 .../apache/cassandra/net/MockMessagingService.java |   2 +-
 .../org/apache/cassandra/net/MockMessagingSpy.java |   2 +-
 .../cassandra/net/PrunableArrayQueueTest.java      |   2 +-
 .../unit/org/apache/cassandra/net/SocketUtils.java |   2 +-
 .../cassandra/schema/MigrationManagerTest.java     |   4 +-
 .../org/apache/cassandra/schema/MockSchema.java    |  14 +-
 .../cassandra/security/EncryptionUtilsTest.java    |   6 +-
 .../apache/cassandra/security/SSLFactoryTest.java  |  16 +-
 .../service/SSTablesGlobalTrackerTest.java         |   3 +-
 .../cassandra/service/SerializationsTest.java      |  10 +-
 .../cassandra/service/StartupChecksTest.java       |  20 +-
 .../service/StorageServiceServerTest.java          |  20 +-
 .../service/reads/SpeculativeRetryParseTest.java   |   2 +-
 .../service/reads/repair/TestableReadRepair.java   |   2 +-
 .../service/snapshot/SnapshotManagerTest.java      |  19 +-
 .../service/snapshot/SnapshotManifestTest.java     |  26 +-
 .../service/snapshot/TableSnapshotTest.java        |  30 +-
 .../compression/CompressedInputStreamTest.java     |   7 +-
 .../apache/cassandra/tools/AuditLogViewerTest.java |   5 +-
 .../cassandra/tools/CompactionStressTest.java      |   4 +-
 .../org/apache/cassandra/tools/JMXToolTest.java    |   2 +-
 .../apache/cassandra/tools/LoaderOptionsTest.java  |  10 +-
 .../apache/cassandra/tools/OfflineToolUtils.java   |  12 +-
 .../tools/SSTableRepairedAtSetterTest.java         |   4 +-
 .../tools/StandaloneSplitterWithCQLTesterTest.java |  27 +-
 .../tools/StandaloneUpgraderOnSStablesTest.java    |   6 +-
 .../tools/nodetool/ClearSnapshotTest.java          |   2 +-
 .../nodetool/InvalidateCredentialsCacheTest.java   |   4 +-
 .../InvalidateJmxPermissionsCacheTest.java         |   4 +-
 .../InvalidateNetworkPermissionsCacheTest.java     |   4 +-
 .../nodetool/InvalidatePermissionsCacheTest.java   |   4 +-
 .../tools/nodetool/InvalidateRolesCacheTest.java   |   4 +-
 .../tools/nodetool/formatter/TableBuilderTest.java |   2 +-
 .../cassandra/transport/CQLUserAuditTest.java      |   2 +-
 .../cassandra/utils/AbstractIteratorTest.java      |   2 +-
 .../apache/cassandra/utils/BloomFilterTest.java    |  15 +-
 .../org/apache/cassandra/utils/GeneratorsTest.java |   2 +-
 .../org/apache/cassandra/utils/KeyGenerator.java   |  12 +-
 .../apache/cassandra/utils/NativeLibraryTest.java  |   4 +-
 .../apache/cassandra/utils/SerializationsTest.java |  12 +-
 .../apache/cassandra/utils/binlog/BinLogTest.java  |   6 +-
 .../utils/binlog/DeletingArchiverTest.java         |  14 +-
 .../utils/binlog/ExternalArchiverTest.java         |  54 +-
 .../cassandra/utils/concurrent/RefCountedTest.java |   2 +-
 .../StreamingTombstoneHistogramBuilderTest.java    |   0
 .../apache/cassandra/fqltool/ResultComparator.java |   2 +-
 .../io/sstable/StressCQLSSTableWriter.java         |   4 +-
 .../cassandra/stress/generate/values/Bytes.java    |   2 +-
 .../stress/operations/userdefined/CASQuery.java    |   2 +-
 .../org/apache/cassandra/stress/report/Timer.java  |   2 +-
 .../apache/cassandra/stress/settings/Command.java  |   2 +-
 .../cassandra/stress/settings/SettingsGraph.java   |   2 +-
 .../stress/util/MultiResultLoggerTest.java         |   2 +-
 458 files changed, 3528 insertions(+), 3146 deletions(-)

diff --git a/checkstyle.xml b/checkstyle.xml
index c745c2e..383e2b3 100644
--- a/checkstyle.xml
+++ b/checkstyle.xml
@@ -44,10 +44,10 @@
     </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"/>
+      <property name="illegalClasses" value="java.io.File,java.io.FileInputStream,java.io.FileOutputStream,java.io.FileReader,java.io.FileWriter,java.io.RandomAccessFile,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,java.util.concurrent.CompletableFuture"/>
     </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))"/>
+      <property name="classes" value="java.io.File,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>
 
diff --git a/src/java/org/apache/cassandra/audit/AuditLogEntryCategory.java b/src/java/org/apache/cassandra/audit/AuditLogEntryCategory.java
index 616658c..9db4ce0 100644
--- a/src/java/org/apache/cassandra/audit/AuditLogEntryCategory.java
+++ b/src/java/org/apache/cassandra/audit/AuditLogEntryCategory.java
@@ -24,4 +24,4 @@ package org.apache.cassandra.audit;
 public enum AuditLogEntryCategory
 {
     QUERY, DML, DDL, DCL, OTHER, AUTH, ERROR, PREPARE
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/cache/AutoSavingCache.java b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
index a9a09fe..03cbde2 100644
--- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java
+++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
@@ -18,15 +18,16 @@
 package org.apache.cassandra.cache;
 
 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.nio.file.NoSuchFileException;
 import java.util.*;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.io.util.File;
 import org.cliffc.high_scale_lib.NonBlockingHashSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -61,7 +62,7 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
     public interface IStreamFactory
     {
         InputStream getInputStream(File dataPath, File crcPath) throws IOException;
-        OutputStream getOutputStream(File dataPath, File crcPath) throws FileNotFoundException;
+        OutputStream getOutputStream(File dataPath, File crcPath);
     }
 
     private static final Logger logger = LoggerFactory.getLogger(AutoSavingCache.class);
@@ -253,12 +254,12 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
             catch (CorruptFileException e)
             {
                 JVMStabilityInspector.inspectThrowable(e);
-                logger.warn(String.format("Non-fatal checksum error reading saved cache %s", dataPath.getAbsolutePath()), e);
+                logger.warn(String.format("Non-fatal checksum error reading saved cache %s", dataPath.absolutePath()), e);
             }
             catch (Throwable t)
             {
                 JVMStabilityInspector.inspectThrowable(t);
-                logger.info(String.format("Harmless error reading saved cache %s", dataPath.getAbsolutePath()), t);
+                logger.info(String.format("Harmless error reading saved cache %s", dataPath.absolutePath()), t);
             }
             finally
             {
@@ -371,7 +372,7 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
                         break;
                 }
             }
-            catch (FileNotFoundException e)
+            catch (FileNotFoundException | NoSuchFileException e)
             {
                 throw new RuntimeException(e);
             }
@@ -383,13 +384,13 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
             File cacheFile = getCacheDataPath(CURRENT_VERSION);
             File crcFile = getCacheCrcPath(CURRENT_VERSION);
 
-            cacheFile.delete(); // ignore error if it didn't exist
-            crcFile.delete();
+            cacheFile.tryDelete(); // ignore error if it didn't exist
+            crcFile.tryDelete();
 
-            if (!cacheFilePaths.left.renameTo(cacheFile))
+            if (!cacheFilePaths.left.tryMove(cacheFile))
                 logger.error("Unable to rename {} to {}", cacheFilePaths.left, cacheFile);
 
-            if (!cacheFilePaths.right.renameTo(crcFile))
+            if (!cacheFilePaths.right.tryMove(crcFile))
                 logger.error("Unable to rename {} to {}", cacheFilePaths.right, crcFile);
 
             logger.info("Saved {} ({} items) in {} ms", cacheType, keysWritten, TimeUnit.NANOSECONDS.toMillis(nanoTime() - start));
@@ -399,15 +400,15 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
         {
             File dataPath = getCacheDataPath(CURRENT_VERSION);
             File crcPath = getCacheCrcPath(CURRENT_VERSION);
-            return Pair.create(FileUtils.createTempFile(dataPath.getName(), null, dataPath.getParentFile()),
-                               FileUtils.createTempFile(crcPath.getName(), null, crcPath.getParentFile()));
+            return Pair.create(FileUtils.createTempFile(dataPath.name(), null, dataPath.parent()),
+                               FileUtils.createTempFile(crcPath.name(), null, crcPath.parent()));
         }
 
         private void deleteOldCacheFiles()
         {
             File savedCachesDir = new File(DatabaseDescriptor.getSavedCachesLocation());
             assert savedCachesDir.exists() && savedCachesDir.isDirectory();
-            File[] files = savedCachesDir.listFiles();
+            File[] files = savedCachesDir.tryList();
             if (files != null)
             {
                 String cacheNameFormat = String.format("%s-%s.db", cacheType.toString(), CURRENT_VERSION);
@@ -416,11 +417,11 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
                     if (!file.isFile())
                         continue; // someone's been messing with our directory.  naughty!
 
-                    if (file.getName().endsWith(cacheNameFormat)
-                     || file.getName().endsWith(cacheType.toString()))
+                    if (file.name().endsWith(cacheNameFormat)
+                     || file.name().endsWith(cacheType.toString()))
                     {
-                        if (!file.delete())
-                            logger.warn("Failed to delete {}", file.getAbsolutePath());
+                        if (!file.tryDelete())
+                            logger.warn("Failed to delete {}", file.absolutePath());
                     }
                 }
             }
diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
index dc1671d..3eb5df1 100644
--- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
+++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
@@ -170,6 +170,8 @@ public enum CassandraRelevantProperties
      */
     LOG_DIR_AUDIT("cassandra.logdir.audit"),
 
+    CONSISTENT_DIRECTORY_LISTINGS("cassandra.consistent_directory_listings", "false"),
+
     //cassandra properties (without the "cassandra." prefix)
 
     /**
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 32a333b..878c9ec 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -17,13 +17,9 @@
  */
 package org.apache.cassandra.config;
 
-import java.io.File;
 import java.io.IOException;
 import java.net.*;
 import java.nio.file.FileStore;
-import java.nio.file.NoSuchFileException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
@@ -36,6 +32,7 @@ import com.google.common.primitives.Ints;
 import com.google.common.primitives.Longs;
 import com.google.common.util.concurrent.RateLimiter;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -59,6 +56,7 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.DiskOptimizationStrategy;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PathUtils;
 import org.apache.cassandra.io.util.SpinningDiskOptimizationStrategy;
 import org.apache.cassandra.io.util.SsdDiskOptimizationStrategy;
 import org.apache.cassandra.locator.DynamicEndpointSnitch;
@@ -549,23 +547,13 @@ public class DatabaseDescriptor
         if (conf.commitlog_total_space_in_mb == null)
         {
             final int preferredSizeInMB = 8192;
-            try
-            {
-                // use 1/4 of available space.  See discussion on #10013 and #10199
-                final long totalSpaceInBytes = guessFileStore(conf.commitlog_directory).getTotalSpace();
-                conf.commitlog_total_space_in_mb = calculateDefaultSpaceInMB("commitlog",
-                                                                             conf.commitlog_directory,
-                                                                             "commitlog_total_space_in_mb",
-                                                                             preferredSizeInMB,
-                                                                             totalSpaceInBytes, 1, 4);
-
-            }
-            catch (IOException e)
-            {
-                logger.debug("Error checking disk space", e);
-                throw new ConfigurationException(String.format("Unable to check disk space available to '%s'. Perhaps the Cassandra user does not have the necessary permissions",
-                                                               conf.commitlog_directory), e);
-            }
+            // use 1/4 of available space.  See discussion on #10013 and #10199
+            final long totalSpaceInBytes = tryGetSpace(conf.commitlog_directory, FileStore::getTotalSpace);
+            conf.commitlog_total_space_in_mb = calculateDefaultSpaceInMB("commitlog",
+                                                                         conf.commitlog_directory,
+                                                                         "commitlog_total_space_in_mb",
+                                                                         preferredSizeInMB,
+                                                                         totalSpaceInBytes, 1, 4);
         }
 
         if (conf.cdc_enabled)
@@ -582,22 +570,13 @@ public class DatabaseDescriptor
             if (conf.cdc_total_space_in_mb == 0)
             {
                 final int preferredSizeInMB = 4096;
-                try
-                {
-                    // use 1/8th of available space.  See discussion on #10013 and #10199 on the CL, taking half that for CDC
-                    final long totalSpaceInBytes = guessFileStore(conf.cdc_raw_directory).getTotalSpace();
-                    conf.cdc_total_space_in_mb = calculateDefaultSpaceInMB("cdc",
-                                                                           conf.cdc_raw_directory,
-                                                                           "cdc_total_space_in_mb",
-                                                                           preferredSizeInMB,
-                                                                           totalSpaceInBytes, 1, 8);
-                }
-                catch (IOException e)
-                {
-                    logger.debug("Error checking disk space", e);
-                    throw new ConfigurationException(String.format("Unable to check disk space available to '%s'. Perhaps the Cassandra user does not have the necessary permissions",
-                                                                   conf.cdc_raw_directory), e);
-                }
+                // use 1/8th of available space.  See discussion on #10013 and #10199 on the CL, taking half that for CDC
+                final long totalSpaceInBytes = tryGetSpace(conf.cdc_raw_directory, FileStore::getTotalSpace);
+                conf.cdc_total_space_in_mb = calculateDefaultSpaceInMB("cdc",
+                                                                       conf.cdc_raw_directory,
+                                                                       "cdc_total_space_in_mb",
+                                                                       preferredSizeInMB,
+                                                                       totalSpaceInBytes, 1, 8);
             }
 
             logger.info("cdc_enabled is true. Starting casssandra node with Change-Data-Capture enabled.");
@@ -609,7 +588,7 @@ public class DatabaseDescriptor
         }
         if (conf.data_file_directories == null || conf.data_file_directories.length == 0)
         {
-            conf.data_file_directories = new String[]{ storagedir("data_file_directories") + File.separator + "data" };
+            conf.data_file_directories = new String[]{ storagedir("data_file_directories") + File.pathSeparator() + "data" };
         }
 
         long dataFreeBytes = 0;
@@ -627,7 +606,7 @@ public class DatabaseDescriptor
             if (datadir.equals(conf.saved_caches_directory))
                 throw new ConfigurationException("saved_caches_directory must not be the same as any data_file_directories", false);
 
-            dataFreeBytes = saturatedSum(dataFreeBytes, getUnallocatedSpace(datadir));
+            dataFreeBytes = saturatedSum(dataFreeBytes, tryGetSpace(datadir, FileStore::getUnallocatedSpace));
         }
         if (dataFreeBytes < 64 * ONE_GB) // 64 GB
             logger.warn("Only {} free across all data volumes. Consider adding more capacity to your cluster or removing obsolete snapshots",
@@ -642,7 +621,7 @@ public class DatabaseDescriptor
             if (conf.local_system_data_file_directory.equals(conf.hints_directory))
                 throw new ConfigurationException("local_system_data_file_directory must not be the same as the hints_directory", false);
 
-            long freeBytes = getUnallocatedSpace(conf.local_system_data_file_directory);
+            long freeBytes = tryGetSpace(conf.local_system_data_file_directory, FileStore::getUnallocatedSpace);
 
             if (freeBytes < ONE_GB)
                 logger.warn("Only {} free in the system data volume. Consider adding more capacity or removing obsolete snapshots",
@@ -875,7 +854,7 @@ public class DatabaseDescriptor
 
     private static String storagedirFor(String type)
     {
-        return storagedir(type + "_directory") + File.separator + type;
+        return storagedir(type + "_directory") + File.pathSeparator() + type;
     }
 
     private static String storagedir(String errMsgType)
@@ -1195,45 +1174,9 @@ public class DatabaseDescriptor
         return sum < 0 ? Long.MAX_VALUE : sum;
     }
 
-    private static FileStore guessFileStore(String dir) throws IOException
+    private static long tryGetSpace(String dir, PathUtils.IOToLongFunction<FileStore> getSpace)
     {
-        Path path = Paths.get(dir);
-        while (true)
-        {
-            try
-            {
-                return FileUtils.getFileStore(path);
-            }
-            catch (IOException e)
-            {
-                if (e instanceof NoSuchFileException)
-                {
-                    path = path.getParent();
-                    if (path == null)
-                    {
-                        throw new ConfigurationException("Unable to find filesystem for '" + dir + "'.");
-                    }
-                }
-                else
-                {
-                    throw e;
-                }
-            }
-        }
-    }
-
-    private static long getUnallocatedSpace(String directory)
-    {
-        try
-        {
-            return guessFileStore(directory).getUnallocatedSpace();
-        }
-        catch (IOException e)
-        {
-            logger.debug("Error checking disk space", e);
-            throw new ConfigurationException(String.format("Unable to check disk space available to %s. Perhaps the Cassandra user does not have the necessary permissions",
-                                                           directory), e);
-        }
+        return PathUtils.tryGetSpace(new File(dir).toPath(), getSpace, e -> { throw new ConfigurationException("Unable check disk space in '" + dir + "'. Perhaps the Cassandra user does not have the necessary permissions"); });
     }
 
     public static IEndpointSnitch createEndpointSnitch(boolean dynamic, String snitchClassName) throws ConfigurationException
diff --git a/src/java/org/apache/cassandra/config/EncryptionOptions.java b/src/java/org/apache/cassandra/config/EncryptionOptions.java
index 2ac9d56..616d344 100644
--- a/src/java/org/apache/cassandra/config/EncryptionOptions.java
+++ b/src/java/org/apache/cassandra/config/EncryptionOptions.java
@@ -27,6 +27,7 @@ import java.util.Set;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java b/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java
index a774a81..b6969f0 100644
--- a/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java
+++ b/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.config;
 
 import java.io.ByteArrayInputStream;
-import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.lang.annotation.Annotation;
@@ -35,6 +34,7 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.io.ByteStreams;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.commons.lang3.SystemUtils;
 
 import org.slf4j.Logger;
@@ -79,14 +79,14 @@ public class YamlConfigurationLoader implements ConfigurationLoader
             url = loader.getResource(configUrl);
             if (url == null)
             {
-                String required = "file:" + File.separator + File.separator;
+                String required = "file:" + File.pathSeparator() + File.pathSeparator();
                 if (!configUrl.startsWith(required))
                     throw new ConfigurationException(String.format(
                         "Expecting URI in variable: [cassandra.config]. Found[%s]. Please prefix the file with [%s%s] for local " +
                         "files and [%s<server>%s] for remote files. If you are executing this from an external tool, it needs " +
                         "to set Config.setClientMode(true) to avoid loading configuration.",
-                        configUrl, required, File.separator, required, File.separator));
-                throw new ConfigurationException("Cannot locate " + configUrl + ".  If this is a local file, please confirm you've provided " + required + File.separator + " as a URI prefix.");
+                        configUrl, required, File.pathSeparator(), required, File.pathSeparator()));
+                throw new ConfigurationException("Cannot locate " + configUrl + ".  If this is a local file, please confirm you've provided " + required + File.pathSeparator() + " as a URI prefix.");
             }
         }
 
diff --git a/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java b/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
index 93ed6ae..d27fd3f 100644
--- a/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
+++ b/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
@@ -861,4 +861,4 @@ public abstract class ColumnCondition
             return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/cql3/selection/ResultSetBuilder.java b/src/java/org/apache/cassandra/cql3/selection/ResultSetBuilder.java
index df82d52..9a02a33 100644
--- a/src/java/org/apache/cassandra/cql3/selection/ResultSetBuilder.java
+++ b/src/java/org/apache/cassandra/cql3/selection/ResultSetBuilder.java
@@ -202,4 +202,4 @@ public final class ResultSetBuilder
         addSize(row);
         return row;
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/StatementType.java b/src/java/org/apache/cassandra/cql3/statements/StatementType.java
index d399931..f9c0d3d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/StatementType.java
+++ b/src/java/org/apache/cassandra/cql3/statements/StatementType.java
@@ -135,4 +135,4 @@ public enum StatementType
     {
         return false;
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index e46e8e0..8a60bff 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.lang.reflect.Constructor;
@@ -47,6 +46,8 @@ import com.google.common.util.concurrent.*;
 
 import org.apache.cassandra.utils.concurrent.AsyncPromise;
 import org.apache.cassandra.utils.concurrent.CountDownLatch;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileOutputStreamPlus;
 import org.apache.cassandra.utils.concurrent.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -516,7 +517,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         List<String> dataPaths = new ArrayList<>();
         for (File dataPath : directories.getCFDirectories())
         {
-            dataPaths.add(dataPath.getCanonicalPath());
+            dataPaths.add(dataPath.canonicalPath());
         }
 
         return dataPaths;
@@ -671,7 +672,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 for (File tmpFile : desc.getTemporaryFiles())
                 {
                     logger.info("Removing unfinished temporary file {}", tmpFile);
-                    tmpFile.delete();
+                    tmpFile.tryDelete();
                 }
             }
 
@@ -697,10 +698,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         if (dir.exists())
         {
             assert dir.isDirectory();
-            for (File file : Objects.requireNonNull(dir.listFiles()))
-                if (tmpCacheFilePattern.matcher(file.getName()).matches())
-                    if (!file.delete())
-                        logger.warn("could not delete {}", file.getAbsolutePath());
+            for (File file : dir.tryList())
+                if (tmpCacheFilePattern.matcher(file.name()).matches())
+                    if (!file.tryDelete())
+                        logger.warn("could not delete {}", file.absolutePath());
         }
 
         // also clean out any index leftovers.
@@ -1881,7 +1882,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 for (SSTableReader ssTable : currentView.sstables)
                 {
                     File snapshotDirectory = Directories.getSnapshotDirectory(ssTable.descriptor, snapshotName);
-                    ssTable.createLinks(snapshotDirectory.getPath(), rateLimiter); // hard links
+                    ssTable.createLinks(snapshotDirectory.path(), rateLimiter); // hard links
 
                     if (logger.isTraceEnabled())
                         logger.trace("Snapshot for {} keyspace data file {} created in {}", keyspace, ssTable.getFilename(), snapshotDirectory);
@@ -1895,7 +1896,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     protected TableSnapshot createSnapshot(String tag, boolean ephemeral, Duration ttl, Set<SSTableReader> sstables, Instant creationTime) {
         Set<File> snapshotDirs = sstables.stream()
-                                         .map(s -> Directories.getSnapshotDirectory(s.descriptor, tag).getAbsoluteFile())
+                                         .map(s -> Directories.getSnapshotDirectory(s.descriptor, tag).toAbsolute())
                                          .filter(dir -> !Directories.isSecondaryIndexFolder(dir)) // Remove secondary index subdirectory
                                          .collect(Collectors.toCollection(HashSet::new));
 
@@ -1903,14 +1904,14 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         SnapshotManifest manifest = new SnapshotManifest(mapToDataFilenames(sstables), ttl, creationTime);
         File manifestFile = getDirectories().getSnapshotManifestFile(tag);
         writeSnapshotManifest(manifest, manifestFile);
-        snapshotDirs.add(manifestFile.getParentFile().getAbsoluteFile()); // manifest may create empty snapshot dir
+        snapshotDirs.add(manifestFile.parent().toAbsolute()); // manifest may create empty snapshot dir
 
         // Write snapshot schema
         if (!SchemaConstants.isLocalSystemKeyspace(metadata.keyspace) && !SchemaConstants.isReplicatedSystemKeyspace(metadata.keyspace))
         {
             File schemaFile = getDirectories().getSnapshotSchemaFile(tag);
             writeSnapshotSchema(schemaFile);
-            snapshotDirs.add(schemaFile.getParentFile().getAbsoluteFile()); // schema may create empty snapshot dir
+            snapshotDirs.add(schemaFile.parent().toAbsolute()); // schema may create empty snapshot dir
         }
 
         // Maybe create ephemeral marker
@@ -1918,7 +1919,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         {
             File ephemeralSnapshotMarker = getDirectories().getNewEphemeralSnapshotMarkerFile(tag);
             createEphemeralSnapshotMarkerFile(tag, ephemeralSnapshotMarker);
-            snapshotDirs.add(ephemeralSnapshotMarker.getParentFile().getAbsoluteFile()); // marker may create empty snapshot dir
+            snapshotDirs.add(ephemeralSnapshotMarker.parent().toAbsolute()); // marker may create empty snapshot dir
         }
 
         TableSnapshot snapshot = new TableSnapshot(metadata.keyspace, metadata.name, tag, manifest.createdAt,
@@ -1932,9 +1933,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         try
         {
-            if (!manifestFile.getParentFile().exists())
-                manifestFile.getParentFile().mkdirs();
-
+            manifestFile.parent().tryCreateDirectories();
             manifest.serializeToJsonFile(manifestFile);
             return manifest;
         }
@@ -1953,10 +1952,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         try
         {
-            if (!schemaFile.getParentFile().exists())
-                schemaFile.getParentFile().mkdirs();
+            if (!schemaFile.parent().exists())
+                schemaFile.parent().tryCreateDirectories();
 
-            try (PrintStream out = new PrintStream(schemaFile))
+            try (PrintStream out = new PrintStream(new FileOutputStreamPlus(schemaFile)))
             {
                 SchemaCQLHelper.reCreateStatementsForSchemaCql(metadata(),
                                                                keyspace.getMetadata().types)
@@ -1973,19 +1972,19 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         try
         {
-            if (!ephemeralSnapshotMarker.getParentFile().exists())
-                ephemeralSnapshotMarker.getParentFile().mkdirs();
+            if (!ephemeralSnapshotMarker.parent().exists())
+                ephemeralSnapshotMarker.parent().tryCreateDirectories();
 
             Files.createFile(ephemeralSnapshotMarker.toPath());
             if (logger.isTraceEnabled())
-                logger.trace("Created ephemeral snapshot marker file on {}.", ephemeralSnapshotMarker.getAbsolutePath());
+                logger.trace("Created ephemeral snapshot marker file on {}.", ephemeralSnapshotMarker.absolutePath());
         }
         catch (IOException e)
         {
             logger.warn(String.format("Could not create marker file %s for ephemeral snapshot %s. " +
                                       "In case there is a failure in the operation that created " +
                                       "this snapshot, you may need to clean it manually afterwards.",
-                                      ephemeralSnapshotMarker.getAbsolutePath(), snapshot), e);
+                                      ephemeralSnapshotMarker.absolutePath(), snapshot), e);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 311cfcb..8b48c5c 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.*;
-import java.nio.file.*;
 import java.time.Instant;
 import java.util.*;
 import java.util.concurrent.ThreadLocalRandom;
@@ -26,11 +24,23 @@ import java.util.function.BiPredicate;
 import java.util.stream.Collectors;
 
 import com.google.common.annotations.VisibleForTesting;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.file.FileStore;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.*;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.BiPredicate;
+
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.RateLimiter;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -44,6 +54,7 @@ import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.io.util.PathUtils;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.snapshot.SnapshotManifest;
 import org.apache.cassandra.service.snapshot.TableSnapshot;
@@ -149,25 +160,25 @@ public class Directories
             switch (action)
             {
                 case X:
-                    privilege = file.canExecute();
+                    privilege = file.isExecutable();
                     break;
                 case W:
-                    privilege = file.canWrite();
+                    privilege = file.isWritable();
                     break;
                 case XW:
-                    privilege = file.canExecute() && file.canWrite();
+                    privilege = file.isExecutable() && file.isWritable();
                     break;
                 case R:
-                    privilege = file.canRead();
+                    privilege = file.isReadable();
                     break;
                 case XR:
-                    privilege = file.canExecute() && file.canRead();
+                    privilege = file.isExecutable() && file.isReadable();
                     break;
                 case RW:
-                    privilege = file.canRead() && file.canWrite();
+                    privilege = file.isReadable() && file.isWritable();
                     break;
                 case XRW:
-                    privilege = file.canExecute() && file.canRead() && file.canWrite();
+                    privilege = file.isExecutable() && file.isReadable() && file.isWritable();
                     break;
             }
             return privilege;
@@ -213,7 +224,7 @@ public class Directories
             // check if old SSTable directory exists
             File dataPath = new File(paths[i].location, oldSSTableRelativePath);
             dataPaths[i] = dataPath;
-            canonicalPathsBuilder.put(Paths.get(FileUtils.getCanonicalPath(dataPath)), paths[i]);
+            canonicalPathsBuilder.put(dataPath.toCanonical().toPath(), paths[i]);
         }
         boolean olderDirectoryExists = Iterables.any(Arrays.asList(dataPaths), File::exists);
         if (!olderDirectoryExists)
@@ -225,7 +236,7 @@ public class Directories
             {
                 File dataPath = new File(paths[i].location, newSSTableRelativePath);
                 dataPaths[i] = dataPath;
-                canonicalPathsBuilder.put(Paths.get(FileUtils.getCanonicalPath(dataPath)), paths[i]);
+                canonicalPathsBuilder.put(dataPath.toCanonical().toPath(), paths[i]);
             }
         }
         // if index, then move to its own directory
@@ -236,7 +247,7 @@ public class Directories
             {
                 File dataPath = new File(dataPaths[i], indexNameWithDot);
                 dataPaths[i] = dataPath;
-                canonicalPathsBuilder.put(Paths.get(FileUtils.getCanonicalPath(dataPath)), paths[i]);
+                canonicalPathsBuilder.put(dataPath.toCanonical().toPath(), paths[i]);
             }
         }
 
@@ -259,22 +270,16 @@ public class Directories
         {
             for (File dataPath : dataPaths)
             {
-                File[] indexFiles = dataPath.getParentFile().listFiles(new FileFilter()
-                {
-                    @Override
-                    public boolean accept(File file)
-                    {
-                        if (file.isDirectory())
-                            return false;
-
-                        Descriptor desc = SSTable.tryDescriptorFromFilename(file);
-                        return desc != null && desc.ksname.equals(metadata.keyspace) && desc.cfname.equals(metadata.name);
+                File[] indexFiles = dataPath.parent().tryList(file -> {
+                    if (file.isDirectory())
+                        return false;
 
-                    }
+                    Descriptor desc = SSTable.tryDescriptorFromFilename(file);
+                    return desc != null && desc.ksname.equals(metadata.keyspace) && desc.cfname.equals(metadata.name);
                 });
                 for (File indexFile : indexFiles)
                 {
-                    File destFile = new File(dataPath, indexFile.getName());
+                    File destFile = new File(dataPath, indexFile.name());
                     logger.trace("Moving index file {} to {}", indexFile, destFile);
                     FileUtils.renameWithConfirm(indexFile, destFile);
                 }
@@ -295,8 +300,8 @@ public class Directories
             for (File dir : dataPaths)
             {
                 // Note that we must compare absolute paths (not canonical) here since keyspace directories might be symlinks
-                Path dirPath = Paths.get(dir.getAbsolutePath());
-                Path locationPath = Paths.get(dataDirectory.location.getAbsolutePath());
+                Path dirPath = dir.toAbsolute().toPath();
+                Path locationPath = dataDirectory.location.toAbsolute().toPath();
                 if (dirPath.startsWith(locationPath))
                     return dir;
             }
@@ -539,7 +544,7 @@ public class Directories
     {
         if (isSecondaryIndexFolder(location))
         {
-            return getOrCreate(location.getParentFile(), SNAPSHOT_SUBDIR, snapshotName, location.getName());
+            return getOrCreate(location.parent(), SNAPSHOT_SUBDIR, snapshotName, location.name());
         }
         else
         {
@@ -584,7 +589,7 @@ public class Directories
     {
         if (isSecondaryIndexFolder(location))
         {
-            return getOrCreate(location.getParentFile(), BACKUPS_SUBDIR, location.getName());
+            return getOrCreate(location.parent(), BACKUPS_SUBDIR, location.name());
         }
         else
         {
@@ -626,9 +631,14 @@ public class Directories
             this.location = location;
         }
 
+        public DataDirectory(Path location)
+        {
+            this.location = new File(location);
+        }
+
         public long getAvailableSpace()
         {
-            long availableSpace = FileUtils.getUsableSpace(location) - DatabaseDescriptor.getMinFreeSpacePerDriveInBytes();
+            long availableSpace = PathUtils.tryGetSpace(location.toPath(), FileStore::getUsableSpace) - DatabaseDescriptor.getMinFreeSpacePerDriveInBytes();
             return availableSpace > 0 ? availableSpace : 0;
         }
 
@@ -1006,7 +1016,7 @@ public class Directories
         for (File snapshot : listAllSnapshots())
         {
             if (getEphemeralSnapshotMarkerFile(snapshot).exists())
-                ephemeralSnapshots.add(snapshot.getName());
+                ephemeralSnapshots.add(snapshot.name());
         }
         return ephemeralSnapshots;
     }
@@ -1017,11 +1027,11 @@ public class Directories
         for (final File dir : dataPaths)
         {
             File snapshotDir = isSecondaryIndexFolder(dir)
-                               ? new File(dir.getParent(), SNAPSHOT_SUBDIR)
+                               ? new File(dir.parentPath(), SNAPSHOT_SUBDIR)
                                : new File(dir, SNAPSHOT_SUBDIR);
             if (snapshotDir.exists() && snapshotDir.isDirectory())
             {
-                final File[] snapshotDirs  = snapshotDir.listFiles();
+                final File[] snapshotDirs  = snapshotDir.tryList();
                 if (snapshotDirs != null)
                 {
                     for (final File snapshot : snapshotDirs)
@@ -1043,17 +1053,17 @@ public class Directories
         for (final File dir : dataPaths)
         {
             File snapshotDir = isSecondaryIndexFolder(dir)
-                               ? new File(dir.getParent(), SNAPSHOT_SUBDIR)
+                               ? new File(dir.parentPath(), SNAPSHOT_SUBDIR)
                                : new File(dir, SNAPSHOT_SUBDIR);
             if (snapshotDir.exists() && snapshotDir.isDirectory())
             {
-                final File[] snapshotDirs  = snapshotDir.listFiles();
+                final File[] snapshotDirs  = snapshotDir.tryList();
                 if (snapshotDirs != null)
                 {
                     for (final File snapshot : snapshotDirs)
                     {
                         if (snapshot.isDirectory()) {
-                            snapshotDirsByTag.computeIfAbsent(snapshot.getName(), k -> new LinkedHashSet<>()).add(snapshot.getAbsoluteFile());
+                            snapshotDirsByTag.computeIfAbsent(snapshot.name(), k -> new LinkedHashSet<>()).add(snapshot.toAbsolute());
                         }
                     }
                 }
@@ -1069,7 +1079,7 @@ public class Directories
             File snapshotDir;
             if (isSecondaryIndexFolder(dir))
             {
-                snapshotDir = new File(dir.getParentFile(), join(SNAPSHOT_SUBDIR, snapshotName, dir.getName()));
+                snapshotDir = new File(dir.parent(), join(SNAPSHOT_SUBDIR, snapshotName, dir.name()));
             }
             else
             {
@@ -1120,7 +1130,7 @@ public class Directories
         for (File dir : dataPaths)
         {
             File snapshotDir = isSecondaryIndexFolder(dir)
-                               ? new File(dir.getParent(), SNAPSHOT_SUBDIR)
+                               ? new File(dir.parentPath(), SNAPSHOT_SUBDIR)
                                : new File(dir, SNAPSHOT_SUBDIR);
             result += getTrueAllocatedSizeIn(snapshotDir);
         }
@@ -1165,7 +1175,7 @@ public class Directories
         for (DataDirectory dataDirectory : dataDirectories.getAllDirectories())
         {
             File ksDir = new File(dataDirectory.location, ksName);
-            File[] cfDirs = ksDir.listFiles();
+            File[] cfDirs = ksDir.tryList();
             if (cfDirs == null)
                 continue;
             for (File cfDir : cfDirs)
@@ -1179,7 +1189,7 @@ public class Directories
 
     public static boolean isSecondaryIndexFolder(File dir)
     {
-        return dir.getName().startsWith(SECONDARY_INDEX_NAME_SEPARATOR);
+        return dir.name().startsWith(SECONDARY_INDEX_NAME_SEPARATOR);
     }
 
     public List<File> getCFDirectories()
@@ -1201,7 +1211,7 @@ public class Directories
             if (!dir.isDirectory())
                 throw new AssertionError(String.format("Invalid directory path %s: path exists but is not a directory", dir));
         }
-        else if (!dir.mkdirs() && !(dir.exists() && dir.isDirectory()))
+        else if (!dir.tryCreateDirectories() && !(dir.exists() && dir.isDirectory()))
         {
             throw new FSWriteError(new IOException("Unable to create directory " + dir), dir);
         }
@@ -1210,7 +1220,7 @@ public class Directories
 
     private static String join(String... s)
     {
-        return StringUtils.join(s, File.separator);
+        return StringUtils.join(s, File.pathSeparator());
     }
 
     private class SSTableSizeSummer extends DirectorySizeCalculator
@@ -1225,7 +1235,7 @@ public class Directories
         @Override
         public boolean isAcceptable(Path path)
         {
-            File file = path.toFile();
+            File file = new File(path);
             Descriptor desc = SSTable.tryDescriptorFromFilename(file);
             return desc != null
                 && desc.ksname.equals(metadata.keyspace)
diff --git a/src/java/org/apache/cassandra/db/DisallowedDirectories.java b/src/java/org/apache/cassandra/db/DisallowedDirectories.java
index f030253..32ada8b 100644
--- a/src/java/org/apache/cassandra/db/DisallowedDirectories.java
+++ b/src/java/org/apache/cassandra/db/DisallowedDirectories.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.db;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
 import java.util.Collections;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArraySet;
@@ -28,6 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.utils.MBeanWrapper;
 
 public class DisallowedDirectories implements DisallowedDirectoriesMBean
@@ -145,11 +145,11 @@ public class DisallowedDirectories implements DisallowedDirectoriesMBean
             return file;
 
         if (file.isFile())
-            return file.getParentFile();
+            return file.parent();
 
         // the file with path cannot be read - try determining the directory manually.
-        if (file.getPath().endsWith(".db"))
-            return file.getParentFile();
+        if (file.path().endsWith(".db"))
+            return file.parent();
 
         // We may not be able to determine if it's a file or a directory if
         // we were called because we couldn't create the file/directory.
diff --git a/src/java/org/apache/cassandra/db/DisallowedDirectoriesMBean.java b/src/java/org/apache/cassandra/db/DisallowedDirectoriesMBean.java
index 64f15e5..7c6d530 100644
--- a/src/java/org/apache/cassandra/db/DisallowedDirectoriesMBean.java
+++ b/src/java/org/apache/cassandra/db/DisallowedDirectoriesMBean.java
@@ -17,9 +17,9 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.File;
 import java.util.Set;
 
+import org.apache.cassandra.io.util.File;
 public interface DisallowedDirectoriesMBean
 {
     public Set<File> getUnreadableDirectories();
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 2a99690..795230e 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.File;
 import java.io.IOException;
 import java.time.Instant;
 import java.util.ArrayList;
@@ -37,9 +36,6 @@ 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;
 
@@ -55,6 +51,7 @@ import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.index.Index;
 import org.apache.cassandra.index.SecondaryIndexManager;
 import org.apache.cassandra.index.transactions.UpdateTransaction;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.metrics.KeyspaceMetrics;
@@ -72,6 +69,8 @@ import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.concurrent.AsyncPromise;
+import org.apache.cassandra.utils.concurrent.Future;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 import org.apache.cassandra.utils.concurrent.Promise;
diff --git a/src/java/org/apache/cassandra/db/ReadResponse.java b/src/java/org/apache/cassandra/db/ReadResponse.java
index 52e6fd5..568b1a1 100644
--- a/src/java/org/apache/cassandra/db/ReadResponse.java
+++ b/src/java/org/apache/cassandra/db/ReadResponse.java
@@ -30,6 +30,7 @@ import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
diff --git a/src/java/org/apache/cassandra/db/SSTableImporter.java b/src/java/org/apache/cassandra/db/SSTableImporter.java
index 989ff12..5949559 100644
--- a/src/java/org/apache/cassandra/db/SSTableImporter.java
+++ b/src/java/org/apache/cassandra/db/SSTableImporter.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.db;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -29,6 +28,7 @@ import java.util.Set;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 279c99e..a7cbe40 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.File;
 import java.io.IOError;
 import java.io.IOException;
 import java.net.InetAddress;
@@ -37,6 +36,8 @@ 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.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -1505,7 +1506,7 @@ public final class SystemKeyspace
             // from there, but it informs us that this isn't a completely new node.
             for (File dataDirectory : Directories.getKSChildDirectories(SchemaConstants.SYSTEM_KEYSPACE_NAME))
             {
-                if (dataDirectory.getName().equals("Versions") && dataDirectory.listFiles().length > 0)
+                if (dataDirectory.name().equals("Versions") && dataDirectory.tryList().length > 0)
                 {
                     logger.trace("Found unreadable versions info in pre 1.2 system.Versions table");
                     return UNREADABLE_VERSION.toString();
diff --git a/src/java/org/apache/cassandra/db/VirtualTableSinglePartitionReadQuery.java b/src/java/org/apache/cassandra/db/VirtualTableSinglePartitionReadQuery.java
index f96f652..32346e8 100644
--- a/src/java/org/apache/cassandra/db/VirtualTableSinglePartitionReadQuery.java
+++ b/src/java/org/apache/cassandra/db/VirtualTableSinglePartitionReadQuery.java
@@ -187,4 +187,4 @@ public class VirtualTableSinglePartitionReadQuery extends VirtualTableReadQuery
                                                     .collect(Collectors.toList()));
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java b/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java
index 134fb11..2cb5eb1 100644
--- a/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java
+++ b/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java
@@ -19,21 +19,25 @@
 package org.apache.cassandra.db;
 
 import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
+
 import java.io.IOException;
+import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileOutputStreamPlus;
+import org.apache.cassandra.io.util.FileReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.util.FileUtils;
 
+import static org.apache.cassandra.io.util.File.WriteMode.APPEND;
+
 
 public class WindowsFailedSnapshotTracker
 {
@@ -44,7 +48,7 @@ public class WindowsFailedSnapshotTracker
     // Need to handle null for unit tests
     public static final String TODELETEFILE = System.getenv("CASSANDRA_HOME") == null
                  ? ".toDelete"
-                 : System.getenv("CASSANDRA_HOME") + File.separator + ".toDelete";
+                 : System.getenv("CASSANDRA_HOME") + File.pathSeparator() + ".toDelete";
 
     public static void deleteOldSnapshots()
     {
@@ -81,7 +85,7 @@ public class WindowsFailedSnapshotTracker
                 }
 
                 // Only delete the old .toDelete file if we succeed in deleting all our known bad snapshots.
-                Files.delete(Paths.get(TODELETEFILE));
+                new File(TODELETEFILE).delete();
             }
             catch (IOException e)
             {
@@ -91,7 +95,7 @@ public class WindowsFailedSnapshotTracker
 
         try
         {
-            _failedSnapshotFile = new PrintWriter(new FileWriter(TODELETEFILE, true));
+            _failedSnapshotFile = new PrintWriter(new OutputStreamWriter(new FileOutputStreamPlus(TODELETEFILE, APPEND)));
         }
         catch (IOException e)
         {
diff --git a/src/java/org/apache/cassandra/db/aggregation/GroupMaker.java b/src/java/org/apache/cassandra/db/aggregation/GroupMaker.java
index 90cdab2..fc62f21 100644
--- a/src/java/org/apache/cassandra/db/aggregation/GroupMaker.java
+++ b/src/java/org/apache/cassandra/db/aggregation/GroupMaker.java
@@ -137,4 +137,4 @@ public abstract class GroupMaker
             return isNew;
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
old mode 100755
new mode 100644
index 2f5983b..0c438d5
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.commitlog;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -39,6 +38,7 @@ 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.File;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.SimpleCachedBufferPool;
 import org.apache.cassandra.schema.Schema;
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index 5245c0a..5f386d1 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -17,13 +17,16 @@
  */
 package org.apache.cassandra.db.commitlog;
 
-import java.io.*;
+
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
+import java.util.function.BiPredicate;
 import java.util.function.Function;
 import java.util.zip.CRC32;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.cassandra.io.util.File;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -147,22 +150,22 @@ public class CommitLog implements CommitLogMBean
      */
     public int recoverSegmentsOnDisk() throws IOException
     {
-        FilenameFilter unmanagedFilesFilter = (dir, name) -> CommitLogDescriptor.isValid(name) && CommitLogSegment.shouldReplay(name);
+        BiPredicate<File, String> unmanagedFilesFilter = (dir, name) -> CommitLogDescriptor.isValid(name) && CommitLogSegment.shouldReplay(name);
 
         // submit all files for this segment manager for archiving prior to recovery - CASSANDRA-6904
         // The files may have already been archived by normal CommitLog operation. This may cause errors in this
         // archiving pass, which we should not treat as serious.
-        for (File file : new File(segmentManager.storageDirectory).listFiles(unmanagedFilesFilter))
+        for (File file : new File(segmentManager.storageDirectory).tryList(unmanagedFilesFilter))
         {
-            archiver.maybeArchive(file.getPath(), file.getName());
-            archiver.maybeWaitForArchiving(file.getName());
+            archiver.maybeArchive(file.path(), file.name());
+            archiver.maybeWaitForArchiving(file.name());
         }
 
         assert archiver.archivePending.isEmpty() : "Not all commit log archive tasks were completed before restore";
         archiver.maybeRestoreArchive();
 
         // List the files again as archiver may have added segments.
-        File[] files = new File(segmentManager.storageDirectory).listFiles(unmanagedFilesFilter);
+        File[] files = new File(segmentManager.storageDirectory).tryList(unmanagedFilesFilter);
         int replayed = 0;
         if (files.length == 0)
         {
@@ -475,7 +478,7 @@ public class CommitLog implements CommitLogMBean
         segmentManager.stopUnsafe(deleteSegments);
         CommitLogSegment.resetReplayLimit();
         if (DatabaseDescriptor.isCDCEnabled() && deleteSegments)
-            for (File f : new File(DatabaseDescriptor.getCDCLogLocation()).listFiles())
+            for (File f : new File(DatabaseDescriptor.getCDCLogLocation()).tryList())
                 FileUtils.deleteWithConfirm(f);
     }
 
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
index f94b269..5ee79ff 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
@@ -20,7 +20,6 @@
  */
 package org.apache.cassandra.db.commitlog;
 
-import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.text.ParseException;
@@ -34,6 +33,7 @@ import java.util.regex.Pattern;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.CompressionParams;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.WrappedRunnable;
@@ -111,7 +111,7 @@ public class CommitLogArchiver
                         File directory = new File(dir);
                         if (!directory.exists())
                         {
-                            if (!directory.mkdir())
+                            if (!directory.tryCreateDirectory())
                             {
                                 throw new RuntimeException("Unable to create directory: " + dir);
                             }
@@ -225,7 +225,7 @@ public class CommitLogArchiver
 
         for (String dir : restoreDirectories.split(DELIMITER))
         {
-            File[] files = new File(dir).listFiles();
+            File[] files = new File(dir).tryList();
             if (files == null)
             {
                 throw new RuntimeException("Unable to list directory " + dir);
@@ -233,14 +233,14 @@ public class CommitLogArchiver
             for (File fromFile : files)
             {
                 CommitLogDescriptor fromHeader = CommitLogDescriptor.fromHeader(fromFile, DatabaseDescriptor.getEncryptionContext());
-                CommitLogDescriptor fromName = CommitLogDescriptor.isValid(fromFile.getName()) ? CommitLogDescriptor.fromFileName(fromFile.getName()) : null;
+                CommitLogDescriptor fromName = CommitLogDescriptor.isValid(fromFile.name()) ? CommitLogDescriptor.fromFileName(fromFile.name()) : null;
                 CommitLogDescriptor descriptor;
                 if (fromHeader == null && fromName == null)
-                    throw new IllegalStateException("Cannot safely construct descriptor for segment, either from its name or its header: " + fromFile.getPath());
+                    throw new IllegalStateException("Cannot safely construct descriptor for segment, either from its name or its header: " + fromFile.path());
                 else if (fromHeader != null && fromName != null && !fromHeader.equalsIgnoringCompression(fromName))
-                    throw new IllegalStateException(String.format("Cannot safely construct descriptor for segment, as name and header descriptors do not match (%s vs %s): %s", fromHeader, fromName, fromFile.getPath()));
+                    throw new IllegalStateException(String.format("Cannot safely construct descriptor for segment, as name and header descriptors do not match (%s vs %s): %s", fromHeader, fromName, fromFile.path()));
                 else if (fromName != null && fromHeader == null)
-                    throw new IllegalStateException("Cannot safely construct descriptor for segment, as name descriptor implies a version that should contain a header descriptor, but that descriptor could not be read: " + fromFile.getPath());
+                    throw new IllegalStateException("Cannot safely construct descriptor for segment, as name descriptor implies a version that should contain a header descriptor, but that descriptor could not be read: " + fromFile.path());
                 else if (fromHeader != null)
                     descriptor = fromHeader;
                 else descriptor = fromName;
@@ -264,12 +264,12 @@ public class CommitLogArchiver
                 if (toFile.exists())
                 {
                     logger.trace("Skipping restore of archive {} as the segment already exists in the restore location {}",
-                                 fromFile.getPath(), toFile.getPath());
+                                 fromFile.path(), toFile.path());
                     continue;
                 }
 
-                String command = FROM.matcher(restoreCommand).replaceAll(Matcher.quoteReplacement(fromFile.getPath()));
-                command = TO.matcher(command).replaceAll(Matcher.quoteReplacement(toFile.getPath()));
+                String command = FROM.matcher(restoreCommand).replaceAll(Matcher.quoteReplacement(fromFile.path()));
+                command = TO.matcher(command).replaceAll(Matcher.quoteReplacement(toFile.path()));
                 try
                 {
                     exec(command);
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java
index 700f12a..9e95658 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java
@@ -22,9 +22,7 @@ package org.apache.cassandra.db.commitlog;
 
 import java.io.DataInput;
 import java.io.EOFException;
-import java.io.File;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.Collections;
@@ -40,6 +38,8 @@ import com.google.common.base.Objects;
 import org.apache.cassandra.config.ParameterizedClass;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.security.EncryptionContext;
 import org.json.simple.JSONValue;
@@ -131,9 +131,8 @@ public class CommitLogDescriptor
 
     public static CommitLogDescriptor fromHeader(File file, EncryptionContext encryptionContext)
     {
-        try (RandomAccessFile raf = new RandomAccessFile(file, "r"))
+        try (FileInputStreamPlus raf = new FileInputStreamPlus(file))
         {
-            assert raf.getFilePointer() == 0;
             return readHeader(raf, encryptionContext);
         }
         catch (EOFException e)
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java
index 5123580..451ee37 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java
@@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.zip.CRC32;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.cassandra.io.util.File;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -99,7 +100,7 @@ public class CommitLogReader
             {
                 if (shouldSkip(file))
                 {
-                    logger.info("Skipping playback of empty log: {}", file.getName());
+                    logger.info("Skipping playback of empty log: {}", file.name());
                 }
                 else
                 {
@@ -172,7 +173,7 @@ public class CommitLogReader
                                      boolean tolerateTruncation) throws IOException
     {
         // just transform from the file name (no reading of headers) to determine version
-        CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(file.getName());
+        CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(file.name());
 
         try(RandomAccessReader reader = RandomAccessReader.open(file))
         {
@@ -263,7 +264,7 @@ public class CommitLogReader
     private boolean shouldSkipSegmentId(File file, CommitLogDescriptor desc, CommitLogPosition minPosition)
     {
         logger.debug("Reading {} (CL version {}, messaging version {}, compression {})",
-            file.getPath(),
+            file.path(),
             desc.version,
             desc.getMessagingVersion(),
             desc.compression);
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index 501ee02..b59480e 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@ -18,7 +18,6 @@
  */
 package org.apache.cassandra.db.commitlog;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -30,6 +29,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Ordering;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.commons.lang3.StringUtils;
 
 import org.apache.cassandra.utils.concurrent.Future;
@@ -171,7 +171,7 @@ public class CommitLogReplayer implements CommitLogReadHandler
         // Can only reach this point if CDC is enabled, thus we have a CDCSegmentManager
         ((CommitLogSegmentManagerCDC)CommitLog.instance.segmentManager).addCDCSize(f.length());
 
-        File dest = new File(DatabaseDescriptor.getCDCLogLocation(), f.getName());
+        File dest = new File(DatabaseDescriptor.getCDCLogLocation(), f.name());
 
         // If hard link already exists, assume it's from a previous node run. If people are mucking around in the cdc_raw
         // directory that's on them.
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index 86442d8..45678f5 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.db.commitlog;
 
-import java.io.File;
-import java.io.FileWriter;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
@@ -31,6 +29,8 @@ import java.util.concurrent.locks.LockSupport;
 import java.util.zip.CRC32;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileWriter;
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 
 import com.codahale.metrics.Timer;
@@ -76,10 +76,10 @@ public abstract class CommitLogSegment
     static
     {
         long maxId = Long.MIN_VALUE;
-        for (File file : new File(DatabaseDescriptor.getCommitLogLocation()).listFiles())
+        for (File file : new File(DatabaseDescriptor.getCommitLogLocation()).tryList())
         {
-            if (CommitLogDescriptor.isValid(file.getName()))
-                maxId = Math.max(CommitLogDescriptor.fromFileName(file.getName()).id, maxId);
+            if (CommitLogDescriptor.isValid(file.name()))
+                maxId = Math.max(CommitLogDescriptor.fromFileName(file.name()).id, maxId);
         }
         replayLimitId = idBase = Math.max(currentTimeMillis(), maxId + 1);
     }
@@ -464,7 +464,7 @@ public abstract class CommitLogSegment
      */
     public String getPath()
     {
-        return logFile.getPath();
+        return logFile.path();
     }
 
     /**
@@ -472,7 +472,7 @@ public abstract class CommitLogSegment
      */
     public String getName()
     {
-        return logFile.getName();
+        return logFile.name();
     }
 
     /**
@@ -480,7 +480,7 @@ public abstract class CommitLogSegment
      */
     public File getCDCFile()
     {
-        return new File(DatabaseDescriptor.getCDCLogLocation(), logFile.getName());
+        return new File(DatabaseDescriptor.getCDCLogLocation(), logFile.name());
     }
 
     /**
@@ -678,8 +678,8 @@ public abstract class CommitLogSegment
     {
         public int compare(File f, File f2)
         {
-            CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(f.getName());
-            CommitLogDescriptor desc2 = CommitLogDescriptor.fromFileName(f2.getName());
+            CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(f.name());
+            CommitLogDescriptor desc2 = CommitLogDescriptor.fromFileName(f2.name());
             return Long.compare(desc.id, desc2.id);
         }
     }
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
index 6561137..6f6a1c2 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.db.commitlog;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.file.FileVisitResult;
 import java.nio.file.Files;
@@ -28,6 +27,7 @@ import java.util.concurrent.*;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.RateLimiter;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -165,8 +165,8 @@ public class CommitLogSegmentManagerCDC extends AbstractCommitLogSegmentManager
         super.handleReplayedSegment(file);
 
         // delete untracked cdc segment hard link files if their index files do not exist
-        File cdcFile = new File(DatabaseDescriptor.getCDCLogLocation(), file.getName());
-        File cdcIndexFile = new File(DatabaseDescriptor.getCDCLogLocation(), CommitLogDescriptor.fromFileName(file.getName()).cdcIndexFileName());
+        File cdcFile = new File(DatabaseDescriptor.getCDCLogLocation(), file.name());
+        File cdcIndexFile = new File(DatabaseDescriptor.getCDCLogLocation(), CommitLogDescriptor.fromFileName(file.name()).cdcIndexFileName());
         if (cdcFile.exists() && !cdcIndexFile.exists())
         {
             logger.trace("(Unopened) CDC segment {} is no longer needed and will be deleted now", cdcFile);
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java
index 0e051cf..c144d09 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java
@@ -19,6 +19,7 @@
 package org.apache.cassandra.db.commitlog;
 
 import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.io.util.FileUtils;
 
 public class CommitLogSegmentManagerStandard extends AbstractCommitLogSegmentManager
diff --git a/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java b/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java
index 45db2f6..1108739 100644
--- a/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java
+++ b/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java
@@ -207,4 +207,4 @@ public class IntervalSet<T extends Comparable<T>>
         }
     }
 
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java b/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java
index 6ecdbd3..d564117 100644
--- a/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java
@@ -90,7 +90,7 @@ public class MemoryMappedSegment extends CommitLogSegment
         {
             throw new FSWriteError(e, getPath());
         }
-        NativeLibrary.trySkipCache(fd, startMarker, nextMarker, logFile.getAbsolutePath());
+        NativeLibrary.trySkipCache(fd, startMarker, nextMarker, logFile.absolutePath());
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java
index 0644f32..ae170a8 100644
--- a/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java
@@ -43,4 +43,4 @@ class PeriodicCommitLogService extends AbstractCommitLogService
             pending.decrementAndGet();
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java b/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java
index e8250b4..2d585e6 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java
@@ -44,6 +44,7 @@ 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.io.util.File;
 import org.apache.cassandra.utils.ExecutorUtils;
 import org.apache.cassandra.utils.NoSpamLogger;
 
@@ -308,13 +309,13 @@ public class CompactionLogger
         private static OutputStreamWriter createStream() throws IOException
         {
             int count = 0;
-            Path compactionLog = Paths.get(logDirectory, "compaction.log");
+            Path compactionLog = new File(logDirectory, "compaction.log").toPath();
             if (Files.exists(compactionLog))
             {
                 Path tryPath = compactionLog;
                 while (Files.exists(tryPath))
                 {
-                    tryPath = Paths.get(logDirectory, String.format("compaction-%d.log", count++));
+                    tryPath = new File(logDirectory, String.format("compaction-%d.log", count++)).toPath();
                 }
                 Files.move(compactionLog, tryPath);
             }
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index e1f940b..c272314 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.compaction;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.Callable;
@@ -41,6 +40,7 @@ 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.io.util.File;
 import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -1006,7 +1006,7 @@ public class CompactionManager implements CompactionManagerMBean
             }
             // group by keyspace/columnfamily
             ColumnFamilyStore cfs = Keyspace.open(desc.ksname).getColumnFamilyStore(desc.cfname);
-            descriptors.put(cfs, cfs.getDirectories().find(new File(filename.trim()).getName()));
+            descriptors.put(cfs, cfs.getDirectories().find(new File(filename.trim()).name()));
         }
 
         List<Future<?>> futures = new ArrayList<>(descriptors.size());
@@ -1032,7 +1032,7 @@ public class CompactionManager implements CompactionManagerMBean
             }
             // group by keyspace/columnfamily
             ColumnFamilyStore cfs = Keyspace.open(desc.ksname).getColumnFamilyStore(desc.cfname);
-            desc = cfs.getDirectories().find(new File(filename.trim()).getName());
+            desc = cfs.getDirectories().find(new File(filename.trim()).name());
             if (desc != null)
                 descriptors.put(cfs, desc);
         }
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
index d7d3ba5..9ee1a97 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.db.compaction;
 
 
-import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -39,6 +38,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.primitives.Longs;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -1144,13 +1144,13 @@ public class CompactionStrategyManager implements INotificationConsumer
                 {
                     int idx = holder.getStrategyIndex(strategy);
                     if (idx >= 0)
-                        return Collections.singletonList(locations[idx].location.getAbsolutePath());
+                        return Collections.singletonList(locations[idx].location.absolutePath());
                 }
             }
             List<String> folders = new ArrayList<>(locations.length);
             for (Directories.DataDirectory location : locations)
             {
-                folders.add(location.location.getAbsolutePath());
+                folders.add(location.location.absolutePath());
             }
             return folders;
         }
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index d271bb3..666b037 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -17,8 +17,10 @@
  */
 package org.apache.cassandra.db.compaction;
 
+import java.io.IOError;
+import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.io.*;
+
 import java.util.*;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
@@ -28,6 +30,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableSet;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java
index 288af2b..eb1d8f9 100644
--- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java
+++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java
@@ -101,4 +101,4 @@ public final class SizeTieredCompactionStrategyOptions
         return String.format("Min sstable size: %d, bucket low: %f, bucket high: %f", minSSTableSize, bucketLow, bucketHigh);
     }
 
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/db/compaction/Upgrader.java b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
index 0cb4500..7f3b0f1 100644
--- a/src/java/org/apache/cassandra/db/compaction/Upgrader.java
+++ b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.compaction;
 
-import java.io.File;
 import java.util.*;
 import java.util.function.LongPredicate;
 
@@ -33,6 +32,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.OutputHandler;
 import org.apache.cassandra.utils.UUIDGen;
@@ -57,7 +57,7 @@ public class Upgrader
         this.sstable = txn.onlyOne();
         this.outputHandler = outputHandler;
 
-        this.directory = new File(sstable.getFilename()).getParentFile();
+        this.directory = new File(sstable.getFilename()).parent();
 
         this.controller = new UpgradeController(cfs);
 
diff --git a/src/java/org/apache/cassandra/db/compaction/Verifier.java b/src/java/org/apache/cassandra/db/compaction/Verifier.java
index 30e74ad..8b7f0d6 100644
--- a/src/java/org/apache/cassandra/db/compaction/Verifier.java
+++ b/src/java/org/apache/cassandra/db/compaction/Verifier.java
@@ -36,8 +36,10 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
 import org.apache.cassandra.io.sstable.metadata.MetadataType;
 import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataIntegrityMetadata;
 import org.apache.cassandra.io.util.DataIntegrityMetadata.FileDigestValidator;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.schema.TableMetadata;
@@ -52,8 +54,8 @@ import org.apache.cassandra.utils.UUIDGen;
 
 import java.io.BufferedInputStream;
 import java.io.Closeable;
+import java.io.DataInput;
 import java.io.DataInputStream;
-import java.io.File;
 import java.io.IOError;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -67,6 +69,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Function;
 import java.util.function.LongPredicate;
 
+import org.apache.cassandra.io.util.File;
+
 public class Verifier implements Closeable
 {
     private final ColumnFamilyStore cfs;
@@ -447,7 +451,7 @@ public class Verifier implements Closeable
         Path bfPath = Paths.get(sstable.descriptor.filenameFor(Component.FILTER));
         if (Files.exists(bfPath))
         {
-            try (DataInputStream stream = new DataInputStream(new BufferedInputStream(Files.newInputStream(bfPath)));
+            try (FileInputStreamPlus stream = new File(bfPath).newInputStream();
                  IFilter bf = BloomFilterSerializer.deserialize(stream, sstable.descriptor.version.hasOldBfFormat()))
             {
             }
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
index 2251f6a..74ebac7 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Set;
 import java.util.UUID;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
index 574c6a4..ff935f8 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.lifecycle;
 
-import java.io.File;
 import java.nio.file.Path;
 import java.util.*;
 import java.util.function.BiPredicate;
@@ -25,6 +24,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
 import com.google.common.collect.*;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogAwareFileLister.java b/src/java/org/apache/cassandra/db/lifecycle/LogAwareFileLister.java
index 254966e..120f9bc 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogAwareFileLister.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogAwareFileLister.java
@@ -20,7 +20,6 @@
  */
 package org.apache.cassandra.db.lifecycle;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.file.DirectoryStream;
 import java.nio.file.Files;
@@ -32,6 +31,7 @@ import java.util.stream.StreamSupport;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -106,7 +106,7 @@ final class LogAwareFileLister
         try
         {
             return StreamSupport.stream(stream.spliterator(), false)
-                                .map(Path::toFile)
+                                .map(File::new)
                                 .filter((f) -> !f.isDirectory())
                                 .collect(Collectors.toList());
         }
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogFile.java b/src/java/org/apache/cassandra/db/lifecycle/LogFile.java
index 1fc371f..5a468f8 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogFile.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogFile.java
@@ -20,7 +20,6 @@
  */
 package org.apache.cassandra.db.lifecycle;
 
-import java.io.File;
 import java.nio.file.Path;
 import java.util.*;
 import java.util.regex.Matcher;
@@ -30,6 +29,7 @@ import java.util.stream.Collectors;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterables;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -77,7 +77,7 @@ final class LogFile implements AutoCloseable
 
     static LogFile make(File logReplica)
     {
-        return make(logReplica.getName(), Collections.singletonList(logReplica));
+        return make(logReplica.name(), Collections.singletonList(logReplica));
     }
 
     static LogFile make(String fileName, List<File> logReplicas)
@@ -139,7 +139,7 @@ final class LogFile implements AutoCloseable
 
     static boolean isLogFile(File file)
     {
-        return LogFile.FILE_REGEX.matcher(file.getName()).matches();
+        return LogFile.FILE_REGEX.matcher(file.name()).matches();
     }
 
     LogFile(OperationType type, UUID id, List<File> replicas)
@@ -324,7 +324,7 @@ final class LogFile implements AutoCloseable
         for (SSTableReader sstable : tables)
         {
             File directory = sstable.descriptor.directory;
-            String fileName = StringUtils.join(directory, File.separator, getFileName());
+            String fileName = StringUtils.join(directory, File.pathSeparator(), getFileName());
             replicas.maybeCreateReplica(directory, fileName, records);
         }
         return LogRecord.make(type, tables);
@@ -333,7 +333,7 @@ final class LogFile implements AutoCloseable
     private LogRecord makeAddRecord(SSTable table)
     {
         File directory = table.descriptor.directory;
-        String fileName = StringUtils.join(directory, File.separator, getFileName());
+        String fileName = StringUtils.join(directory, File.pathSeparator(), getFileName());
         replicas.maybeCreateReplica(directory, fileName, records);
         return LogRecord.make(Type.ADD, table);
     }
@@ -348,7 +348,7 @@ final class LogFile implements AutoCloseable
         assert type == Type.ADD || type == Type.REMOVE;
 
         File directory = table.descriptor.directory;
-        String fileName = StringUtils.join(directory, File.separator, getFileName());
+        String fileName = StringUtils.join(directory, File.pathSeparator(), getFileName());
         replicas.maybeCreateReplica(directory, fileName, records);
         return record.asType(type);
     }
@@ -444,7 +444,7 @@ final class LogFile implements AutoCloseable
     private static Set<File> getRecordFiles(NavigableSet<File> files, LogRecord record)
     {
         String fileName = record.fileName();
-        return files.stream().filter(f -> f.getName().startsWith(fileName)).collect(Collectors.toSet());
+        return files.stream().filter(f -> f.name().startsWith(fileName)).collect(Collectors.toSet());
     }
 
     boolean exists()
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java b/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
index 513ad87..4fb3947 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
@@ -20,11 +20,11 @@
  */
 package org.apache.cassandra.db.lifecycle;
 
-import java.io.File;
-import java.io.FilenameFilter;
+
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.*;
+import java.util.function.BiPredicate;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
@@ -33,7 +33,9 @@ import java.util.zip.CRC32;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.SSTable;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PathUtils;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -283,8 +285,8 @@ final class LogRecord
 
     public static List<File> getExistingFiles(String absoluteFilePath)
     {
-        Path path = Paths.get(absoluteFilePath);
-        File[] files = path.getParent().toFile().listFiles((dir, name) -> name.startsWith(path.getFileName().toString()));
+        File file = new File(absoluteFilePath);
+        File[] files = file.parent().tryList((dir, name) -> name.startsWith(file.name()));
         // files may be null if the directory does not exist yet, e.g. when tracking new files
         return files == null ? Collections.emptyList() : Arrays.asList(files);
     }
@@ -302,13 +304,13 @@ final class LogRecord
         Map<File, TreeSet<String>> dirToFileNamePrefix = new HashMap<>();
         for (String absolutePath : absoluteFilePaths)
         {
-            Path fullPath = Paths.get(absolutePath);
+            Path fullPath = new File(absolutePath).toPath();
             Path path = fullPath.getParent();
             if (path != null)
-                dirToFileNamePrefix.computeIfAbsent(path.toFile(), (k) -> new TreeSet<>()).add(fullPath.getFileName().toString());
+                dirToFileNamePrefix.computeIfAbsent(new File(path), (k) -> new TreeSet<>()).add(fullPath.getFileName().toString());
         }
 
-        FilenameFilter ff = (dir, name) -> {
+        BiPredicate<File, String> ff = (dir, name) -> {
             TreeSet<String> dirSet = dirToFileNamePrefix.get(dir);
             // if the set contains a prefix of the current file name, the file name we have here should sort directly
             // after the prefix in the tree set, which means we can use 'floor' to get the prefix (returns the largest
@@ -317,7 +319,7 @@ final class LogRecord
             String baseName = dirSet.floor(name);
             if (baseName != null && name.startsWith(baseName))
             {
-                String absolutePath = new File(dir, baseName).getPath();
+                String absolutePath = new File(dir, baseName).path();
                 fileMap.computeIfAbsent(absolutePath, k -> new ArrayList<>()).add(new File(dir, name));
             }
             return false;
@@ -325,7 +327,7 @@ final class LogRecord
 
         // populate the file map:
         for (File f : dirToFileNamePrefix.keySet())
-            f.listFiles(ff);
+            f.tryList(ff);
 
         return fileMap;
     }
@@ -338,14 +340,12 @@ final class LogRecord
 
     String fileName()
     {
-        return absolutePath.isPresent() ? Paths.get(absolutePath.get()).getFileName().toString() : "";
+        return absolutePath.isPresent() ? new File(absolutePath.get()).name() : "";
     }
 
     boolean isInFolder(Path folder)
     {
-        return absolutePath.isPresent()
-               ? FileUtils.isContained(folder.toFile(), Paths.get(absolutePath.get()).toFile())
-               : false;
+        return absolutePath.isPresent() && PathUtils.isContained(folder, new File(absolutePath.get()).toPath());
     }
 
     String absolutePath()
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogReplica.java b/src/java/org/apache/cassandra/db/lifecycle/LogReplica.java
index cdc4c35..efd56d8 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogReplica.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogReplica.java
@@ -18,12 +18,12 @@
 
 package org.apache.cassandra.db.lifecycle;
 
-import java.io.File;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.io.IOException;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -54,18 +54,18 @@ final class LogReplica implements AutoCloseable
 
     static LogReplica create(File directory, String fileName)
     {
-        int folderFD = NativeLibrary.tryOpenDirectory(directory.getPath());
+        int folderFD = NativeLibrary.tryOpenDirectory(directory.path());
         if (folderFD == -1 && !FBUtilities.isWindows)
-            throw new FSReadError(new IOException(String.format("Invalid folder descriptor trying to create log replica %s", directory.getPath())), directory.getPath());
+            throw new FSReadError(new IOException(String.format("Invalid folder descriptor trying to create log replica %s", directory.path())), directory.path());
 
         return new LogReplica(new File(fileName), folderFD);
     }
 
     static LogReplica open(File file)
     {
-        int folderFD = NativeLibrary.tryOpenDirectory(file.getParentFile().getPath());
+        int folderFD = NativeLibrary.tryOpenDirectory(file.parent().path());
         if (folderFD == -1 && !FBUtilities.isWindows)
-            throw new FSReadError(new IOException(String.format("Invalid folder descriptor trying to create log replica %s", file.getParentFile().getPath())), file.getParentFile().getPath());
+            throw new FSReadError(new IOException(String.format("Invalid folder descriptor trying to create log replica %s", file.parent().path())), file.parent().path());
 
         return new LogReplica(file, folderFD);
     }
@@ -88,12 +88,12 @@ final class LogReplica implements AutoCloseable
 
     String getFileName()
     {
-        return file.getName();
+        return file.name();
     }
 
     String getDirectory()
     {
-        return file.getParent();
+        return file.parentPath();
     }
 
     void append(LogRecord record)
@@ -162,7 +162,7 @@ final class LogReplica implements AutoCloseable
 
     void printContentsWithAnyErrors(StringBuilder str)
     {
-        str.append(file.getPath());
+        str.append(file.path());
         str.append(System.lineSeparator());
         FileUtils.readLines(file).forEach(line -> printLineWithAnyError(str, line));
     }
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java b/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java
index 0295357..316e4b6 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.lifecycle;
 
-import java.io.File;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedHashMap;
@@ -29,6 +28,7 @@ import java.util.function.Function;
 import java.util.stream.Collectors;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -61,7 +61,7 @@ public class LogReplicaSet implements AutoCloseable
 
     void addReplica(File file)
     {
-        File directory = file.getParentFile();
+        File directory = file.parent();
         assert !replicasByFile.containsKey(directory);
         try
         {
@@ -268,6 +268,6 @@ public class LogReplicaSet implements AutoCloseable
     @VisibleForTesting
     List<String> getFilePaths()
     {
-        return replicas().stream().map(LogReplica::file).map(File::getPath).collect(Collectors.toList());
+        return replicas().stream().map(LogReplica::file).map(File::path).collect(Collectors.toList());
     }
 }
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
index 85df4d6..09717fc 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.db.lifecycle;
 
 import java.io.ByteArrayOutputStream;
-import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.nio.file.Files;
@@ -31,6 +30,7 @@ import java.util.function.Predicate;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.Runnables;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -506,16 +506,16 @@ class LogTransaction extends Transactional.AbstractTransactional implements Tran
 
         void list(File directory)
         {
-            Arrays.stream(directory.listFiles(LogFile::isLogFile)).forEach(this::add);
+            Arrays.stream(directory.tryList(LogFile::isLogFile)).forEach(this::add);
         }
 
         void add(File file)
         {
-            List<File> filesByName = files.get(file.getName());
+            List<File> filesByName = files.get(file.name());
             if (filesByName == null)
             {
                 filesByName = new ArrayList<>();
-                files.put(file.getName(), filesByName);
+                files.put(file.name(), filesByName);
             }
 
             filesByName.add(file);
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index 3d72a11..e15347b 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.lifecycle;
 
-import java.io.File;
 import java.util.*;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicReference;
@@ -32,6 +31,7 @@ import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Memtable;
 import org.apache.cassandra.db.commitlog.CommitLogPosition;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java
index 0bfe993..e547e0f 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java
@@ -18,13 +18,13 @@
 
 package org.apache.cassandra.db.streaming;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.function.UnaryOperator;
 
 import com.google.common.base.Throwables;
 import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java
index 6835fad..17c40b8 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java
@@ -25,6 +25,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
 import com.google.common.collect.UnmodifiableIterator;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java
index 6481f4b..cb513ab 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java
@@ -17,11 +17,11 @@
  */
 package org.apache.cassandra.db.streaming;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collection;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/db/streaming/ComponentContext.java b/src/java/org/apache/cassandra/db/streaming/ComponentContext.java
index b9c60b9..c8c08aa 100644
--- a/src/java/org/apache/cassandra/db/streaming/ComponentContext.java
+++ b/src/java/org/apache/cassandra/db/streaming/ComponentContext.java
@@ -26,9 +26,7 @@ import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.util.FileUtils;
 
-import java.io.File;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.channels.FileChannel;
 import java.util.HashMap;
 import java.util.Map;
@@ -38,6 +36,8 @@ import java.util.Set;
  * Mutable SSTable components and their hardlinks to avoid concurrent sstable component modification
  * during entire-sstable-streaming.
  */
+import org.apache.cassandra.io.util.File;
+
 public class ComponentContext implements AutoCloseable
 {
     private static final Logger logger = LoggerFactory.getLogger(ComponentContext.class);
@@ -81,9 +81,9 @@ public class ComponentContext implements AutoCloseable
      */
     public FileChannel channel(Descriptor descriptor, Component component, long size) throws IOException
     {
-        String toTransfer = hardLinks.containsKey(component) ? hardLinks.get(component).getPath() : descriptor.filenameFor(component);
+        String toTransfer = hardLinks.containsKey(component) ? hardLinks.get(component).path() : descriptor.filenameFor(component);
         @SuppressWarnings("resource") // file channel will be closed by Caller
-        FileChannel channel = new RandomAccessFile(toTransfer, "r").getChannel();
+        FileChannel channel = new File(toTransfer).newReadChannel();
 
         assert size == channel.size() : String.format("Entire sstable streaming expects %s file size to be %s but got %s.",
                                                       component, size, channel.size());
diff --git a/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java b/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java
index bb896ca..71aa0f8 100644
--- a/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java
+++ b/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java
@@ -28,13 +28,14 @@ import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.*;
 
 /**
  * SSTable components and their sizes to be tranfered via entire-sstable-streaming
  */
+import org.apache.cassandra.io.util.File;
+
 public final class ComponentManifest implements Iterable<Component>
 {
     private static final List<Component> STREAM_COMPONENTS = ImmutableList.of(Component.DATA, Component.PRIMARY_INDEX, Component.STATS,
diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java b/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java
index f8c13df..dc7f407 100644
--- a/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java
+++ b/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java
@@ -187,4 +187,4 @@ public class OfflineTokenAllocator
             throw new IllegalStateException("Unexpected UnknownHostException", e);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/ReplicationStrategy.java b/src/java/org/apache/cassandra/dht/tokenallocator/ReplicationStrategy.java
index 6dbd37c..8cb5fe1 100644
--- a/src/java/org/apache/cassandra/dht/tokenallocator/ReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/dht/tokenallocator/ReplicationStrategy.java
@@ -26,4 +26,4 @@ interface ReplicationStrategy<Unit>
      * @return Some hashable object.
      */
     Object getGroup(Unit unit);
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocator.java b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocator.java
index 2eb9a4c..7645317 100644
--- a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocator.java
+++ b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocator.java
@@ -24,4 +24,4 @@ import org.apache.cassandra.dht.Token;
 public interface TokenAllocator<Unit>
 {
     Collection<Token> addUnit(Unit newUnit, int numTokens);
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocatorDiagnostics.java b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocatorDiagnostics.java
index 04d7455..c29980c 100644
--- a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocatorDiagnostics.java
+++ b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocatorDiagnostics.java
@@ -193,4 +193,4 @@ final class TokenAllocatorDiagnostics
         return service.isEnabled(TokenAllocatorEvent.class, type);
     }
 
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocatorEvent.java b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocatorEvent.java
index ca59938..acb5ed2 100644
--- a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocatorEvent.java
+++ b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocatorEvent.java
@@ -110,4 +110,4 @@ final class TokenAllocatorEvent<Unit> extends DiagnosticEvent
         ret.put("tokenInfo", String.valueOf(tokenInfo));
         return ret;
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/diag/DiagnosticEventService.java b/src/java/org/apache/cassandra/diag/DiagnosticEventService.java
index cab9596..8a8391c 100644
--- a/src/java/org/apache/cassandra/diag/DiagnosticEventService.java
+++ b/src/java/org/apache/cassandra/diag/DiagnosticEventService.java
@@ -335,4 +335,4 @@ public final class DiagnosticEventService implements DiagnosticEventServiceMBean
             return Objects.hash(wrapped);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/fql/FullQueryLogger.java b/src/java/org/apache/cassandra/fql/FullQueryLogger.java
index 49e326e..ba88127 100644
--- a/src/java/org/apache/cassandra/fql/FullQueryLogger.java
+++ b/src/java/org/apache/cassandra/fql/FullQueryLogger.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.fql;
 
-import java.io.File;
 import java.nio.ByteBuffer;
 import java.nio.file.Path;
 import java.util.ArrayList;
@@ -30,6 +29,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
 import com.google.common.primitives.Ints;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -194,7 +194,7 @@ public class FullQueryLogger implements QueryEvents.Listener
             //Then decide whether to clean the last used path, possibly configured by JMX
             if (binLog != null && binLog.path != null)
             {
-                File pathFile = binLog.path.toFile();
+                File pathFile = new File(binLog.path);
                 if (pathFile.exists())
                 {
                     pathsToClean.add(pathFile);
diff --git a/src/java/org/apache/cassandra/gms/EndpointState.java b/src/java/org/apache/cassandra/gms/EndpointState.java
index 9a69de3..a48a857 100644
--- a/src/java/org/apache/cassandra/gms/EndpointState.java
+++ b/src/java/org/apache/cassandra/gms/EndpointState.java
@@ -24,6 +24,7 @@ import java.util.stream.Collectors;
 
 import javax.annotation.Nullable;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/gms/FailureDetector.java b/src/java/org/apache/cassandra/gms/FailureDetector.java
index 3ba96af..522e082 100644
--- a/src/java/org/apache/cassandra/gms/FailureDetector.java
+++ b/src/java/org/apache/cassandra/gms/FailureDetector.java
@@ -28,6 +28,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
 import javax.management.openmbean.*;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.Replica;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -253,7 +254,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
         }
         catch (IOException e)
         {
-            throw new FSWriteError(e, (path == null) ? null : path.toFile());
+            throw new FSWriteError(e, path);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/gms/GossipDigest.java b/src/java/org/apache/cassandra/gms/GossipDigest.java
index 53f6c5c..4115c38 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigest.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigest.java
@@ -23,6 +23,7 @@ import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.InetAddressAndPort;
 
 import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer;
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAck2.java b/src/java/org/apache/cassandra/gms/GossipDigestAck2.java
index 0e4062b..7324763 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestAck2.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestAck2.java
@@ -25,6 +25,7 @@ import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.InetAddressAndPort;
 
 import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer;
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestSyn.java b/src/java/org/apache/cassandra/gms/GossipDigestSyn.java
index 17c8da3..c2c736f 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestSyn.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestSyn.java
@@ -30,6 +30,8 @@ import org.apache.cassandra.io.util.DataOutputPlus;
  * This is the first message that gets sent out as a start of the Gossip protocol in a
  * round.
  */
+import org.apache.cassandra.io.util.File;
+
 public class GossipDigestSyn
 {
     public static final IVersionedSerializer<GossipDigestSyn> serializer = new GossipDigestSynSerializer();
diff --git a/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java
index 83c8568..f2622ef 100644
--- a/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java
+++ b/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java
@@ -39,4 +39,4 @@ public class GossipShutdownVerbHandler implements IVerbHandler
         Gossiper.instance.markAsShutdown(message.from());
     }
 
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/gms/GossiperDiagnostics.java b/src/java/org/apache/cassandra/gms/GossiperDiagnostics.java
index 57552cc..13f8647 100644
--- a/src/java/org/apache/cassandra/gms/GossiperDiagnostics.java
+++ b/src/java/org/apache/cassandra/gms/GossiperDiagnostics.java
@@ -110,4 +110,4 @@ final class GossiperDiagnostics
     {
         return service.isEnabled(GossiperEvent.class, type);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/gms/GossiperEvent.java b/src/java/org/apache/cassandra/gms/GossiperEvent.java
index 4ec0cf4..71fee7c 100644
--- a/src/java/org/apache/cassandra/gms/GossiperEvent.java
+++ b/src/java/org/apache/cassandra/gms/GossiperEvent.java
@@ -108,4 +108,4 @@ public final class GossiperEvent extends DiagnosticEvent
         ret.put("unreachableEndpoints", String.valueOf(unreachableEndpoints));
         return ret;
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/gms/GossiperMBean.java b/src/java/org/apache/cassandra/gms/GossiperMBean.java
index 92df2cd..47d7207 100644
--- a/src/java/org/apache/cassandra/gms/GossiperMBean.java
+++ b/src/java/org/apache/cassandra/gms/GossiperMBean.java
@@ -38,4 +38,4 @@ public interface GossiperMBean
     /** Returns each node's database release version */
     public Map<String, List<String>> getReleaseVersionsWithPort();
 
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/gms/HeartBeatState.java b/src/java/org/apache/cassandra/gms/HeartBeatState.java
index 75f4f56..104d957 100644
--- a/src/java/org/apache/cassandra/gms/HeartBeatState.java
+++ b/src/java/org/apache/cassandra/gms/HeartBeatState.java
@@ -27,6 +27,8 @@ import org.apache.cassandra.io.util.DataOutputPlus;
 /**
  * HeartBeat State associated with any given endpoint.
  */
+import org.apache.cassandra.io.util.File;
+
 public class HeartBeatState
 {
     public static final int EMPTY_VERSION = -1;
diff --git a/src/java/org/apache/cassandra/gms/TokenSerializer.java b/src/java/org/apache/cassandra/gms/TokenSerializer.java
index 40d14f8..d73b077 100644
--- a/src/java/org/apache/cassandra/gms/TokenSerializer.java
+++ b/src/java/org/apache/cassandra/gms/TokenSerializer.java
@@ -31,6 +31,8 @@ import java.util.ArrayList;
 import java.util.Collection;
 
 
+import org.apache.cassandra.io.util.File;
+
 public class TokenSerializer
 {
     private static final Logger logger = LoggerFactory.getLogger(TokenSerializer.class);
@@ -62,4 +64,4 @@ public class TokenSerializer
         }
         return tokens;
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/gms/VersionedValue.java b/src/java/org/apache/cassandra/gms/VersionedValue.java
index 880cb98..659f61b 100644
--- a/src/java/org/apache/cassandra/gms/VersionedValue.java
+++ b/src/java/org/apache/cassandra/gms/VersionedValue.java
@@ -24,6 +24,7 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.stream.Collectors;
 
+import org.apache.cassandra.io.util.File;
 import static java.nio.charset.StandardCharsets.ISO_8859_1;
 
 import com.google.common.annotations.VisibleForTesting;
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
index 45ffa4e..b34048d 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.hadoop.cql3;
 
 import java.io.Closeable;
-import java.io.File;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
@@ -28,6 +27,7 @@ import java.util.*;
 import java.util.concurrent.*;
 
 import com.google.common.net.HostAndPort;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -226,9 +226,9 @@ public class CqlBulkRecordWriter extends RecordWriter<Object, List<ByteBuffer>>
     
     private File getTableDirectory() throws IOException
     {
-        File dir = new File(String.format("%s%s%s%s%s-%s", getOutputLocation(), File.separator, keyspace, File.separator, table, UUID.randomUUID().toString()));
+        File dir = new File(String.format("%s%s%s%s%s-%s", getOutputLocation(), File.pathSeparator(), keyspace, File.pathSeparator(), table, UUID.randomUUID().toString()));
         
-        if (!dir.exists() && !dir.mkdirs())
+        if (!dir.exists() && !dir.tryCreateDirectories())
         {
             throw new IOException("Failed to created output directory: " + dir);
         }
diff --git a/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java b/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
index a717027..463f33e 100644
--- a/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
+++ b/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.hints;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.zip.CRC32;
@@ -26,6 +25,7 @@ import com.google.common.base.Preconditions;
 
 import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.utils.Throwables;
 import org.apache.cassandra.utils.NativeLibrary;
 
diff --git a/src/java/org/apache/cassandra/hints/CompressedHintsWriter.java b/src/java/org/apache/cassandra/hints/CompressedHintsWriter.java
index 8792e32..63a59cd 100644
--- a/src/java/org/apache/cassandra/hints/CompressedHintsWriter.java
+++ b/src/java/org/apache/cassandra/hints/CompressedHintsWriter.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.hints;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
@@ -28,6 +27,8 @@ import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.cassandra.io.compress.ICompressor;
 
+import org.apache.cassandra.io.util.File;
+
 public class CompressedHintsWriter extends HintsWriter
 {
     // compressed and uncompressed size is stored at the beginning of each compressed block
diff --git a/src/java/org/apache/cassandra/hints/EncryptedHintsWriter.java b/src/java/org/apache/cassandra/hints/EncryptedHintsWriter.java
index 4786d9c..f9822d9 100644
--- a/src/java/org/apache/cassandra/hints/EncryptedHintsWriter.java
+++ b/src/java/org/apache/cassandra/hints/EncryptedHintsWriter.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.hints;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
@@ -26,6 +25,7 @@ import javax.crypto.Cipher;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.security.EncryptionUtils;
 import org.apache.cassandra.io.compress.ICompressor;
 
diff --git a/src/java/org/apache/cassandra/hints/HintDiagnostics.java b/src/java/org/apache/cassandra/hints/HintDiagnostics.java
index 3ff0834..285193b 100644
--- a/src/java/org/apache/cassandra/hints/HintDiagnostics.java
+++ b/src/java/org/apache/cassandra/hints/HintDiagnostics.java
@@ -82,4 +82,4 @@ final class HintDiagnostics
         return service.isEnabled(HintEvent.class, type);
     }
 
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/hints/HintEvent.java b/src/java/org/apache/cassandra/hints/HintEvent.java
index d8b6943..695357e 100644
--- a/src/java/org/apache/cassandra/hints/HintEvent.java
+++ b/src/java/org/apache/cassandra/hints/HintEvent.java
@@ -99,4 +99,4 @@ final class HintEvent extends DiagnosticEvent
         }
         return ret;
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/hints/HintsCatalog.java b/src/java/org/apache/cassandra/hints/HintsCatalog.java
index 81ec98e..af7940a 100644
--- a/src/java/org/apache/cassandra/hints/HintsCatalog.java
+++ b/src/java/org/apache/cassandra/hints/HintsCatalog.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.hints;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
@@ -27,6 +26,7 @@ import java.util.stream.Stream;
 import javax.annotation.Nullable;
 
 import com.google.common.collect.ImmutableMap;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -148,7 +148,7 @@ final class HintsCatalog
 
     void fsyncDirectory()
     {
-        int fd = NativeLibrary.tryOpenDirectory(hintsDirectory.getAbsolutePath());
+        int fd = NativeLibrary.tryOpenDirectory(hintsDirectory.absolutePath());
         if (fd != -1)
         {
             try
@@ -158,14 +158,14 @@ final class HintsCatalog
             }
             catch (FSError e) // trySync failed
             {
-                logger.error("Unable to sync directory {}", hintsDirectory.getAbsolutePath(), e);
+                logger.error("Unable to sync directory {}", hintsDirectory.absolutePath(), e);
                 FileUtils.handleFSErrorAndPropagate(e);
             }
         }
         else if (!FBUtilities.isWindows)
         {
-            logger.error("Unable to open directory {}", hintsDirectory.getAbsolutePath());
-            FileUtils.handleFSErrorAndPropagate(new FSWriteError(new IOException(String.format("Unable to open hint directory %s", hintsDirectory.getAbsolutePath())), hintsDirectory.getAbsolutePath()));
+            logger.error("Unable to open directory {}", hintsDirectory.absolutePath());
+            FileUtils.handleFSErrorAndPropagate(new FSWriteError(new IOException(String.format("Unable to open hint directory %s", hintsDirectory.absolutePath())), hintsDirectory.absolutePath()));
         }
     }
 
diff --git a/src/java/org/apache/cassandra/hints/HintsDescriptor.java b/src/java/org/apache/cassandra/hints/HintsDescriptor.java
index 1979637..4c7acf1 100644
--- a/src/java/org/apache/cassandra/hints/HintsDescriptor.java
+++ b/src/java/org/apache/cassandra/hints/HintsDescriptor.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.hints;
 
 import java.io.DataInput;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Path;
@@ -35,6 +34,9 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Objects;
 import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -232,13 +234,13 @@ final class HintsDescriptor
 
     static Optional<HintsDescriptor> readFromFileQuietly(Path path)
     {
-        try (RandomAccessFile raf = new RandomAccessFile(path.toFile(), "r"))
+        try (FileInputStreamPlus raf = new FileInputStreamPlus(path))
         {
             return Optional.of(deserialize(raf));
         }
         catch (ChecksumMismatchException e)
         {
-            throw new FSReadError(e, path.toFile());
+            throw new FSReadError(e, path);
         }
         catch (IOException e)
         {
@@ -271,15 +273,15 @@ final class HintsDescriptor
         }
     }
 
-    static HintsDescriptor readFromFile(Path path)
+    static HintsDescriptor readFromFile(File path)
     {
-        try (RandomAccessFile raf = new RandomAccessFile(path.toFile(), "r"))
+        try (FileInputStreamPlus raf = new FileInputStreamPlus(path))
         {
             return deserialize(raf);
         }
         catch (IOException e)
         {
-            throw new FSReadError(e, path.toFile());
+            throw new FSReadError(e, path);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java b/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
index 705715c..b6de749 100644
--- a/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
+++ b/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.hints;
 
-import java.io.File;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
@@ -29,12 +28,14 @@ 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.ExecutorPlus;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
diff --git a/src/java/org/apache/cassandra/hints/HintsDispatcher.java b/src/java/org/apache/cassandra/hints/HintsDispatcher.java
index 4fe1ae3..2b6d9a3 100644
--- a/src/java/org/apache/cassandra/hints/HintsDispatcher.java
+++ b/src/java/org/apache/cassandra/hints/HintsDispatcher.java
@@ -17,23 +17,23 @@
  */
 package org.apache.cassandra.hints;
 
-import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.function.BooleanSupplier;
 import java.util.function.Function;
 
 import com.google.common.util.concurrent.RateLimiter;
-import org.apache.cassandra.utils.concurrent.Condition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.net.RequestCallback;
 import org.apache.cassandra.exceptions.RequestFailureReason;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.HintsServiceMetrics;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.utils.concurrent.Condition;
 
 
 import static org.apache.cassandra.hints.HintsDispatcher.Callback.Outcome.*;
diff --git a/src/java/org/apache/cassandra/hints/HintsReader.java b/src/java/org/apache/cassandra/hints/HintsReader.java
index 7514fd4..a3944eb 100644
--- a/src/java/org/apache/cassandra/hints/HintsReader.java
+++ b/src/java/org/apache/cassandra/hints/HintsReader.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.hints;
 
 import java.io.EOFException;
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Iterator;
@@ -28,6 +27,7 @@ import javax.annotation.Nullable;
 import com.google.common.primitives.Ints;
 import com.google.common.util.concurrent.RateLimiter;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/hints/HintsService.java b/src/java/org/apache/cassandra/hints/HintsService.java
index b727b35..8108b50 100644
--- a/src/java/org/apache/cassandra/hints/HintsService.java
+++ b/src/java/org/apache/cassandra/hints/HintsService.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.hints;
 
-import java.io.File;
 import java.net.UnknownHostException;
 import java.util.Collection;
 import java.util.Collections;
@@ -33,6 +32,7 @@ import java.util.stream.Collectors;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableMap;
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.utils.concurrent.Future;
 import org.slf4j.Logger;
diff --git a/src/java/org/apache/cassandra/hints/HintsStore.java b/src/java/org/apache/cassandra/hints/HintsStore.java
index 1e72f8b..02a1699 100644
--- a/src/java/org/apache/cassandra/hints/HintsStore.java
+++ b/src/java/org/apache/cassandra/hints/HintsStore.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.hints;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
@@ -28,6 +27,7 @@ import java.util.function.Predicate;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -187,7 +187,7 @@ final class HintsStore
     void delete(HintsDescriptor descriptor)
     {
         File hintsFile = new File(hintsDirectory, descriptor.fileName());
-        if (hintsFile.delete())
+        if (hintsFile.tryDelete())
             logger.info("Deleted hint file {}", descriptor.fileName());
         else if (hintsFile.exists())
             logger.error("Failed to delete hint file {}", descriptor.fileName());
@@ -195,7 +195,7 @@ final class HintsStore
             logger.info("Already deleted hint file {}", descriptor.fileName());
 
         //noinspection ResultOfMethodCallIgnored
-        new File(hintsDirectory, descriptor.checksumFileName()).delete();
+        new File(hintsDirectory, descriptor.checksumFileName()).tryDelete();
     }
 
     boolean hasFiles()
diff --git a/src/java/org/apache/cassandra/hints/HintsWriter.java b/src/java/org/apache/cassandra/hints/HintsWriter.java
index 589802b..8c70893 100644
--- a/src/java/org/apache/cassandra/hints/HintsWriter.java
+++ b/src/java/org/apache/cassandra/hints/HintsWriter.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.hints;
 
-import java.io.File;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
@@ -33,6 +32,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputBufferFixed;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.utils.NativeLibrary;
 import org.apache.cassandra.utils.SyncUtil;
 import org.apache.cassandra.utils.Throwables;
@@ -295,7 +295,7 @@ class HintsWriter implements AutoCloseable
             // don't skip page cache for tiny files, on the assumption that if they are tiny, the target node is probably
             // alive, and if so, the file will be closed and dispatched shortly (within a minute), and the file will be dropped.
             if (position >= DatabaseDescriptor.getTrickleFsyncIntervalInKb() * 1024L)
-                NativeLibrary.trySkipCache(fd, 0, position - (position % PAGE_SIZE), file.getPath());
+                NativeLibrary.trySkipCache(fd, 0, position - (position % PAGE_SIZE), file.path());
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/hints/package-info.java b/src/java/org/apache/cassandra/hints/package-info.java
index faa7b9f..b853f31 100644
--- a/src/java/org/apache/cassandra/hints/package-info.java
+++ b/src/java/org/apache/cassandra/hints/package-info.java
@@ -41,4 +41,4 @@
  * {@link org.apache.cassandra.hints.HintsService} wraps the catalog, the pool, and the two executors, acting as a front-end
  * for hints.
  */
-package org.apache.cassandra.hints;
\ No newline at end of file
+package org.apache.cassandra.hints;
diff --git a/src/java/org/apache/cassandra/index/internal/keys/KeysIndex.java b/src/java/org/apache/cassandra/index/internal/keys/KeysIndex.java
index b044bbe..4c8e75d 100644
--- a/src/java/org/apache/cassandra/index/internal/keys/KeysIndex.java
+++ b/src/java/org/apache/cassandra/index/internal/keys/KeysIndex.java
@@ -86,4 +86,4 @@ public class KeysIndex extends CassandraIndex
                 || !cell.isLive(nowInSec)
                 || compare(indexValue, cell) != 0);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
index 2114d42..9e865d9 100644
--- a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
@@ -160,4 +160,4 @@ public class KeysSearcher extends CassandraIndexSearcher
             return iterator;
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java b/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java
index bb42dc2..2171653 100644
--- a/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java
+++ b/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java
@@ -20,10 +20,10 @@
  */
 package org.apache.cassandra.index.sasi;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.*;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
diff --git a/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java b/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java
index c67c39c..d756737 100644
--- a/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java
+++ b/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.index.sasi;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -33,6 +32,7 @@ import org.apache.cassandra.index.sasi.plan.Expression;
 import org.apache.cassandra.index.sasi.utils.RangeIterator;
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.concurrent.Ref;
 
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/DelimiterAnalyzer.java b/src/java/org/apache/cassandra/index/sasi/analyzer/DelimiterAnalyzer.java
index 05dfedc..b7f297b 100644
--- a/src/java/org/apache/cassandra/index/sasi/analyzer/DelimiterAnalyzer.java
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/DelimiterAnalyzer.java
@@ -108,4 +108,4 @@ public class DelimiterAnalyzer extends AbstractAnalyzer
     {
         return VALID_ANALYZABLE_TYPES.containsKey(validator);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StopWordFactory.java b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StopWordFactory.java
index 1548a6a..56c07f3 100644
--- a/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StopWordFactory.java
+++ b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/StopWordFactory.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.index.sasi.analyzer.filter;
 
 import java.io.BufferedReader;
-import java.io.File;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.nio.charset.StandardCharsets;
@@ -33,6 +32,7 @@ import com.google.common.util.concurrent.MoreExecutors;
 import com.github.benmanes.caffeine.cache.Caffeine;
 import com.github.benmanes.caffeine.cache.LoadingCache;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,7 +45,7 @@ public class StopWordFactory
 
     private static final String DEFAULT_RESOURCE_EXT = "_ST.txt";
     private static final String DEFAULT_RESOURCE_PREFIX = StopWordFactory.class.getPackage()
-            .getName().replace(".", File.separator);
+            .getName().replace(".", File.pathSeparator());
     private static final Set<String> SUPPORTED_LANGUAGES = new HashSet<>(
             Arrays.asList("ar","bg","cs","de","en","es","fi","fr","hi","hu","it",
             "pl","pt","ro","ru","sv"));
@@ -74,7 +74,7 @@ public class StopWordFactory
     private static Set<String> getStopWordsFromResource(String language)
     {
         Set<String> stopWords = new HashSet<>();
-        String resourceName = DEFAULT_RESOURCE_PREFIX + File.separator + language + DEFAULT_RESOURCE_EXT;
+        String resourceName = DEFAULT_RESOURCE_PREFIX + File.pathSeparator() + language + DEFAULT_RESOURCE_EXT;
         try (InputStream is = StopWordFactory.class.getClassLoader().getResourceAsStream(resourceName);
              BufferedReader r = new BufferedReader(new InputStreamReader(is, StandardCharsets.UTF_8)))
         {
diff --git a/src/java/org/apache/cassandra/index/sasi/conf/DataTracker.java b/src/java/org/apache/cassandra/index/sasi/conf/DataTracker.java
index 99516b8..bf2293f 100644
--- a/src/java/org/apache/cassandra/index/sasi/conf/DataTracker.java
+++ b/src/java/org/apache/cassandra/index/sasi/conf/DataTracker.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.index.sasi.conf;
 
-import java.io.File;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
@@ -29,6 +28,7 @@ import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.index.sasi.SSTableIndex;
 import org.apache.cassandra.index.sasi.conf.view.View;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.utils.Pair;
 
 import org.slf4j.Logger;
@@ -181,7 +181,7 @@ public class DataTracker
             }
             catch (Throwable t)
             {
-                logger.error("Can't open index file at " + indexFile.getAbsolutePath() + ", skipping.", t);
+                logger.error("Can't open index file at " + indexFile.absolutePath() + ", skipping.", t);
                 if (index != null)
                     index.release();
             }
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndex.java b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndex.java
index 4d43cd9..e438079 100644
--- a/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndex.java
+++ b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndex.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.index.sasi.disk;
 
 import java.io.*;
 import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
 import java.util.*;
 import java.util.stream.Collectors;
 
@@ -33,6 +34,8 @@ import org.apache.cassandra.index.sasi.utils.RangeIterator;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
@@ -121,13 +124,11 @@ public class OnDiskIndex implements Iterable<OnDiskIndex.DataTerm>, Closeable
         keyFetcher = keyReader;
 
         comparator = cmp;
-        indexPath = index.getAbsolutePath();
+        indexPath = index.absolutePath();
 
-        RandomAccessFile backingFile = null;
-        try
-        {
-            backingFile = new RandomAccessFile(index, "r");
 
+        try (FileInputStreamPlus backingFile = new FileInputStreamPlus(index))
+        {
             descriptor = new Descriptor(backingFile.readUTF());
 
             termSize = OnDiskIndexBuilder.TermSize.of(backingFile.readShort());
@@ -141,32 +142,29 @@ public class OnDiskIndex implements Iterable<OnDiskIndex.DataTerm>, Closeable
             mode = OnDiskIndexBuilder.Mode.mode(backingFile.readUTF());
             hasMarkedPartials = backingFile.readBoolean();
 
-            indexSize = backingFile.length();
-            indexFile = new MappedBuffer(new ChannelProxy(indexPath, backingFile.getChannel()));
-
-            // start of the levels
-            indexFile.position(indexFile.getLong(indexSize - 8));
-
-            int numLevels = indexFile.getInt();
-            levels = new PointerLevel[numLevels];
-            for (int i = 0; i < levels.length; i++)
-            {
-                int blockCount = indexFile.getInt();
-                levels[i] = new PointerLevel(indexFile.position(), blockCount);
-                indexFile.position(indexFile.position() + blockCount * 8);
-            }
-
-            int blockCount = indexFile.getInt();
-            dataLevel = new DataLevel(indexFile.position(), blockCount);
+            FileChannel channel = index.newReadChannel();
+            indexSize = channel.size();
+            indexFile = new MappedBuffer(new ChannelProxy(indexPath, channel));
         }
         catch (IOException e)
         {
             throw new FSReadError(e, index);
         }
-        finally
+
+        // start of the levels
+        indexFile.position(indexFile.getLong(indexSize - 8));
+
+        int numLevels = indexFile.getInt();
+        levels = new PointerLevel[numLevels];
+        for (int i = 0; i < levels.length; i++)
         {
-            FileUtils.closeQuietly(backingFile);
+            int blockCount = indexFile.getInt();
+            levels[i] = new PointerLevel(indexFile.position(), blockCount);
+            indexFile.position(indexFile.position() + blockCount * 8);
         }
+
+        int blockCount = indexFile.getInt();
+        dataLevel = new DataLevel(indexFile.position(), blockCount);
     }
 
     public boolean hasMarkedPartials()
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java
index 0298539..9ba9f9c 100644
--- a/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java
+++ b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.index.sasi.disk;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
@@ -32,6 +31,7 @@ import org.apache.cassandra.index.sasi.sa.SuffixSA;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
@@ -247,15 +247,7 @@ public class OnDiskIndexBuilder
         // no terms means there is nothing to build
         if (terms.isEmpty())
         {
-            try
-            {
-                file.createNewFile();
-            }
-            catch (IOException e)
-            {
-                throw new FSWriteError(e, file);
-            }
-
+            file.createFileIfNotExists();
             return false;
         }
 
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 444db83..fb5e9b9 100644
--- a/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
+++ b/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
@@ -17,13 +17,13 @@
  */
 package org.apache.cassandra.index.sasi.disk;
 
-import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.*;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.DecoratedKey;
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/TokenTree.java b/src/java/org/apache/cassandra/index/sasi/disk/TokenTree.java
index e510cdd..3a401ca 100644
--- a/src/java/org/apache/cassandra/index/sasi/disk/TokenTree.java
+++ b/src/java/org/apache/cassandra/index/sasi/disk/TokenTree.java
@@ -520,4 +520,4 @@ public class TokenTree
             return index < offsets.length ? keyFetcher.apply(offsets[index++]) : endOfData();
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/TokenTreeBuilder.java b/src/java/org/apache/cassandra/index/sasi/disk/TokenTreeBuilder.java
index 29cecc8..01a536c 100644
--- a/src/java/org/apache/cassandra/index/sasi/disk/TokenTreeBuilder.java
+++ b/src/java/org/apache/cassandra/index/sasi/disk/TokenTreeBuilder.java
@@ -75,4 +75,4 @@ public interface TokenTreeBuilder extends Iterable<Pair<Long, LongSet>>
 
     int serializedSize();
     void write(DataOutputPlus out) throws IOException;
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/index/sasi/memory/KeyRangeIterator.java b/src/java/org/apache/cassandra/index/sasi/memory/KeyRangeIterator.java
index 0f681b7..d60914d 100644
--- a/src/java/org/apache/cassandra/index/sasi/memory/KeyRangeIterator.java
+++ b/src/java/org/apache/cassandra/index/sasi/memory/KeyRangeIterator.java
@@ -126,4 +126,4 @@ public class KeyRangeIterator extends RangeIterator<Long, Token>
             return keys.iterator();
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/CombinedTerm.java b/src/java/org/apache/cassandra/index/sasi/utils/CombinedTerm.java
index 81e535d..cc327bc 100644
--- a/src/java/org/apache/cassandra/index/sasi/utils/CombinedTerm.java
+++ b/src/java/org/apache/cassandra/index/sasi/utils/CombinedTerm.java
@@ -81,4 +81,4 @@ public class CombinedTerm implements CombinedValue<DataTerm>
     {
         return term.compareTo(comparator, o.get().getTerm());
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/trie/Cursor.java b/src/java/org/apache/cassandra/index/sasi/utils/trie/Cursor.java
index dde3c8a..7c86b97 100644
--- a/src/java/org/apache/cassandra/index/sasi/utils/trie/Cursor.java
+++ b/src/java/org/apache/cassandra/index/sasi/utils/trie/Cursor.java
@@ -80,4 +80,4 @@ public interface Cursor<K, V>
      * Note: Not all operations support {@link Decision#REMOVE}.
      */
     Decision select(Map.Entry<? extends K, ? extends V> entry);
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/trie/PatriciaTrie.java b/src/java/org/apache/cassandra/index/sasi/utils/trie/PatriciaTrie.java
index 9187894..a36af98 100644
--- a/src/java/org/apache/cassandra/index/sasi/utils/trie/PatriciaTrie.java
+++ b/src/java/org/apache/cassandra/index/sasi/utils/trie/PatriciaTrie.java
@@ -1258,4 +1258,4 @@ public class PatriciaTrie<K, V> extends AbstractPatriciaTrie<K, V> implements Se
             }
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/io/FSError.java b/src/java/org/apache/cassandra/io/FSError.java
index e09bac7..6531394 100644
--- a/src/java/org/apache/cassandra/io/FSError.java
+++ b/src/java/org/apache/cassandra/io/FSError.java
@@ -17,17 +17,25 @@
  */
 package org.apache.cassandra.io;
 
-import java.io.File;
 import java.io.IOError;
+import java.nio.file.Path;
+
+import org.apache.cassandra.io.util.File;
 
 public abstract class FSError extends IOError
 {
-    public final File path;
+    public final String path;
 
     public FSError(Throwable cause, File path)
     {
         super(cause);
-        this.path = path;
+        this.path = path.toString();
+    }
+
+    public FSError(Throwable cause, Path path)
+    {
+        super(cause);
+        this.path = path.toString();
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/io/FSReadError.java b/src/java/org/apache/cassandra/io/FSReadError.java
index c557fc5..688182a 100644
--- a/src/java/org/apache/cassandra/io/FSReadError.java
+++ b/src/java/org/apache/cassandra/io/FSReadError.java
@@ -17,10 +17,18 @@
  */
 package org.apache.cassandra.io;
 
-import java.io.File;
+
+import java.nio.file.Path;
+
+import org.apache.cassandra.io.util.File;
 
 public class FSReadError extends FSError
 {
+    public FSReadError(Throwable cause, Path path)
+    {
+        super(cause, path);
+    }
+
     public FSReadError(Throwable cause, File path)
     {
         super(cause, path);
diff --git a/src/java/org/apache/cassandra/io/FSWriteError.java b/src/java/org/apache/cassandra/io/FSWriteError.java
index b419086..481795d 100644
--- a/src/java/org/apache/cassandra/io/FSWriteError.java
+++ b/src/java/org/apache/cassandra/io/FSWriteError.java
@@ -17,10 +17,18 @@
  */
 package org.apache.cassandra.io;
 
-import java.io.File;
+
+import java.nio.file.Path;
+
+import org.apache.cassandra.io.util.File;
 
 public class FSWriteError extends FSError
 {
+    public FSWriteError(Throwable cause, Path path)
+    {
+        super(cause, path);
+    }
+
     public FSWriteError(Throwable cause, File path)
     {
         super(cause, path);
diff --git a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 2190824..8321345 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.io.compress;
 
 import java.io.DataOutputStream;
 import java.io.EOFException;
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
@@ -31,6 +30,7 @@ import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.CompressionParams;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
diff --git a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
index e472ca8..cc6ce6b 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
@@ -18,16 +18,15 @@
 package org.apache.cassandra.io.compress;
 
 import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
 import java.nio.file.Paths;
-import java.io.BufferedOutputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.DataOutput;
-import java.io.DataOutputStream;
 import java.io.EOFException;
-import java.io.File;
+
+import org.apache.cassandra.io.util.*;
 import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.HashMap;
@@ -48,12 +47,7 @@ import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.Memory;
-import org.apache.cassandra.io.util.SafeMemory;
 import org.apache.cassandra.schema.CompressionParams;
-import org.apache.cassandra.utils.SyncUtil;
 import org.apache.cassandra.utils.concurrent.Transactional;
 import org.apache.cassandra.utils.concurrent.Ref;
 
@@ -104,7 +98,7 @@ public class CompressionMetadata
     {
         this.indexFilePath = indexFilePath;
 
-        try (DataInputStream stream = new DataInputStream(Files.newInputStream(Paths.get(indexFilePath))))
+        try (FileInputStreamPlus stream = new File(indexFilePath).newInputStream())
         {
             String compressorName = stream.readUTF();
             int optionCount = stream.readInt();
@@ -132,7 +126,7 @@ public class CompressionMetadata
             compressedFileLength = compressedLength;
             chunkOffsets = readChunkOffsets(stream);
         }
-        catch (FileNotFoundException e)
+        catch (FileNotFoundException | NoSuchFileException e)
         {
             throw new RuntimeException(e);
         }
@@ -410,17 +404,16 @@ public class CompressionMetadata
             }
 
             // flush the data to disk
-            try (FileOutputStream fos = new FileOutputStream(filePath);
-                 DataOutputStream out = new DataOutputStream(new BufferedOutputStream(fos)))
+            try (FileOutputStreamPlus out = new FileOutputStreamPlus(filePath))
             {
                 writeHeader(out, dataLength, count);
                 for (int i = 0; i < count; i++)
                     out.writeLong(offsets.getLong(i * 8L));
 
                 out.flush();
-                SyncUtil.sync(fos);
+                out.sync();
             }
-            catch (FileNotFoundException fnfe)
+            catch (FileNotFoundException | NoSuchFileException fnfe)
             {
                 throw Throwables.propagate(fnfe);
             }
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 4eaf1fe..0e086e6 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -17,8 +17,7 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.File;
-import java.io.FileFilter;
+
 import java.io.IOException;
 import java.io.Closeable;
 import java.nio.ByteBuffer;
@@ -31,6 +30,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.EncodingStats;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.service.ActiveRepairService;
 
@@ -91,19 +91,15 @@ abstract class AbstractSSTableSimpleWriter implements Closeable
     private static int getNextGeneration(File directory, final String columnFamily)
     {
         final Set<Descriptor> existing = new HashSet<>();
-        directory.listFiles(new FileFilter()
-        {
-            public boolean accept(File file)
-            {
-                Descriptor desc = SSTable.tryDescriptorFromFilename(file);
-                if (desc == null)
-                    return false;
+        directory.tryList(file -> {
+            Descriptor desc = SSTable.tryDescriptorFromFilename(file);
+            if (desc == null)
+                return false;
 
-                if (desc.cfname.equals(columnFamily))
-                    existing.add(desc);
+            if (desc.cfname.equals(columnFamily))
+                existing.add(desc);
 
-                return false;
-            }
+            return false;
         });
         int maxGen = generation.getAndIncrement();
         for (Descriptor desc : existing)
diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index fa8a60f..6c07032 100644
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.io.sstable;
 
 import java.io.Closeable;
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -49,6 +48,7 @@ import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.transport.ProtocolVersion;
@@ -379,7 +379,7 @@ public class CQLSSTableWriter implements Closeable
         {
             if (!directory.exists())
                 throw new IllegalArgumentException(directory + " doesn't exists");
-            if (!directory.canWrite())
+            if (!directory.isWritable())
                 throw new IllegalArgumentException(directory + " exists but is not writable");
 
             this.directory = directory;
diff --git a/src/java/org/apache/cassandra/io/sstable/CorruptSSTableException.java b/src/java/org/apache/cassandra/io/sstable/CorruptSSTableException.java
index 93be2ee..991a91d 100644
--- a/src/java/org/apache/cassandra/io/sstable/CorruptSSTableException.java
+++ b/src/java/org/apache/cassandra/io/sstable/CorruptSSTableException.java
@@ -17,7 +17,8 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.File;
+
+import org.apache.cassandra.io.util.File;
 
 public class CorruptSSTableException extends RuntimeException
 {
diff --git a/src/java/org/apache/cassandra/io/sstable/Descriptor.java b/src/java/org/apache/cassandra/io/sstable/Descriptor.java
index b781ebf..faca487 100644
--- a/src/java/org/apache/cassandra/io/sstable/Descriptor.java
+++ b/src/java/org/apache/cassandra/io/sstable/Descriptor.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.File;
 import java.io.IOError;
 import java.io.IOException;
 import java.util.*;
@@ -32,6 +31,7 @@ import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.sstable.metadata.IMetadataSerializer;
 import org.apache.cassandra.io.sstable.metadata.MetadataSerializer;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.UUIDGen;
 
@@ -90,14 +90,7 @@ public class Descriptor
     {
         assert version != null && directory != null && ksname != null && cfname != null && formatType.info.getLatestVersion().getClass().equals(version.getClass());
         this.version = version;
-        try
-        {
-            this.directory = directory.getCanonicalFile();
-        }
-        catch (IOException e)
-        {
-            throw new IOError(e);
-        }
+        this.directory = directory.toCanonical();
         this.ksname = ksname;
         this.cfname = cfname;
         this.generation = generation;
@@ -139,7 +132,7 @@ public class Descriptor
     public String baseFilename()
     {
         StringBuilder buff = new StringBuilder();
-        buff.append(directory).append(File.separatorChar);
+        buff.append(directory).append(File.pathSeparator());
         appendFileName(buff);
         return buff.toString();
     }
@@ -156,7 +149,7 @@ public class Descriptor
         final StringBuilder buff = new StringBuilder();
         if (Directories.isSecondaryIndexFolder(directory))
         {
-            buff.append(directory.getName()).append(File.separator);
+            buff.append(directory.name()).append(File.pathSeparator());
         }
 
         appendFileName(buff);
@@ -172,7 +165,7 @@ public class Descriptor
     /** Return any temporary files found in the directory */
     public List<File> getTemporaryFiles()
     {
-        File[] tmpFiles = directory.listFiles((dir, name) ->
+        File[] tmpFiles = directory.tryList((dir, name) ->
                                               name.endsWith(Descriptor.TMP_EXT));
 
         List<File> ret = new ArrayList<>(tmpFiles.length);
@@ -184,7 +177,7 @@ public class Descriptor
 
     public static boolean isValidFile(File file)
     {
-        String filename = file.getName();
+        String filename = file.name();
         return filename.endsWith(".db") && !LEGACY_TMP_REGEX.matcher(filename).matches();
     }
 
@@ -242,9 +235,9 @@ public class Descriptor
         // We need to extract the keyspace and table names from the parent directories, so make sure we deal with the
         // absolute path.
         if (!file.isAbsolute())
-            file = file.getAbsoluteFile();
+            file = file.toAbsolute();
 
-        String name = file.getName();
+        String name = file.name();
         List<String> tokens = filenameSplitter.splitToList(name);
         int size = tokens.size();
 
@@ -300,25 +293,25 @@ public class Descriptor
         String indexName = "";
         if (Directories.isSecondaryIndexFolder(tableDir))
         {
-            indexName = tableDir.getName();
+            indexName = tableDir.name();
             tableDir = parentOf(name, tableDir);
         }
 
         // Then it can be a backup or a snapshot
-        if (tableDir.getName().equals(Directories.BACKUPS_SUBDIR))
-            tableDir = tableDir.getParentFile();
-        else if (parentOf(name, tableDir).getName().equals(Directories.SNAPSHOT_SUBDIR))
+        if (tableDir.name().equals(Directories.BACKUPS_SUBDIR))
+            tableDir = tableDir.parent();
+        else if (parentOf(name, tableDir).name().equals(Directories.SNAPSHOT_SUBDIR))
             tableDir = parentOf(name, parentOf(name, tableDir));
 
-        String table = tableDir.getName().split("-")[0] + indexName;
-        String keyspace = parentOf(name, tableDir).getName();
+        String table = tableDir.name().split("-")[0] + indexName;
+        String keyspace = parentOf(name, tableDir).name();
 
         return Pair.create(new Descriptor(version, directory, keyspace, table, generation, format), component);
     }
 
     private static File parentOf(String name, File file)
     {
-        File parent = file.getParentFile();
+        File parent = file.parent();
         if (parent == null)
             throw invalidSSTable(name, "cannot extract keyspace and table name; make sure the sstable is in the proper sub-directories");
         return parent;
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManagerMBean.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManagerMBean.java
index 3382350..ad79185 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManagerMBean.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManagerMBean.java
@@ -42,4 +42,4 @@ public interface IndexSummaryManagerMBean
 
     public int getResizeIntervalInMinutes();
     public void setResizeIntervalInMinutes(int resizeIntervalInMinutes);
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/KeyIterator.java b/src/java/org/apache/cassandra/io/sstable/KeyIterator.java
index 1a5792c..ceacf87 100644
--- a/src/java/org/apache/cassandra/io/sstable/KeyIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/KeyIterator.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -26,6 +25,7 @@ import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.AbstractIterator;
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTable.java b/src/java/org/apache/cassandra/io/sstable/SSTable.java
index 0471be3..b56a286 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -17,9 +17,14 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.*;
+
+import java.io.FileNotFoundException;
+import java.io.IOError;
+import java.io.IOException;
+import java.io.PrintWriter;
 import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
 import java.util.*;
 import java.util.concurrent.CopyOnWriteArraySet;
 
@@ -27,7 +32,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Predicates;
 import com.google.common.collect.Collections2;
 import com.google.common.collect.Sets;
-import com.google.common.io.Files;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,6 +52,7 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.memory.HeapAllocator;
 
+import static org.apache.cassandra.io.util.File.WriteMode.APPEND;
 import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
 import static org.apache.cassandra.service.ActiveRepairService.UNREPAIRED_SSTABLE;
 
@@ -233,7 +239,7 @@ public abstract class SSTable
             {
                 return readTOC(desc);
             }
-            catch (FileNotFoundException e)
+            catch (FileNotFoundException | NoSuchFileException e)
             {
                 Set<Component> components = discoverComponentsFor(desc);
                 if (components.isEmpty())
@@ -317,7 +323,7 @@ public abstract class SSTable
     protected static Set<Component> readTOC(Descriptor descriptor, boolean skipMissing) throws IOException
     {
         File tocFile = new File(descriptor.filenameFor(Component.TOC));
-        List<String> componentNames = Files.readLines(tocFile, Charset.defaultCharset());
+        List<String> componentNames = Files.readAllLines(tocFile.toPath());
         Set<Component> components = Sets.newHashSetWithExpectedSize(componentNames.size());
         for (String componentName : componentNames)
         {
@@ -336,7 +342,7 @@ public abstract class SSTable
     protected static void appendTOC(Descriptor descriptor, Collection<Component> components)
     {
         File tocFile = new File(descriptor.filenameFor(Component.TOC));
-        try (PrintWriter w = new PrintWriter(new FileWriter(tocFile, true)))
+        try (PrintWriter w = new PrintWriter(tocFile.newWriter(APPEND)))
         {
             for (Component component : components)
                 w.println(component.name);
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableHeaderFix.java b/src/java/org/apache/cassandra/io/sstable/SSTableHeaderFix.java
index 3577259..ad0a722 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableHeaderFix.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableHeaderFix.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
@@ -36,6 +35,7 @@ import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -298,7 +298,7 @@ public abstract class SSTableHeaderFix
     {
         Stream.of(path)
               .flatMap(SSTableHeaderFix::maybeExpandDirectory)
-              .filter(p -> Descriptor.fromFilenameWithComponent(p.toFile()).right.type == Component.Type.DATA)
+              .filter(p -> Descriptor.fromFilenameWithComponent(new File(p)).right.type == Component.Type.DATA)
               .map(Path::toString)
               .map(Descriptor::fromFilename)
               .forEach(descriptors::add);
@@ -888,26 +888,26 @@ public abstract class SSTableHeaderFix
         private void scanDataDirectory(Directories.DataDirectory dataDirectory)
         {
             info.accept(String.format("Scanning data directory %s", dataDirectory.location));
-            File[] ksDirs = dataDirectory.location.listFiles();
+            File[] ksDirs = dataDirectory.location.tryList();
             if (ksDirs == null)
                 return;
             for (File ksDir : ksDirs)
             {
-                if (!ksDir.isDirectory() || !ksDir.canRead())
+                if (!ksDir.isDirectory() || !ksDir.isReadable())
                     continue;
 
-                String name = ksDir.getName();
+                String name = ksDir.name();
 
                 // silently ignore all system keyspaces
                 if (SchemaConstants.isLocalSystemKeyspace(name) || SchemaConstants.isReplicatedSystemKeyspace(name))
                     continue;
 
-                File[] tabDirs = ksDir.listFiles();
+                File[] tabDirs = ksDir.tryList();
                 if (tabDirs == null)
                     continue;
                 for (File tabDir : tabDirs)
                 {
-                    if (!tabDir.isDirectory() || !tabDir.canRead())
+                    if (!tabDir.isDirectory() || !tabDir.isReadable())
                         continue;
 
                     processFileOrDirectory(tabDir.toPath());
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
index 76e12c8..2b32278 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.io.sstable;
 
 import java.io.*;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
index 47de00c..7df9258 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
@@ -17,13 +17,13 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.File;
 import java.util.*;
 
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 
 import org.apache.cassandra.db.streaming.CassandraOutgoingFile;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.schema.TableMetadataRef;
@@ -62,7 +62,7 @@ public class SSTableLoader implements StreamEventHandler
     public SSTableLoader(File directory, Client client, OutputHandler outputHandler, int connectionsPerHost, String targetKeyspace)
     {
         this.directory = directory;
-        this.keyspace = targetKeyspace != null ? targetKeyspace : directory.getParentFile().getName();
+        this.keyspace = targetKeyspace != null ? targetKeyspace : directory.parent().name();
         this.client = client;
         this.outputHandler = outputHandler;
         this.connectionsPerHost = connectionsPerHost;
@@ -76,8 +76,8 @@ public class SSTableLoader implements StreamEventHandler
         LifecycleTransaction.getFiles(directory.toPath(),
                                       (file, type) ->
                                       {
-                                          File dir = file.getParentFile();
-                                          String name = file.getName();
+                                          File dir = file.parent();
+                                          String name = file.name();
 
                                           if (type != Directories.FileType.FINAL)
                                           {
@@ -99,15 +99,15 @@ public class SSTableLoader implements StreamEventHandler
                                           TableMetadataRef metadata = client.getTableMetadata(desc.cfname);
 
                                           if (metadata == null && // we did not find metadata
-                                              directory.getName().equals(Directories.BACKUPS_SUBDIR)) // and it's likely we hit CASSANDRA-16235
+                                              directory.name().equals(Directories.BACKUPS_SUBDIR)) // and it's likely we hit CASSANDRA-16235
                                           {
-                                              File parentDirectory = directory.getParentFile();
-                                              File parentParentDirectory = parentDirectory != null ? parentDirectory.getParentFile() : null;
+                                              File parentDirectory = directory.parent();
+                                              File parentParentDirectory = parentDirectory != null ? parentDirectory.parent() : null;
                                               // check that descriptor's cfname and ksname are 1 directory closer to root than they should be
                                               if (parentDirectory != null &&
                                                   parentParentDirectory != null &&
-                                                  desc.cfname.equals(parentDirectory.getName()) &&
-                                                  desc.ksname.equals(parentParentDirectory.getName()))
+                                                  desc.cfname.equals(parentDirectory.name()) &&
+                                                  desc.ksname.equals(parentParentDirectory.name()))
                                               {
                                                   Descriptor newDesc = new Descriptor(desc.directory,
                                                                                       desc.ksname,
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index 1d5aaa0..3b1ed1b 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.Map;
 import java.util.TreeMap;
@@ -35,6 +34,7 @@ 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.io.util.File;
 import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
index 530a03b..7c60454 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
@@ -17,13 +17,13 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.File;
 import java.io.IOException;
 
 import com.google.common.base.Throwables;
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.TableMetadataRef;
 
 /**
diff --git a/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java b/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java
index d23c488..4d6b209 100644
--- a/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java
+++ b/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java
@@ -18,11 +18,11 @@
 
 package org.apache.cassandra.io.sstable;
 
-import java.io.File;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java
index f0b6bac..569925e 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java
@@ -52,4 +52,4 @@ public interface SSTableFlushObserver
      * Called when all data is written to the file and it's ready to be finished up.
      */
     void complete();
-}
\ No newline at end of file
+}
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 d5dee48..1748a43 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -17,9 +17,11 @@
  */
 package org.apache.cassandra.io.sstable.format;
 
-import java.io.*;
+
+import java.io.IOException;
 import java.lang.ref.WeakReference;
 import java.nio.ByteBuffer;
+import java.nio.file.NoSuchFileException;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -30,6 +32,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.io.util.File;
 import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -621,7 +624,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
                 if (expectedComponents.contains(Component.COMPRESSION_INFO) && !actualComponents.contains(Component.COMPRESSION_INFO))
                 {
                     String compressionInfoFileName = descriptor.filenameFor(Component.COMPRESSION_INFO);
-                    throw new CorruptSSTableException(new FileNotFoundException(compressionInfoFileName), compressionInfoFileName);
+                    throw new CorruptSSTableException(new NoSuchFileException(compressionInfoFileName), compressionInfoFileName);
                 }
             }
             catch (IOException e)
@@ -727,7 +730,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         if (summariesFile.exists())
             FileUtils.deleteWithConfirm(summariesFile);
 
-        try (DataOutputStreamPlus oStream = new BufferedDataOutputStreamPlus(new FileOutputStream(summariesFile)))
+        try (DataOutputStreamPlus oStream = new FileOutputStreamPlus(summariesFile))
         {
             IndexSummary.serializer.serialize(summary, oStream);
             ByteBufferUtil.writeWithLength(first.getKey(), oStream);
@@ -746,7 +749,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
     public static void saveBloomFilter(Descriptor descriptor, IFilter filter)
     {
         File filterFile = new File(descriptor.filenameFor(Component.FILTER));
-        try (DataOutputStreamPlus stream = new BufferedDataOutputStreamPlus(new FileOutputStream(filterFile)))
+        try (DataOutputStreamPlus stream = new FileOutputStreamPlus(filterFile))
         {
             BloomFilterSerializer.serialize((BloomFilter) filter, stream);
             stream.flush();
@@ -1628,7 +1631,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
                 continue;
             if (null != limiter)
                 limiter.acquire();
-            File targetLink = new File(snapshotDirectoryPath, sourceFile.getName());
+            File targetLink = new File(snapshotDirectoryPath, sourceFile.name());
             FileUtils.createHardLink(sourceFile, targetLink);
         }
     }
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java
index e8266f8..3386c23 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java
@@ -28,7 +28,9 @@ import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
 import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
 import org.apache.cassandra.io.util.DiskOptimizationStrategy;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.schema.TableMetadata;
@@ -39,7 +41,6 @@ import org.slf4j.LoggerFactory;
 
 import java.io.BufferedInputStream;
 import java.io.DataInputStream;
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.file.Files;
@@ -129,7 +130,7 @@ public abstract class SSTableReaderBuilder
         if (!summariesFile.exists())
         {
             if (logger.isDebugEnabled())
-                logger.debug("SSTable Summary File {} does not exist", summariesFile.getAbsolutePath());
+                logger.debug("SSTable Summary File {} does not exist", summariesFile.absolutePath());
             return;
         }
 
@@ -148,7 +149,7 @@ public abstract class SSTableReaderBuilder
         {
             if (summary != null)
                 summary.close();
-            logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
+            logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.path(), e.getMessage());
             // corrupted; delete it and fall back to creating a new summary
             FileUtils.closeQuietly(iStream);
             // delete it and fall back to creating a new summary
@@ -237,7 +238,7 @@ public abstract class SSTableReaderBuilder
      */
     IFilter loadBloomFilter() throws IOException
     {
-        try (DataInputStream stream = new DataInputStream(new BufferedInputStream(Files.newInputStream(Paths.get(descriptor.filenameFor(Component.FILTER))))))
+        try (FileInputStreamPlus stream = new File(descriptor.filenameFor(Component.FILTER)).newInputStream())
         {
             return BloomFilterSerializer.deserialize(stream, descriptor.version.hasOldBfFormat());
         }
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
index 281a11d..7929059 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
@@ -17,13 +17,15 @@
  */
 package org.apache.cassandra.io.sstable.format.big;
 
-import java.io.*;
+
+import java.io.IOException;
 import java.nio.BufferOverflowException;
 import java.nio.ByteBuffer;
 import java.util.*;
 
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -470,7 +472,7 @@ public class BigTableWriter extends SSTableWriter
         }
         catch (IOException e)
         {
-            throw new FSWriteError(e, file.getPath());
+            throw new FSWriteError(e, file.path());
         }
     }
 
@@ -547,13 +549,12 @@ public class BigTableWriter extends SSTableWriter
             if (components.contains(Component.FILTER))
             {
                 String path = descriptor.filenameFor(Component.FILTER);
-                try (FileOutputStream fos = new FileOutputStream(path);
-                     DataOutputStreamPlus stream = new BufferedDataOutputStreamPlus(fos))
+                try (FileOutputStreamPlus stream = new FileOutputStreamPlus(path))
                 {
                     // bloom filter
                     BloomFilterSerializer.serialize((BloomFilter) bf, stream);
                     stream.flush();
-                    SyncUtil.sync(fos);
+                    stream.sync();
                 }
                 catch (IOException e)
                 {
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java
index f05ea94..717e9d9 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.io.sstable.format.big;
 
 import java.io.EOFException;
-import java.io.File;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.EnumMap;
@@ -28,6 +27,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
 import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -227,4 +227,4 @@ public class BigTableZeroCopyWriter extends SSTable implements SSTableMultiWrite
             throw new FSWriteError(e, writer.getPath());
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
old mode 100755
new mode 100644
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
index 042103e..91889a7 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
@@ -17,7 +17,10 @@
  */
 package org.apache.cassandra.io.sstable.metadata;
 
-import java.io.*;
+import org.apache.cassandra.io.util.*;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
 import java.util.*;
 import java.util.function.UnaryOperator;
 import java.util.zip.CRC32;
@@ -25,6 +28,7 @@ import java.util.zip.CRC32;
 import com.google.common.base.Throwables;
 import com.google.common.collect.Lists;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -33,14 +37,6 @@ import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.format.Version;
-import org.apache.cassandra.io.util.DataInputBuffer;
-import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
-import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt;
@@ -263,7 +259,7 @@ public class MetadataSerializer implements IMetadataSerializer
     public void rewriteSSTableMetadata(Descriptor descriptor, Map<MetadataType, MetadataComponent> currentComponents) throws IOException
     {
         String filePath = descriptor.tmpFilenameFor(Component.STATS);
-        try (DataOutputStreamPlus out = new BufferedDataOutputStreamPlus(new FileOutputStream(filePath)))
+        try (DataOutputStreamPlus out = new FileOutputStreamPlus(filePath))
         {
             serialize(currentComponents, out, descriptor.version);
             out.flush();
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
old mode 100755
new mode 100644
diff --git a/src/java/org/apache/cassandra/io/util/AbstractReaderFileProxy.java b/src/java/org/apache/cassandra/io/util/AbstractReaderFileProxy.java
index 7962c0f..a8b61dd 100644
--- a/src/java/org/apache/cassandra/io/util/AbstractReaderFileProxy.java
+++ b/src/java/org/apache/cassandra/io/util/AbstractReaderFileProxy.java
@@ -58,4 +58,4 @@ public abstract class AbstractReaderFileProxy implements ReaderFileProxy
     {
         return 0; // Only valid for compressed files.
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java
index 516508e..a490ff6 100644
--- a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java
+++ b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java
@@ -17,9 +17,7 @@
  */
 package org.apache.cassandra.io.util;
 
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.nio.channels.WritableByteChannel;
@@ -42,26 +40,6 @@ public class BufferedDataOutputStreamPlus extends DataOutputStreamPlus
 
     protected ByteBuffer buffer;
 
-    public BufferedDataOutputStreamPlus(RandomAccessFile ras)
-    {
-        this(ras.getChannel());
-    }
-
-    public BufferedDataOutputStreamPlus(RandomAccessFile ras, int bufferSize)
-    {
-        this(ras.getChannel(), bufferSize);
-    }
-
-    public BufferedDataOutputStreamPlus(FileOutputStream fos)
-    {
-        this(fos.getChannel());
-    }
-
-    public BufferedDataOutputStreamPlus(FileOutputStream fos, int bufferSize)
-    {
-        this(fos.getChannel(), bufferSize);
-    }
-
     public BufferedDataOutputStreamPlus(WritableByteChannel wbc)
     {
         this(wbc, DEFAULT_BUFFER_SIZE);
diff --git a/src/java/org/apache/cassandra/io/util/ChannelProxy.java b/src/java/org/apache/cassandra/io/util/ChannelProxy.java
index 9ff46b7..717def7 100644
--- a/src/java/org/apache/cassandra/io/util/ChannelProxy.java
+++ b/src/java/org/apache/cassandra/io/util/ChannelProxy.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.io.util;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.MappedByteBuffer;
@@ -63,7 +62,7 @@ public final class ChannelProxy extends SharedCloseableImpl
 
     public ChannelProxy(File file)
     {
-        this(file.getPath(), openChannel(file));
+        this(file.path(), openChannel(file));
     }
 
     public ChannelProxy(String filePath, FileChannel channel)
diff --git a/src/java/org/apache/cassandra/io/util/ChecksumWriter.java b/src/java/org/apache/cassandra/io/util/ChecksumWriter.java
index dc5eaea..d8ba360 100644
--- a/src/java/org/apache/cassandra/io/util/ChecksumWriter.java
+++ b/src/java/org/apache/cassandra/io/util/ChecksumWriter.java
@@ -29,6 +29,8 @@ import com.google.common.base.Charsets;
 
 import org.apache.cassandra.io.FSWriteError;
 
+import org.apache.cassandra.io.util.File;
+
 public class ChecksumWriter
 {
     private final CRC32 incrementalChecksum = new CRC32();
diff --git a/src/java/org/apache/cassandra/io/util/ChecksummedRandomAccessReader.java b/src/java/org/apache/cassandra/io/util/ChecksummedRandomAccessReader.java
index 2e59e3b..62927f5 100644
--- a/src/java/org/apache/cassandra/io/util/ChecksummedRandomAccessReader.java
+++ b/src/java/org/apache/cassandra/io/util/ChecksummedRandomAccessReader.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.io.util;
 
-import java.io.File;
 import java.io.IOException;
 
 import org.apache.cassandra.utils.ChecksumType;
@@ -32,7 +31,7 @@ public final class ChecksummedRandomAccessReader
         {
             DataIntegrityMetadata.ChecksumValidator validator = new DataIntegrityMetadata.ChecksumValidator(ChecksumType.CRC32,
                                                                                                             RandomAccessReader.open(crcFile),
-                                                                                                            file.getPath());
+                                                                                                            file.path());
             Rebufferer rebufferer = new ChecksummedRebufferer(channel, validator);
             return new RandomAccessReader.RandomAccessReaderWithOwnChannel(rebufferer);
         }
diff --git a/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java b/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java
index f89e7cc..1477c5c 100644
--- a/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java
@@ -17,10 +17,11 @@
  */
 package org.apache.cassandra.io.util;
 
-import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.Optional;
 
+import org.apache.cassandra.io.util.File;
+
 public class ChecksummedSequentialWriter extends SequentialWriter
 {
     private static final SequentialWriterOption CRC_WRITER_OPTION = SequentialWriterOption.newBuilder()
diff --git a/src/java/org/apache/cassandra/io/util/ChunkReader.java b/src/java/org/apache/cassandra/io/util/ChunkReader.java
index 1d3439e..33bf792 100644
--- a/src/java/org/apache/cassandra/io/util/ChunkReader.java
+++ b/src/java/org/apache/cassandra/io/util/ChunkReader.java
@@ -48,4 +48,4 @@ public interface ChunkReader extends RebuffererFactory
      * This is not guaranteed to be fulfilled.
      */
     BufferType preferredBufferType();
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/io/util/DataInputPlus.java b/src/java/org/apache/cassandra/io/util/DataInputPlus.java
index 7c29ee1..41b422a 100644
--- a/src/java/org/apache/cassandra/io/util/DataInputPlus.java
+++ b/src/java/org/apache/cassandra/io/util/DataInputPlus.java
@@ -60,6 +60,8 @@ public interface DataInputPlus extends DataInput
 
     /**
      * Wrapper around an InputStream that provides no buffering but can decode varints
+     *
+     * TODO: probably shouldn't use DataInputStream as a parent
      */
     public class DataInputStreamPlus extends DataInputStream implements DataInputPlus
     {
diff --git a/src/java/org/apache/cassandra/io/util/DataIntegrityMetadata.java b/src/java/org/apache/cassandra/io/util/DataIntegrityMetadata.java
index 277b359..f611f91 100644
--- a/src/java/org/apache/cassandra/io/util/DataIntegrityMetadata.java
+++ b/src/java/org/apache/cassandra/io/util/DataIntegrityMetadata.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.io.util;
 
 import java.io.Closeable;
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.zip.CheckedInputStream;
@@ -28,6 +27,7 @@ import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.utils.ChecksumType;
 import org.apache.cassandra.utils.Throwables;
 
diff --git a/src/java/org/apache/cassandra/io/util/File.java b/src/java/org/apache/cassandra/io/util/File.java
new file mode 100644
index 0000000..2b60904
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/File.java
@@ -0,0 +1,608 @@
+/*
+ * 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.io.util;
+
+import java.io.IOException;
+import java.net.URI;
+import java.nio.channels.FileChannel;
+import java.nio.file.*;
+import java.util.Objects;
+import java.util.function.BiPredicate;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.IntFunction;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+
+import javax.annotation.Nullable;
+
+import com.google.common.util.concurrent.RateLimiter;
+
+import org.apache.cassandra.io.FSWriteError;
+
+import static org.apache.cassandra.io.util.PathUtils.filename;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+
+/**
+ * A thin wrapper around java.nio.file.Path to provide more ergonomic functionality.
+ *
+ * TODO codebase probably should not use tryList, as unexpected exceptions are hidden;
+ *      probably want to introduce e.g. listIfExists
+ * TODO codebase probably should not use Paths.get() to ensure we can override the filesystem
+ */
+public class File implements Comparable<File>
+{
+    private static final FileSystem filesystem = FileSystems.getDefault();
+    public enum WriteMode { OVERWRITE, APPEND }
+
+    public static String pathSeparator()
+    {
+        return filesystem.getSeparator();
+    }
+
+    @Nullable final Path path; // nullable to support concept of empty path, that resolves to the working directory if converted to an absolute path
+
+    /**
+     * Construct a File representing the child {@code child} of {@code parent}
+     */
+    public File(String parent, String child)
+    {
+        this(parent.isEmpty() ? null : filesystem.getPath(parent), child);
+    }
+
+    /**
+     * Construct a File representing the child {@code child} of {@code parent}
+     */
+    public File(File parent, String child)
+    {
+        this(parent.path, child);
+    }
+
+    /**
+     * Construct a File representing the child {@code child} of {@code parent}
+     */
+    public File(Path parent, String child)
+    {
+        // if "empty abstract path" (a la java.io.File) is provided, we should behave as though resolving relative path
+        if (child.startsWith(pathSeparator()))
+            child = child.substring(pathSeparator().length());
+        this.path = parent == null ? filesystem.getPath(child) : parent.resolve(child);
+    }
+
+    /**
+     * Construct a File representing the provided {@code path}
+     */
+    public File(String path)
+    {
+        this(path.isEmpty() ? null : filesystem.getPath(path));
+    }
+
+    /**
+     * Create a File equivalent to the java.io.File provided
+     */
+    public File(java.io.File file)
+    {
+        this(file.getPath().isEmpty() ? null : file.toPath());
+    }
+
+    /**
+     * Construct a File representing the child {@code child} of {@code parent}
+     */
+    public File(java.io.File parent, String child)
+    {
+        this(new File(parent), child);
+    }
+
+    /**
+     * Convenience constructor equivalent to {@code new File(Paths.get(path))}
+     */
+    public File(URI path)
+    {
+        this(Paths.get(path));
+        if (!path.isAbsolute() || path.isOpaque()) throw new IllegalArgumentException();
+    }
+
+    /**
+     * @param path the path to wrap
+     */
+    public File(Path path)
+    {
+        if (path != null && path.getFileSystem() != filesystem)
+            throw new IllegalArgumentException("Incompatible file system");
+
+        this.path = path;
+    }
+
+    /**
+     * Try to delete the file, returning true iff it was deleted by us. Does not ordinarily throw exceptions.
+     */
+    public boolean tryDelete()
+    {
+        return path != null && PathUtils.tryDelete(path);
+    }
+
+    /**
+     * This file will be deleted, and any exceptions encountered merged with {@code accumulate} to the return value
+     */
+    public Throwable delete(Throwable accumulate)
+    {
+        return delete(accumulate, null);
+    }
+
+    /**
+     * This file will be deleted, obeying the provided rate limiter.
+     * Any exceptions encountered will be merged with {@code accumulate} to the return value
+     */
+    public Throwable delete(Throwable accumulate, RateLimiter rateLimiter)
+    {
+        return PathUtils.delete(toPathForWrite(), accumulate, rateLimiter);
+    }
+
+    /**
+     * This file will be deleted, with any failures being reported with an FSError
+     * @throws FSWriteError if cannot be deleted
+     */
+    public void delete()
+    {
+        maybeFail(delete(null, null));
+    }
+
+    /**
+     * This file will be deleted, obeying the provided rate limiter.
+     * @throws FSWriteError if cannot be deleted
+     */
+    public void delete(RateLimiter rateLimiter)
+    {
+        maybeFail(delete(null, rateLimiter));
+    }
+
+    /**
+     * Deletes all files and subdirectories under "dir".
+     * @throws FSWriteError if any part of the tree cannot be deleted
+     */
+    public void deleteRecursive(RateLimiter rateLimiter)
+    {
+        PathUtils.deleteRecursive(toPathForWrite(), rateLimiter);
+    }
+
+    /**
+     * Deletes all files and subdirectories under "dir".
+     * @throws FSWriteError if any part of the tree cannot be deleted
+     */
+    public void deleteRecursive()
+    {
+        PathUtils.deleteRecursive(toPathForWrite());
+    }
+
+    /**
+     * Try to delete the file on process exit.
+     */
+    public void deleteOnExit()
+    {
+        if (path != null) PathUtils.deleteOnExit(path);
+    }
+
+    /**
+     * This file will be deleted on clean shutdown; if it is a directory, its entire contents
+     * <i>at the time of shutdown</i> will be deleted
+     */
+    public void deleteRecursiveOnExit()
+    {
+        if (path != null)
+            PathUtils.deleteRecursiveOnExit(path);
+    }
+
+    /**
+     * Try to rename the file atomically, if the system supports it.
+     * @return true iff successful, false if it fails for any reason.
+     */
+    public boolean tryMove(File to)
+    {
+        return path != null && PathUtils.tryRename(path, to.path);
+    }
+
+    /**
+     * Atomically (if supported) rename/move this file to {@code to}
+     * @throws FSWriteError if any part of the tree cannot be deleted
+     */
+    public void move(File to)
+    {
+        PathUtils.rename(toPathForRead(), to.toPathForWrite());
+    }
+
+    /**
+     * @return the length of the file if it exists and if we can read it; 0 otherwise.
+     */
+    public long length()
+    {
+        return path == null ? 0L : PathUtils.tryGetLength(path);
+    }
+
+    /**
+     * @return the last modified time in millis of the path if it exists and we can read it; 0 otherwise.
+     */
+    public long lastModified()
+    {
+        return path == null ? 0L : PathUtils.tryGetLastModified(path);
+    }
+
+    /**
+     * Try to set the last modified time in millis of the path
+     * @return true if it exists and we can write it; return false otherwise.
+     */
+    public boolean trySetLastModified(long value)
+    {
+        return path != null && PathUtils.trySetLastModified(path, value);
+    }
+
+    /**
+     * Try to set if the path is readable by its owner
+     * @return true if it exists and we can write it; return false otherwise.
+     */
+    public boolean trySetReadable(boolean value)
+    {
+        return path != null && PathUtils.trySetReadable(path, value);
+    }
+
+    /**
+     * Try to set if the path is writable by its owner
+     * @return true if it exists and we can write it; return false otherwise.
+     */
+    public boolean trySetWritable(boolean value)
+    {
+        return path != null && PathUtils.trySetWritable(path, value);
+    }
+
+    /**
+     * Try to set if the path is executable by its owner
+     * @return true if it exists and we can write it; return false otherwise.
+     */
+    public boolean trySetExecutable(boolean value)
+    {
+        return path != null && PathUtils.trySetExecutable(path, value);
+    }
+
+    /**
+     * @return true if the path exists, false if it does not, or we cannot determine due to some exception
+     */
+    public boolean exists()
+    {
+        return path != null && PathUtils.exists(path);
+    }
+
+    /**
+     * @return true if the path refers to a directory
+     */
+    public boolean isDirectory()
+    {
+        return path != null && PathUtils.isDirectory(path);
+    }
+
+    /**
+     * @return true if the path refers to a regular file
+     */
+    public boolean isFile()
+    {
+        return path != null && PathUtils.isFile(path);
+    }
+
+    /**
+     * @return true if the path can be read by us
+     */
+    public boolean isReadable()
+    {
+        return path != null && Files.isReadable(path);
+    }
+
+    /**
+     * @return true if the path can be written by us
+     */
+    public boolean isWritable()
+    {
+        return path != null && Files.isWritable(path);
+    }
+
+    /**
+     * @return true if the path can be executed by us
+     */
+    public boolean isExecutable()
+    {
+        return path != null && Files.isExecutable(path);
+    }
+
+    /**
+     * Try to create a new regular file at this path.
+     * @return true if successful, false if it already exists
+     */
+    public boolean createFileIfNotExists()
+    {
+        return PathUtils.createFileIfNotExists(toPathForWrite());
+    }
+
+    /**
+     * Try to create a directory at this path.
+     * Return true if a new directory was created at this path, and false otherwise.
+     */
+    public boolean tryCreateDirectory()
+    {
+        return path != null && PathUtils.tryCreateDirectory(path);
+    }
+
+    /**
+     * Try to create a directory at this path, creating any parent directories as necessary.
+     * @return true if a new directory was created at this path, and false otherwise.
+     */
+    public boolean tryCreateDirectories()
+    {
+        return path != null && PathUtils.tryCreateDirectories(path);
+    }
+
+    /**
+     * @return the parent file, or null if none
+     */
+    public File parent()
+    {
+        if (path == null) return null;
+        Path parent = path.getParent();
+        if (parent == null) return null;
+        return new File(parent);
+    }
+
+    /**
+     * @return the parent file's path, or null if none
+     */
+    public String parentPath()
+    {
+        File parent = parent();
+        return parent == null ? null : parent.toString();
+    }
+
+    /**
+     * @return true if the path has no relative path elements
+     */
+    public boolean isAbsolute()
+    {
+        return path != null && path.isAbsolute();
+    }
+
+    public boolean isAncestorOf(File child)
+    {
+        return PathUtils.isContained(toPath(), child.toPath());
+    }
+
+    /**
+     * @return a File that represents the same path as this File with any relative path elements resolved.
+     *         If this is the empty File, returns the working directory.
+     */
+    public File toAbsolute()
+    {
+        return new File(toPath().toAbsolutePath());
+    }
+
+    /** {@link #toAbsolute} */
+    public String absolutePath()
+    {
+        return toPath().toAbsolutePath().toString();
+    }
+
+    /**
+     * @return a File that represents the same path as this File with any relative path elements and links resolved.
+     *         If this is the empty File, returns the working directory.
+     */
+    public File toCanonical()
+    {
+        Path canonical = PathUtils.toCanonicalPath(toPath());
+        return canonical == path ? this : new File(canonical);
+    }
+
+    /** {@link #toCanonical} */
+    public String canonicalPath()
+    {
+        return toCanonical().toString();
+    }
+
+    /**
+     * @return the last path element for this file
+     */
+    public String name()
+    {
+        return path == null ? "" : filename(path);
+    }
+
+    public void forEach(Consumer<File> forEach)
+    {
+        PathUtils.forEach(path, path -> forEach.accept(new File(path)));
+    }
+
+    public void forEachRecursive(Consumer<File> forEach)
+    {
+        PathUtils.forEachRecursive(path, path -> forEach.accept(new File(path)));
+    }
+
+    /**
+     * @return if a directory, the names of the files within; null otherwise
+     */
+    public String[] tryListNames()
+    {
+        return tryListNames(path, Function.identity());
+    }
+
+    /**
+     * @return if a directory, the names of the files within, filtered by the provided predicate; null otherwise
+     */
+    public String[] tryListNames(BiPredicate<File, String> filter)
+    {
+        return tryList(path, stream -> stream.map(PathUtils::filename).filter(filename -> filter.test(this, filename)), String[]::new);
+    }
+
+    /**
+     * @return if a directory, the files within; null otherwise
+     */
+    public File[] tryList()
+    {
+        return tryList(path, Function.identity());
+    }
+
+    /**
+     * @return if a directory, the files within, filtered by the provided predicate; null otherwise
+     */
+    public File[] tryList(Predicate<File> filter)
+    {
+        return tryList(path, stream -> stream.filter(filter));
+    }
+
+    /**
+     * @return if a directory, the files within, filtered by the provided predicate; null otherwise
+     */
+    public File[] tryList(BiPredicate<File, String> filter)
+    {
+        return tryList(path, stream -> stream.filter(file -> filter.test(this, file.name())));
+    }
+
+    private static String[] tryListNames(Path path, Function<Stream<File>, Stream<File>> toFiles)
+    {
+        if (path == null)
+            return null;
+        return PathUtils.tryList(path, stream -> toFiles.apply(stream.map(File::new)).map(File::name), String[]::new);
+    }
+
+    private static <T> T[] tryList(Path path, Function<Stream<Path>, Stream<T>> transformation, IntFunction<T[]> constructor)
+    {
+        if (path == null)
+            return null;
+        return PathUtils.tryList(path, transformation, constructor);
+    }
+
+    private static File[] tryList(Path path, Function<Stream<File>, Stream<File>> toFiles)
+    {
+        if (path == null)
+            return null;
+        return PathUtils.tryList(path, stream -> toFiles.apply(stream.map(File::new)), File[]::new);
+    }
+
+    /**
+     * @return the path of this file
+     */
+    public String path()
+    {
+        return toString();
+    }
+
+    /**
+     * @return the {@link Path} of this file
+     */
+    public Path toPath()
+    {
+        return path == null ? filesystem.getPath("") : path;
+    }
+
+    /**
+     * @return the path of this file
+     */
+    @Override
+    public String toString()
+    {
+        return path == null ? "" : path.toString();
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return path == null ? 0 : path.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj)
+    {
+        return obj instanceof File && Objects.equals(path, ((File) obj).path);
+    }
+
+    @Override
+    public int compareTo(File that)
+    {
+        if (this.path == null || that.path == null)
+            return this.path == null && that.path == null ? 0 : this.path == null ? -1 : 1;
+        return this.path.compareTo(that.path);
+    }
+
+    public java.io.File toJavaIOFile()
+    {
+        return path == null ? new java.io.File("") : path.toFile();
+    }
+
+    /**
+     * @return a new {@link FileChannel} for reading
+     */
+    public FileChannel newReadChannel() throws NoSuchFileException
+    {
+        return PathUtils.newReadChannel(toPathForRead());
+    }
+
+    /**
+     * @return a new {@link FileChannel} for reading or writing; file will be created if it doesn't exist
+     */
+    public FileChannel newReadWriteChannel() throws NoSuchFileException
+    {
+        return PathUtils.newReadWriteChannel(toPathForRead());
+    }
+
+    /**
+     * @param mode whether or not the channel appends to the underlying file
+     * @return a new {@link FileChannel} for writing; file will be created if it doesn't exist
+     */
+    public FileChannel newWriteChannel(WriteMode mode) throws NoSuchFileException
+    {
+        switch (mode)
+        {
+            default: throw new AssertionError();
+            case APPEND: return PathUtils.newWriteAppendChannel(toPathForWrite());
+            case OVERWRITE: return PathUtils.newWriteOverwriteChannel(toPathForWrite());
+        }
+    }
+
+    public FileWriter newWriter(WriteMode mode) throws IOException
+    {
+        return new FileWriter(this, mode);
+    }
+
+    public FileOutputStreamPlus newOutputStream(WriteMode mode) throws NoSuchFileException
+    {
+        return new FileOutputStreamPlus(this, mode);
+    }
+
+    public FileInputStreamPlus newInputStream() throws NoSuchFileException
+    {
+        return new FileInputStreamPlus(this);
+    }
+
+    private Path toPathForWrite()
+    {
+        if (path == null)
+            throw new IllegalStateException("Cannot write to an empty path");
+        return path;
+    }
+
+    private Path toPathForRead()
+    {
+        if (path == null)
+            throw new IllegalStateException("Cannot read from an empty path");
+        return path;
+    }
+}
+
diff --git a/src/java/org/apache/cassandra/io/util/FileInputStreamPlus.java b/src/java/org/apache/cassandra/io/util/FileInputStreamPlus.java
new file mode 100644
index 0000000..79e8438
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/FileInputStreamPlus.java
@@ -0,0 +1,95 @@
+/*
+ * 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.io.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+
+public class FileInputStreamPlus extends RebufferingInputStream
+{
+    final FileChannel channel;
+
+    public FileInputStreamPlus(String file) throws NoSuchFileException
+    {
+        this(new File(file));
+    }
+
+    public FileInputStreamPlus(File file) throws NoSuchFileException
+    {
+        this(file.newReadChannel());
+    }
+
+    public FileInputStreamPlus(Path path) throws NoSuchFileException
+    {
+        this(PathUtils.newReadChannel(path));
+    }
+
+    public FileInputStreamPlus(Path path, int bufferSize) throws NoSuchFileException
+    {
+        this(PathUtils.newReadChannel(path), bufferSize);
+    }
+
+    private FileInputStreamPlus(FileChannel channel)
+    {
+        this(channel, 1 << 14);
+    }
+
+    private FileInputStreamPlus(FileChannel channel, int bufferSize)
+    {
+        super(ByteBuffer.allocateDirect(bufferSize));
+        this.channel = channel;
+        this.buffer.limit(0);
+    }
+
+    @Override
+    protected void reBuffer() throws IOException
+    {
+        buffer.clear();
+        channel.read(buffer);
+        buffer.flip();
+    }
+
+    public FileChannel getChannel()
+    {
+        return channel;
+    }
+
+    @Override
+    public void close() throws IOException
+    {
+        try
+        {
+            super.close();
+        }
+        finally
+        {
+            try
+            {
+                FileUtils.clean(buffer);
+            }
+            finally
+            {
+                channel.close();
+            }
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/util/FileOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/FileOutputStreamPlus.java
new file mode 100644
index 0000000..0cfd3e3
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/FileOutputStreamPlus.java
@@ -0,0 +1,69 @@
+/*
+ * 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.io.util;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+
+import static org.apache.cassandra.io.util.File.WriteMode.OVERWRITE;
+
+public class FileOutputStreamPlus extends BufferedDataOutputStreamPlus
+{
+    public FileOutputStreamPlus(String path) throws NoSuchFileException
+    {
+        this(path, OVERWRITE);
+    }
+
+    public FileOutputStreamPlus(String path, File.WriteMode mode) throws NoSuchFileException
+    {
+        this(new File(path), mode);
+    }
+
+    public FileOutputStreamPlus(File file) throws NoSuchFileException
+    {
+        this(file, OVERWRITE);
+    }
+
+    public FileOutputStreamPlus(File file, File.WriteMode mode) throws NoSuchFileException
+    {
+        super(file.newWriteChannel(mode));
+    }
+
+    public FileOutputStreamPlus(Path path) throws NoSuchFileException
+    {
+        this(path, OVERWRITE);
+    }
+
+    public FileOutputStreamPlus(Path path, File.WriteMode mode) throws NoSuchFileException
+    {
+        this(new File(path), mode);
+    }
+
+    public void sync() throws IOException
+    {
+        ((FileChannel)channel).force(true);
+    }
+
+    public FileChannel getChannel()
+    {
+        return (FileChannel) channel;
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/util/RewindableDataInput.java b/src/java/org/apache/cassandra/io/util/FileReader.java
similarity index 64%
copy from src/java/org/apache/cassandra/io/util/RewindableDataInput.java
copy to src/java/org/apache/cassandra/io/util/FileReader.java
index c202f60..55b8fbb 100644
--- a/src/java/org/apache/cassandra/io/util/RewindableDataInput.java
+++ b/src/java/org/apache/cassandra/io/util/FileReader.java
@@ -19,12 +19,20 @@
 package org.apache.cassandra.io.util;
 
 import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
 
-public interface RewindableDataInput extends DataInputPlus
+public class FileReader extends InputStreamReader
 {
-    DataPosition mark();
+    @SuppressWarnings("resource") // FISP is closed by ISR::close
+    public FileReader(String file) throws IOException
+    {
+        super(new FileInputStreamPlus(file));
+    }
 
-    void reset(DataPosition mark) throws IOException;
-
-    long bytesPastMark(DataPosition mark);
-}
\ No newline at end of file
+    @SuppressWarnings("resource") // FISP is closed by ISR::close
+    public FileReader(File file) throws IOException
+    {
+        super(new FileInputStreamPlus(file));
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index 7798bd7..45abd7b 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -17,7 +17,10 @@
  */
 package org.apache.cassandra.io.util;
 
-import java.io.*;
+import java.io.BufferedWriter;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
 import java.lang.invoke.MethodHandle;
 import java.lang.invoke.MethodHandles;
 import java.lang.reflect.Method;
@@ -27,18 +30,23 @@ import java.nio.channels.ClosedChannelException;
 import java.nio.channels.FileChannel;
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
-import java.nio.file.*;
+import java.nio.file.DirectoryNotEmptyException;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.StandardOpenOption;
 import java.nio.file.attribute.BasicFileAttributes;
-import java.nio.file.attribute.FileAttributeView;
-import java.nio.file.attribute.FileStoreAttributeView;
 import java.text.DecimalFormat;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashSet;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.stream.Collectors;
@@ -50,28 +58,22 @@ import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.FSErrorHandler;
-import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.NoSpamLogger;
 import org.apache.cassandra.utils.SyncUtil;
 
 import static com.google.common.base.Throwables.propagate;
 import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_IO_TMPDIR;
 import static org.apache.cassandra.utils.Throwables.maybeFail;
-import static org.apache.cassandra.utils.Throwables.merge;
 
 public final class FileUtils
 {
     public static final Charset CHARSET = StandardCharsets.UTF_8;
 
     private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
-    private static final NoSpamLogger nospam1m = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
 
     public static final long ONE_KB = 1024;
     public static final long ONE_MB = 1024 * ONE_KB;
@@ -81,9 +83,9 @@ public final class FileUtils
     private static final DecimalFormat df = new DecimalFormat("#.##");
     private static final AtomicReference<Optional<FSErrorHandler>> fsErrorHandler = new AtomicReference<>(Optional.empty());
 
-    private static Class clsDirectBuffer;
-    private static MethodHandle mhDirectBufferCleaner;
-    private static MethodHandle mhCleanerClean;
+    private static final Class clsDirectBuffer;
+    private static final MethodHandle mhDirectBufferCleaner;
+    private static final MethodHandle mhCleanerClean;
 
     static
     {
@@ -123,39 +125,32 @@ public final class FileUtils
     }
 
     /**
-     * Pretty much like {@link File#createTempFile(String, String, File)}, but with
+     * Pretty much like {@link java.io.File#createTempFile(String, String, java.io.File)}, but with
      * the guarantee that the "random" part of the generated file name between
      * {@code prefix} and {@code suffix} is a positive, increasing {@code long} value.
      */
     public static File createTempFile(String prefix, String suffix, File directory)
     {
-        try
-        {
-            // Do not use java.io.File.createTempFile(), because some tests rely on the
-            // behavior that the "random" part in the temp file name is a positive 'long'.
-            // However, at least since Java 9 the code to generate the "random" part
-            // uses an _unsigned_ random long generated like this:
-            // Long.toUnsignedString(new java.util.Random.nextLong())
+        // Do not use java.io.File.createTempFile(), because some tests rely on the
+        // behavior that the "random" part in the temp file name is a positive 'long'.
+        // However, at least since Java 9 the code to generate the "random" part
+        // uses an _unsigned_ random long generated like this:
+        // Long.toUnsignedString(new java.util.Random.nextLong())
 
-            while (true)
-            {
-                // The contract of File.createTempFile() says, that it must not return
-                // the same file name again. We do that here in a very simple way,
-                // that probably doesn't cover all edge cases. Just rely on system
-                // wall clock and return strictly increasing values from that.
-                long num = tempFileNum.getAndIncrement();
-
-                // We have a positive long here, which is safe to use for example
-                // for CommitLogTest.
-                String fileName = prefix + Long.toString(num) + suffix;
-                File candidate = new File(directory, fileName);
-                if (candidate.createNewFile())
-                    return candidate;
-            }
-        }
-        catch (IOException e)
+        while (true)
         {
-            throw new FSWriteError(e, directory);
+            // The contract of File.createTempFile() says, that it must not return
+            // the same file name again. We do that here in a very simple way,
+            // that probably doesn't cover all edge cases. Just rely on system
+            // wall clock and return strictly increasing values from that.
+            long num = tempFileNum.getAndIncrement();
+
+            // We have a positive long here, which is safe to use for example
+            // for CommitLogTest.
+            String fileName = prefix + num + suffix;
+            File candidate = new File(directory, fileName);
+            if (candidate.createFileIfNotExists())
+                return candidate;
         }
     }
 
@@ -193,6 +188,11 @@ public final class FileUtils
         }
     }
 
+    public static void createHardLinkWithConfirm(String from, String to)
+    {
+        createHardLinkWithConfirm(new File(from), new File(to));
+    }
+
     public static void createHardLinkWithConfirm(File from, File to)
     {
         try
@@ -209,11 +209,6 @@ public final class FileUtils
         }
     }
 
-    public static void createHardLinkWithConfirm(String from, String to)
-    {
-        createHardLinkWithConfirm(new File(from), new File(to));
-    }
-
     public static void createHardLinkWithoutConfirm(String from, String to)
     {
         try
@@ -227,57 +222,6 @@ public final class FileUtils
         }
     }
 
-    public static Throwable deleteWithConfirm(String filePath, Throwable accumulate)
-    {
-        return deleteWithConfirm(new File(filePath), accumulate, null);
-    }
-
-    public static Throwable deleteWithConfirm(File file, Throwable accumulate)
-    {
-        return deleteWithConfirm(file, accumulate, null);
-    }
-    
-    public static Throwable deleteWithConfirm(File file, Throwable accumulate, RateLimiter rateLimiter)
-    {
-        try
-        {
-            if (rateLimiter != null)
-            {
-                double throttled = rateLimiter.acquire();
-                if (throttled > 0.0)
-                    nospam1m.warn("Throttling file deletion: waited {} seconds to delete {}", throttled, file);
-            }
-            Files.delete(file.toPath());
-        }
-        catch (Throwable t)
-        {
-            try
-            {
-                throw new FSWriteError(t, file);
-            }
-            catch (Throwable t2)
-            {
-                accumulate = merge(accumulate, t2);
-            }
-        }
-        return accumulate;
-    }
-
-    public static void deleteWithConfirm(String file)
-    {
-        deleteWithConfirm(new File(file));
-    }
-
-    public static void deleteWithConfirm(File file)
-    {
-        maybeFail(deleteWithConfirm(file, null, null));
-    }
-
-    public static void deleteWithConfirmWithThrottle(File file, RateLimiter rateLimiter)
-    {
-        maybeFail(deleteWithConfirm(file, null, rateLimiter));
-    }
-
     public static void copyWithOutConfirm(String from, String to)
     {
         try
@@ -300,7 +244,7 @@ public final class FileUtils
     {
         assert from.exists();
         if (logger.isTraceEnabled())
-            logger.trace("Copying {} to {}", from.getPath(), to.getPath());
+            logger.trace("Copying {} to {}", from.path(), to.path());
 
         try
         {
@@ -312,74 +256,16 @@ public final class FileUtils
         }
     }
 
-    public static void renameWithOutConfirm(String from, String to)
-    {
-        try
-        {
-            atomicMoveWithFallback(new File(from).toPath(), new File(to).toPath());
-        }
-        catch (IOException e)
-        {
-            if (logger.isTraceEnabled())
-                logger.trace("Could not move file "+from+" to "+to, e);
-        }
-    }
-
-    public static void renameWithConfirm(String from, String to)
-    {
-        renameWithConfirm(new File(from), new File(to));
-    }
-
-    public static void renameWithConfirm(File from, File to)
-    {
-        assert from.exists();
-        if (logger.isTraceEnabled())
-            logger.trace("Renaming {} to {}", from.getPath(), to.getPath());
-        // this is not FSWE because usually when we see it it's because we didn't close the file before renaming it,
-        // and Windows is picky about that.
-        try
-        {
-            atomicMoveWithFallback(from.toPath(), to.toPath());
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(String.format("Failed to rename %s to %s", from.getPath(), to.getPath()), e);
-        }
-    }
-
-    /**
-     * Move a file atomically, if it fails, it falls back to a non-atomic operation
-     * @param from
-     * @param to
-     * @throws IOException
-     */
-    private static void atomicMoveWithFallback(Path from, Path to) throws IOException
-    {
-        try
-        {
-            Files.move(from, to, StandardCopyOption.REPLACE_EXISTING, StandardCopyOption.ATOMIC_MOVE);
-        }
-        catch (AtomicMoveNotSupportedException e)
-        {
-            logger.trace("Could not do an atomic move", e);
-            Files.move(from, to, StandardCopyOption.REPLACE_EXISTING);
-        }
-
-    }
-
     public static void truncate(String path, long size)
     {
-        try(FileChannel channel = FileChannel.open(Paths.get(path), StandardOpenOption.READ, StandardOpenOption.WRITE))
+        File file = new File(path);
+        try (FileChannel channel = file.newReadWriteChannel())
         {
             channel.truncate(size);
         }
-        catch (NoSuchFileException | FileNotFoundException nfe)
-        {
-            throw new RuntimeException(nfe);
-        }
         catch (IOException e)
         {
-            throw new FSWriteError(e, path);
+            throw PathUtils.propagateUnchecked(e, file.toPath(), true);
         }
     }
 
@@ -452,49 +338,18 @@ public final class FileUtils
 
     public static String getCanonicalPath(String filename)
     {
-        try
-        {
-            return new File(filename).getCanonicalPath();
-        }
-        catch (IOException e)
-        {
-            throw new FSReadError(e, filename);
-        }
+        return new File(filename).canonicalPath();
     }
 
     public static String getCanonicalPath(File file)
     {
-        try
-        {
-            return file.getCanonicalPath();
-        }
-        catch (IOException e)
-        {
-            throw new FSReadError(e, file);
-        }
+        return file.canonicalPath();
     }
 
     /** Return true if file is contained in folder */
     public static boolean isContained(File folder, File file)
     {
-        Path folderPath = Paths.get(getCanonicalPath(folder));
-        Path filePath = Paths.get(getCanonicalPath(file));
-
-        return filePath.startsWith(folderPath);
-    }
-
-    /** Convert absolute path into a path relative to the base path */
-    public static String getRelativePath(String basePath, String path)
-    {
-        try
-        {
-            return Paths.get(basePath).relativize(Paths.get(path)).toString();
-        }
-        catch(Exception ex)
-        {
-            String absDataPath = FileUtils.getCanonicalPath(basePath);
-            return Paths.get(absDataPath).relativize(Paths.get(path)).toString();
-        }
+        return folder.isAncestorOf(file);
     }
 
     public static void clean(ByteBuffer buffer)
@@ -525,52 +380,6 @@ public final class FileUtils
         }
     }
 
-    public static void createDirectory(String directory)
-    {
-        createDirectory(new File(directory));
-    }
-
-    public static void createDirectory(File directory)
-    {
-        if (!directory.exists())
-        {
-            if (!directory.mkdirs())
-                throw new FSWriteError(new IOException("Failed to mkdirs " + directory), directory);
-        }
-    }
-
-    public static boolean delete(String file)
-    {
-        if (!StorageService.instance.isDaemonSetupCompleted())
-            logger.info("Deleting file during startup: {}", file);
-
-        File f = new File(file);
-        return f.delete();
-    }
-
-    public static void delete(File... files)
-    {
-        for ( File file : files )
-        {
-            if (!StorageService.instance.isDaemonSetupCompleted())
-                logger.info("Deleting file during startup: {}", file);
-
-            file.delete();
-        }
-    }
-
-    public static void deleteAsync(final String file)
-    {
-        Runnable runnable = new Runnable()
-        {
-            public void run()
-            {
-                deleteWithConfirm(new File(file));
-            }
-        };
-        ScheduledExecutors.nonPeriodicTasks.execute(runnable);
-    }
-
     public static long parseFileSize(String value)
     {
         long result;
@@ -644,72 +453,6 @@ public final class FileUtils
         }
     }
 
-    /**
-     * Deletes all files and subdirectories under "dir".
-     * @param dir Directory to be deleted
-     * @throws FSWriteError if any part of the tree cannot be deleted
-     */
-    public static void deleteRecursiveWithThrottle(File dir, RateLimiter rateLimiter)
-    {
-        if (dir.isDirectory())
-        {
-            String[] children = dir.list();
-            for (String child : children)
-                deleteRecursiveWithThrottle(new File(dir, child), rateLimiter);
-        }
-
-        // The directory is now empty so now it can be smoked
-        deleteWithConfirmWithThrottle(dir, rateLimiter);
-    }
-
-
-    /**
-     * Deletes the specified directory after having deleted its content.
-     *
-     * @param dir Directory to be deleted
-     * @throws FSWriteError if any part of the tree cannot be deleted
-     */
-    public static void deleteRecursive(File dir)
-    {
-        deleteChildrenRecursive(dir);
-
-        // The directory is now empty so now it can be smoked
-        deleteWithConfirm(dir);
-    }
-
-    /**
-     * Deletes all files and subdirectories under "dir".
-     *
-     * @param dir Directory to be deleted
-     * @throws FSWriteError if any part of the tree cannot be deleted
-     */
-    public static void deleteChildrenRecursive(File dir)
-    {
-        if (dir.isDirectory())
-        {
-            String[] children = dir.list();
-            for (String child : children)
-                deleteRecursive(new File(dir, child));
-        }
-    }
-
-    /**
-     * Schedules deletion of all file and subdirectories under "dir" on JVM shutdown.
-     * @param dir Directory to be deleted
-     */
-    public static void deleteRecursiveOnExit(File dir)
-    {
-        if (dir.isDirectory())
-        {
-            String[] children = dir.list();
-            for (String child : children)
-                deleteRecursiveOnExit(new File(dir, child));
-        }
-
-        logger.trace("Scheduling deferred deletion of file: {}", dir);
-        dir.deleteOnExit();
-    }
-
     public static void handleCorruptSSTable(CorruptSSTableException e)
     {
         fsErrorHandler.get().ifPresent(handler -> handler.handleCorruptSSTable(e));
@@ -761,41 +504,6 @@ public final class FileUtils
         return sizeArr[0];
     }
 
-    public static void copyTo(DataInput in, OutputStream out, int length) throws IOException
-    {
-        byte[] buffer = new byte[64 * 1024];
-        int copiedBytes = 0;
-
-        while (copiedBytes + buffer.length < length)
-        {
-            in.readFully(buffer);
-            out.write(buffer);
-            copiedBytes += buffer.length;
-        }
-
-        if (copiedBytes < length)
-        {
-            int left = length - copiedBytes;
-            in.readFully(buffer, 0, left);
-            out.write(buffer, 0, left);
-        }
-    }
-
-    public static boolean isSubDirectory(File parent, File child) throws IOException
-    {
-        parent = parent.getCanonicalFile();
-        child = child.getCanonicalFile();
-
-        File toCheck = child;
-        while (toCheck != null)
-        {
-            if (parent.equals(toCheck))
-                return true;
-            toCheck = toCheck.getParentFile();
-        }
-        return false;
-    }
-
     public static void append(File file, String ... lines)
     {
         if (file.exists())
@@ -829,7 +537,10 @@ public final class FileUtils
      */
     public static void write(File file, List<String> lines, StandardOpenOption ... options)
     {
-        Set<StandardOpenOption> optionsSet = new HashSet<>(Arrays.asList(options));
+        Set<StandardOpenOption> optionsSet = EnumSet.noneOf(StandardOpenOption.class);
+        for (StandardOpenOption option : options)
+            optionsSet.add(option);
+
         //Emulate the old FileSystemProvider.newOutputStream behavior for open options.
         if (optionsSet.isEmpty())
         {
@@ -888,159 +599,116 @@ public final class FileUtils
         fsErrorHandler.getAndSet(Optional.ofNullable(handler));
     }
 
-    /**
-     * Returns the size of the specified partition.
-     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  size overflow.
-     * See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information.</p>
-     *
-     * @param file the partition
-     * @return the size, in bytes, of the partition or {@code 0L} if the abstract pathname does not name a partition
-     */
-    public static long getTotalSpace(File file)
+    @Deprecated
+    public static void createDirectory(String directory)
     {
-        return handleLargeFileSystem(file.getTotalSpace());
+        createDirectory(new File(directory));
     }
 
-    /**
-     * Returns the number of unallocated bytes on the specified partition.
-     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of unallocated bytes
-     * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
-     *
-     * @param file the partition
-     * @return the number of unallocated bytes on the partition or {@code 0L}
-     * if the abstract pathname does not name a partition.
-     */
-    public static long getFreeSpace(File file)
+    @Deprecated
+    public static void createDirectory(File directory)
     {
-        return handleLargeFileSystem(file.getFreeSpace());
+        PathUtils.createDirectoriesIfNotExists(directory.toPath());
     }
 
-    /**
-     * Returns the number of available bytes on the specified partition.
-     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of available bytes
-     * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
-     *
-     * @param file the partition
-     * @return the number of available bytes on the partition or {@code 0L}
-     * if the abstract pathname does not name a partition.
-     */
-    public static long getUsableSpace(File file)
+    @Deprecated
+    public static boolean delete(String file)
     {
-        return handleLargeFileSystem(file.getUsableSpace());
+        return new File(file).tryDelete();
     }
 
-    /**
-     * Returns the {@link FileStore} representing the file store where a file
-     * is located. This {@link FileStore} handles large file system by returning {@code Long.MAX_VALUE}
-     * from {@code FileStore#getTotalSpace()}, {@code FileStore#getUnallocatedSpace()} and {@code FileStore#getUsableSpace()}
-     * it the value is bigger than {@code Long.MAX_VALUE}. See <a href='https://bugs.openjdk.java.net/browse/JDK-8162520'>JDK-8162520</a>
-     * for more information.
-     *
-     * @param path the path to the file
-     * @return the file store where the file is stored
-     */
-    public static FileStore getFileStore(Path path) throws IOException
+    @Deprecated
+    public static void delete(File... files)
     {
-        return new SafeFileStore(Files.getFileStore(path));
+        for (File file : files)
+            file.tryDelete();
     }
 
     /**
-     * Handle large file system by returning {@code Long.MAX_VALUE} when the size overflows.
-     * @param size returned by the Java's FileStore methods
-     * @return the size or {@code Long.MAX_VALUE} if the size was bigger than {@code Long.MAX_VALUE}
+     * Deletes all files and subdirectories under "dir".
+     * @param dir Directory to be deleted
+     * @throws FSWriteError if any part of the tree cannot be deleted
      */
-    private static long handleLargeFileSystem(long size)
+    @Deprecated
+    public static void deleteRecursiveWithThrottle(File dir, RateLimiter rateLimiter)
     {
-        return size < 0 ? Long.MAX_VALUE : size;
+        dir.deleteRecursive(rateLimiter);
     }
 
     /**
-     * Private constructor as the class contains only static methods.
+     * Deletes all files and subdirectories under "dir".
+     * @param dir Directory to be deleted
+     * @throws FSWriteError if any part of the tree cannot be deleted
      */
-    private FileUtils()
+    @Deprecated
+    public static void deleteRecursive(File dir)
     {
+        dir.deleteRecursive();
     }
 
     /**
-     * FileStore decorator used to safely handle large file system.
-     *
-     * <p>Java's FileStore methods (getTotalSpace/getUnallocatedSpace/getUsableSpace) are limited to reporting bytes as
-     * signed long (2^63-1), if the filesystem is any bigger, then the size overflows. {@code SafeFileStore} will
-     * return {@code Long.MAX_VALUE} if the size overflow.</p>
-     *
-     * @see <a href="https://bugs.openjdk.java.net/browse/JDK-8162520">JDK-8162520</a>.
+     * Schedules deletion of all file and subdirectories under "dir" on JVM shutdown.
+     * @param dir Directory to be deleted
      */
-    private static final class SafeFileStore extends FileStore
+    @Deprecated
+    public static void deleteRecursiveOnExit(File dir)
     {
-        /**
-         * The decorated {@code FileStore}
-         */
-        private final FileStore fileStore;
-
-        public SafeFileStore(FileStore fileStore)
-        {
-            this.fileStore = fileStore;
-        }
-
-        @Override
-        public String name()
-        {
-            return fileStore.name();
-        }
+        dir.deleteRecursiveOnExit();
+    }
 
-        @Override
-        public String type()
-        {
-            return fileStore.type();
-        }
+    @Deprecated
+    public static boolean isSubDirectory(File parent, File child)
+    {
+        return parent.isAncestorOf(child);
+    }
 
-        @Override
-        public boolean isReadOnly()
-        {
-            return fileStore.isReadOnly();
-        }
+    @Deprecated
+    public static Throwable deleteWithConfirm(File file, Throwable accumulate)
+    {
+        return file.delete(accumulate, null);
+    }
 
-        @Override
-        public long getTotalSpace() throws IOException
-        {
-            return handleLargeFileSystem(fileStore.getTotalSpace());
-        }
+    @Deprecated
+    public static Throwable deleteWithConfirm(File file, Throwable accumulate, RateLimiter rateLimiter)
+    {
+        return file.delete(accumulate, rateLimiter);
+    }
 
-        @Override
-        public long getUsableSpace() throws IOException
-        {
-            return handleLargeFileSystem(fileStore.getUsableSpace());
-        }
+    @Deprecated
+    public static void deleteWithConfirm(String file)
+    {
+        deleteWithConfirm(new File(file));
+    }
 
-        @Override
-        public long getUnallocatedSpace() throws IOException
-        {
-            return handleLargeFileSystem(fileStore.getUnallocatedSpace());
-        }
+    @Deprecated
+    public static void deleteWithConfirm(File file)
+    {
+        file.delete();
+    }
 
-        @Override
-        public boolean supportsFileAttributeView(Class<? extends FileAttributeView> type)
-        {
-            return fileStore.supportsFileAttributeView(type);
-        }
+    @Deprecated
+    public static void renameWithOutConfirm(String from, String to)
+    {
+        new File(from).tryMove(new File(to));
+    }
 
-        @Override
-        public boolean supportsFileAttributeView(String name)
-        {
-            return fileStore.supportsFileAttributeView(name);
-        }
+    @Deprecated
+    public static void renameWithConfirm(String from, String to)
+    {
+        renameWithConfirm(new File(from), new File(to));
+    }
 
-        @Override
-        public <V extends FileStoreAttributeView> V getFileStoreAttributeView(Class<V> type)
-        {
-            return fileStore.getFileStoreAttributeView(type);
-        }
+    @Deprecated
+    public static void renameWithConfirm(File from, File to)
+    {
+        from.move(to);
+    }
 
-        @Override
-        public Object getAttribute(String attribute) throws IOException
-        {
-            return fileStore.getAttribute(attribute);
-        }
+    /**
+     * Private constructor as the class contains only static methods.
+     */
+    private FileUtils()
+    {
     }
 
     /**
@@ -1060,9 +728,9 @@ public final class FileUtils
         {
             Files.createDirectories(target);
 
-            for (File f : source.toFile().listFiles())
+            for (File f : new File(source).tryList())
             {
-                String fileName = f.getName();
+                String fileName = f.name();
                 moveRecursively(source.resolve(fileName), target.resolve(fileName));
             }
 
diff --git a/src/java/org/apache/cassandra/io/util/RewindableDataInput.java b/src/java/org/apache/cassandra/io/util/FileWriter.java
similarity index 63%
copy from src/java/org/apache/cassandra/io/util/RewindableDataInput.java
copy to src/java/org/apache/cassandra/io/util/FileWriter.java
index c202f60..bbfb595 100644
--- a/src/java/org/apache/cassandra/io/util/RewindableDataInput.java
+++ b/src/java/org/apache/cassandra/io/util/FileWriter.java
@@ -19,12 +19,21 @@
 package org.apache.cassandra.io.util;
 
 import java.io.IOException;
+import java.io.OutputStreamWriter;
 
-public interface RewindableDataInput extends DataInputPlus
-{
-    DataPosition mark();
+import org.apache.cassandra.io.util.File.WriteMode;
 
-    void reset(DataPosition mark) throws IOException;
+public class FileWriter extends OutputStreamWriter
+{
+    @SuppressWarnings("resource") // FOSP is closed by OSW::close
+    public FileWriter(File file) throws IOException
+    {
+        super(new FileOutputStreamPlus(file));
+    }
 
-    long bytesPastMark(DataPosition mark);
-}
\ No newline at end of file
+    @SuppressWarnings("resource") // FOSP is closed by OSW::close
+    public FileWriter(File file, WriteMode mode) throws IOException
+    {
+        super(new FileOutputStreamPlus(file, mode));
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/util/PathUtils.java b/src/java/org/apache/cassandra/io/util/PathUtils.java
new file mode 100644
index 0000000..6df5be1
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/PathUtils.java
@@ -0,0 +1,731 @@
+/*
+ * 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.io.util;
+
+import java.io.*;
+import java.nio.channels.FileChannel;
+import java.nio.file.*;
+import java.nio.file.attribute.*;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+import java.util.function.*;
+import java.util.stream.Stream;
+
+import javax.annotation.Nullable;
+
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.RateLimiter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.CassandraRelevantProperties;
+import org.apache.cassandra.io.FSError;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.NoSpamLogger;
+
+import static java.nio.file.StandardOpenOption.*;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static java.util.Collections.unmodifiableSet;
+import static org.apache.cassandra.utils.Throwables.merge;
+
+/**
+ * Vernacular: tryX means return false or 0L on any failure; XIfNotY means propagate any exceptions besides those caused by Y
+ *
+ * This class tries to apply uniform IOException handling, and does not propagate IOException except for NoSuchFileException.
+ * Any harmless/application error exceptions are propagated as UncheckedIOException, and anything else as an FSReadError or FSWriteError.
+ * Semantically this is a little incoherent throughout the codebase, as we intercept IOException haphazardly and treaat
+ * it inconsistently - we should ideally migrate to using {@link #propagate(IOException, Path, boolean)} et al globally.
+ */
+public final class PathUtils
+{
+    private static final boolean consistentDirectoryListings = CassandraRelevantProperties.CONSISTENT_DIRECTORY_LISTINGS.getBoolean();
+
+    private static final Set<StandardOpenOption> READ_OPTIONS = unmodifiableSet(EnumSet.of(READ));
+    private static final Set<StandardOpenOption> WRITE_OPTIONS = unmodifiableSet(EnumSet.of(WRITE, CREATE));
+    private static final Set<StandardOpenOption> WRITE_APPEND_OPTIONS = unmodifiableSet(EnumSet.of(WRITE, CREATE, APPEND));
+    private static final Set<StandardOpenOption> READ_WRITE_OPTIONS = unmodifiableSet(EnumSet.of(READ, WRITE, CREATE));
+    private static final FileAttribute<?>[] NO_ATTRIBUTES = new FileAttribute[0];
+
+    private static final Logger logger = LoggerFactory.getLogger(PathUtils.class);
+    private static final NoSpamLogger nospam1m = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
+    private static Consumer<Path> onDeletion = path -> {
+        if (StorageService.instance.isDaemonSetupCompleted())
+            setDeletionListener(ignore -> {});
+        else
+            logger.info("Deleting file during startup: {}", path);
+    };
+
+    public static FileChannel newReadChannel(Path path) throws NoSuchFileException
+    {
+        return newFileChannel(path, READ_OPTIONS);
+    }
+
+    public static FileChannel newReadWriteChannel(Path path) throws NoSuchFileException
+    {
+        return newFileChannel(path, READ_WRITE_OPTIONS);
+    }
+
+    public static FileChannel newWriteOverwriteChannel(Path path) throws NoSuchFileException
+    {
+        return newFileChannel(path, WRITE_OPTIONS);
+    }
+
+    public static FileChannel newWriteAppendChannel(Path path) throws NoSuchFileException
+    {
+        return newFileChannel(path, WRITE_APPEND_OPTIONS);
+    }
+
+    private static FileChannel newFileChannel(Path path, Set<StandardOpenOption> options) throws NoSuchFileException
+    {
+        try
+        {
+            return FileChannel.open(path, options, PathUtils.NO_ATTRIBUTES);
+        }
+        catch (IOException e)
+        {
+            throw propagateUncheckedOrNoSuchFileException(e, path, options.contains(WRITE));
+        }
+    }
+
+    public static void setDeletionListener(Consumer<Path> newOnDeletion)
+    {
+        onDeletion = newOnDeletion;
+    }
+
+    public static String filename(Path path)
+    {
+        return path.getFileName().toString();
+    }
+
+    public static <T> T[] list(Path path, Function<Stream<Path>, Stream<T>> transform, IntFunction<T[]> arrayFactory)
+    {
+        try (Stream<Path> stream = Files.list(path))
+        {
+            return transform.apply(consistentDirectoryListings ? stream.sorted() : stream)
+                    .toArray(arrayFactory);
+        }
+        catch (NoSuchFileException e)
+        {
+            return null;
+        }
+        catch (IOException e)
+        {
+            throw propagateUnchecked(e, path, false);
+        }
+    }
+
+    public static <T> T[] tryList(Path path, Function<Stream<Path>, Stream<T>> transform, IntFunction<T[]> arrayFactory)
+    {
+        try (Stream<Path> stream = Files.list(path))
+        {
+            return transform.apply(consistentDirectoryListings ? stream.sorted() : stream)
+                    .toArray(arrayFactory);
+        }
+        catch (IOException e)
+        {
+            return null;
+        }
+    }
+
+    public static void forEach(Path path, Consumer<Path> forEach)
+    {
+        try (Stream<Path> stream = Files.list(path))
+        {
+            (consistentDirectoryListings ? stream.sorted() : stream).forEach(forEach);
+        }
+        catch (IOException e)
+        {
+            throw propagateUnchecked(e, path, false);
+        }
+    }
+
+    public static void forEachRecursive(Path path, Consumer<Path> forEach)
+    {
+        Consumer<Path> forEachRecursive = new Consumer<Path>()
+        {
+            @Override
+            public void accept(Path child)
+            {
+                forEach.accept(child);
+                forEach(child, this);
+            }
+        };
+        forEach(path, forEachRecursive);
+    }
+
+    public static long tryGetLength(Path path)
+    {
+        return tryOnPath(path, Files::size);
+    }
+
+    public static long tryGetLastModified(Path path)
+    {
+        return tryOnPath(path, p -> Files.getLastModifiedTime(p).toMillis());
+    }
+
+    public static boolean trySetLastModified(Path path, long lastModified)
+    {
+        try
+        {
+            Files.setLastModifiedTime(path, FileTime.fromMillis(lastModified));
+            return true;
+        }
+        catch (IOException e)
+        {
+            return false;
+        }
+    }
+
+    public static boolean trySetReadable(Path path, boolean readable)
+    {
+        return trySet(path, PosixFilePermission.OWNER_READ, readable);
+    }
+
+    public static boolean trySetWritable(Path path, boolean writeable)
+    {
+        return trySet(path, PosixFilePermission.OWNER_WRITE, writeable);
+    }
+
+    public static boolean trySetExecutable(Path path, boolean executable)
+    {
+        return trySet(path, PosixFilePermission.OWNER_EXECUTE, executable);
+    }
+
+    public static boolean trySet(Path path, PosixFilePermission permission, boolean set)
+    {
+        try
+        {
+            PosixFileAttributeView view = path.getFileSystem().provider().getFileAttributeView(path, PosixFileAttributeView.class);
+            PosixFileAttributes attributes = view.readAttributes();
+            Set<PosixFilePermission> permissions = attributes.permissions();
+            if (set == permissions.contains(permission))
+                return true;
+            if (set) permissions.add(permission);
+            else permissions.remove(permission);
+            view.setPermissions(permissions);
+            return true;
+        }
+        catch (IOException e)
+        {
+            return false;
+        }
+    }
+
+    public static Throwable delete(Path file, Throwable accumulate)
+    {
+        try
+        {
+            delete(file);
+        }
+        catch (FSError t)
+        {
+            accumulate = merge(accumulate, t);
+        }
+        return accumulate;
+    }
+
+    public static void delete(Path file)
+    {
+        try
+        {
+            Files.delete(file);
+            onDeletion.accept(file);
+        }
+        catch (IOException e)
+        {
+            throw propagateUnchecked(e, file, true);
+        }
+    }
+
+    public static boolean tryDelete(Path file)
+    {
+        try
+        {
+            Files.delete(file);
+            onDeletion.accept(file);
+            return true;
+        }
+        catch (IOException e)
+        {
+            return false;
+        }
+    }
+
+    public static void delete(Path file, @Nullable RateLimiter rateLimiter)
+    {
+        if (rateLimiter != null)
+        {
+            double throttled = rateLimiter.acquire();
+            if (throttled > 0.0)
+                nospam1m.warn("Throttling file deletion: waited {} seconds to delete {}", throttled, file);
+        }
+        delete(file);
+    }
+
+    public static Throwable delete(Path file, Throwable accumulate, @Nullable RateLimiter rateLimiter)
+    {
+        try
+        {
+            delete(file, rateLimiter);
+        }
+        catch (Throwable t)
+        {
+            accumulate = merge(accumulate, t);
+        }
+        return accumulate;
+    }
+
+    /**
+     * Deletes all files and subdirectories under "path".
+     * @param path file to be deleted
+     * @throws FSWriteError if any part of the tree cannot be deleted
+     */
+    public static void deleteRecursive(Path path)
+    {
+        if (isDirectory(path))
+            forEach(path, PathUtils::deleteRecursive);
+
+        // The directory is now empty so now it can be smoked
+        delete(path);
+    }
+
+    /**
+     * Deletes all files and subdirectories under "path".
+     * @param path file to be deleted
+     * @throws FSWriteError if any part of the tree cannot be deleted
+     */
+    public static void deleteRecursive(Path path, RateLimiter rateLimiter)
+    {
+        deleteRecursive(path, rateLimiter, p -> deleteRecursive(p, rateLimiter));
+    }
+
+    /**
+     * Deletes all files and subdirectories under "path".
+     * @param path file to be deleted
+     * @throws FSWriteError if any part of the tree cannot be deleted
+     */
+    private static void deleteRecursive(Path path, RateLimiter rateLimiter, Consumer<Path> deleteRecursive)
+    {
+        if (isDirectory(path))
+            forEach(path, deleteRecursive);
+
+        // The directory is now empty so now it can be smoked
+        delete(path, rateLimiter);
+    }
+
+    /**
+     * Schedules deletion of all file and subdirectories under "dir" on JVM shutdown.
+     * @param dir Directory to be deleted
+     */
+    public synchronized static void deleteRecursiveOnExit(Path dir)
+    {
+        ON_EXIT.add(dir, true);
+    }
+
+    /**
+     * Schedules deletion of the file only on JVM shutdown.
+     * @param file File to be deleted
+     */
+    public synchronized static void deleteOnExit(Path file)
+    {
+        ON_EXIT.add(file, false);
+    }
+
+    public static boolean tryRename(Path from, Path to)
+    {
+        logger.trace("Renaming {} to {}", from, to);
+        // this is not FSWE because usually when we see it it's because we didn't close the file before renaming it,
+        // and Windows is picky about that.
+        try
+        {
+            atomicMoveWithFallback(from, to);
+            return true;
+        }
+        catch (IOException e)
+        {
+            logger.trace("Could not move file {} to {}", from, to, e);
+            return false;
+        }
+    }
+
+    public static void rename(Path from, Path to)
+    {
+        logger.trace("Renaming {} to {}", from, to);
+        // this is not FSWE because usually when we see it it's because we didn't close the file before renaming it,
+        // and Windows is picky about that.
+        try
+        {
+            atomicMoveWithFallback(from, to);
+        }
+        catch (IOException e)
+        {
+            logger.trace("Could not move file {} to {}", from, to, e);
+
+            // TODO: this should be an FSError (either read or write)?
+            // (but for now this is maintaining legacy semantics)
+            throw new RuntimeException(String.format("Failed to rename %s to %s", from, to), e);
+        }
+    }
+
+    /**
+     * Move a file atomically, if it fails, it falls back to a non-atomic operation
+     */
+    private static void atomicMoveWithFallback(Path from, Path to) throws IOException
+    {
+        try
+        {
+            Files.move(from, to, StandardCopyOption.REPLACE_EXISTING, StandardCopyOption.ATOMIC_MOVE);
+        }
+        catch (AtomicMoveNotSupportedException e)
+        {
+            logger.trace("Could not do an atomic move", e);
+            Files.move(from, to, StandardCopyOption.REPLACE_EXISTING);
+        }
+    }
+
+    // true if can determine exists, false if any exception occurs
+    public static boolean exists(Path path)
+    {
+        return Files.exists(path);
+    }
+
+    // true if can determine is a directory, false if any exception occurs
+    public static boolean isDirectory(Path path)
+    {
+        return Files.isDirectory(path);
+    }
+
+    // true if can determine is a regular file, false if any exception occurs
+    public static boolean isFile(Path path)
+    {
+        return Files.isRegularFile(path);
+    }
+
+    /**
+     * @param path create file if not exists
+     * @throws IOError if cannot perform the operation
+     * @return true if a new file was created
+     */
+    public static boolean createFileIfNotExists(Path path)
+    {
+        return ifNotExists(path, Files::createFile);
+    }
+
+    /**
+     * @param path create directory if not exists
+     * @throws IOError if cannot perform the operation
+     * @return true if a new directory was created
+     */
+    public static boolean createDirectoryIfNotExists(Path path)
+    {
+        return ifNotExists(path, Files::createDirectory);
+    }
+
+    /**
+     * @param path create directory (and parents) if not exists
+     * @throws IOError if cannot perform the operation
+     * @return true if a new directory was created
+     */
+    public static boolean createDirectoriesIfNotExists(Path path)
+    {
+        return ifNotExists(path, Files::createDirectories);
+    }
+
+    /**
+     * @param path create directory if not exists and action can be performed
+     * @return true if a new directory was created, false otherwise (for any reason)
+     */
+    public static boolean tryCreateDirectory(Path path)
+    {
+        return tryConsume(path, Files::createDirectory);
+    }
+
+    /**
+     * @param path create directory (and parents) if not exists and action can be performed
+     * @return true if the new directory was created, false otherwise (for any reason)
+     */
+    public static boolean tryCreateDirectories(Path path)
+    {
+        if (exists(path))
+            return false;
+
+        tryCreateDirectories(path.toAbsolutePath().getParent());
+        return tryCreateDirectory(path);
+    }
+
+    /**
+     * @return file if exists, otherwise nearest parent that exists; null if nothing in path exists
+     */
+    public static Path findExistingAncestor(Path file)
+    {
+        if (!file.equals(file.normalize()))
+            throw new IllegalArgumentException("Must be invoked on a path without redundant elements");
+
+        Path parent = file;
+        while (parent != null && !Files.exists(parent))
+            parent = parent.getParent();
+        return parent;
+    }
+
+    /**
+     * 1) Convert to an absolute path without redundant path elements;
+     * 2) If the file exists, resolve any links to the underlying fille;
+     * 3) If the file does not exist, find the first ancestor that does and resolve the path from there
+     */
+    public static Path toCanonicalPath(Path file)
+    {
+        Preconditions.checkNotNull(file);
+
+        file = file.toAbsolutePath().normalize();
+        Path parent = findExistingAncestor(file);
+
+        if (parent == null)
+            return file;
+        if (parent == file)
+            return toRealPath(file);
+        return toRealPath(parent).resolve(parent.relativize(file));
+    }
+
+    private static Path toRealPath(Path path)
+    {
+        try
+        {
+            return path.toRealPath();
+        }
+        catch (IOException e)
+        {
+            throw propagateUnchecked(e, path, false);
+        }
+    }
+
+    /**
+     * Return true if file's canonical path is contained in folder's canonical path.
+     *
+     * Propagates any exceptions encountered finding canonical paths.
+     */
+    public static boolean isContained(Path folder, Path file)
+    {
+        Path realFolder = toCanonicalPath(folder), realFile = toCanonicalPath(file);
+        return realFile.startsWith(realFolder);
+    }
+
+    private static final class DeleteOnExit implements Runnable
+    {
+        private boolean isRegistered;
+        private final Set<Path> deleteRecursivelyOnExit = new HashSet<>();
+        private final Set<Path> deleteOnExit = new HashSet<>();
+
+        DeleteOnExit()
+        {
+            Runtime.getRuntime().addShutdownHook(new Thread(this));
+        }
+
+        synchronized void add(Path path, boolean recursive)
+        {
+            if (!isRegistered)
+            {
+                isRegistered = true;
+            }
+            logger.trace("Scheduling deferred {}deletion of file: {}", recursive ? "recursive " : "", path);
+            (recursive ? deleteRecursivelyOnExit : deleteOnExit).add(path);
+        }
+
+        public void run()
+        {
+            for (Path path : deleteOnExit)
+            {
+                try
+                {
+                    if (exists(path))
+                        delete(path);
+                }
+                catch (Throwable t)
+                {
+                    logger.warn("Failed to delete {} on exit", path, t);
+                }
+            }
+            for (Path path : deleteRecursivelyOnExit)
+            {
+                try
+                {
+                    if (exists(path))
+                        deleteRecursive(path);
+                }
+                catch (Throwable t)
+                {
+                    logger.warn("Failed to delete {} on exit", path, t);
+                }
+            }
+        }
+    }
+    private static final DeleteOnExit ON_EXIT = new DeleteOnExit();
+
+    public interface IOConsumer { void accept(Path path) throws IOException; }
+    public interface IOToLongFunction<V> { long apply(V path) throws IOException; }
+
+    private static boolean ifNotExists(Path path, IOConsumer consumer)
+    {
+        try
+        {
+            consumer.accept(path);
+            return true;
+        }
+        catch (FileAlreadyExistsException fae)
+        {
+            return false;
+        }
+        catch (IOException e)
+        {
+            throw propagateUnchecked(e, path, true);
+        }
+    }
+
+    private static boolean tryConsume(Path path, IOConsumer function)
+    {
+        try
+        {
+            function.accept(path);
+            return true;
+        }
+        catch (IOException e)
+        {
+            return false;
+        }
+    }
+
+    private static long tryOnPath(Path path, IOToLongFunction<Path> function)
+    {
+        try
+        {
+            return function.apply(path);
+        }
+        catch (IOException e)
+        {
+            return 0L;
+        }
+    }
+
+    private static long tryOnFileStore(Path path, IOToLongFunction<FileStore> function)
+    {
+        return tryOnFileStore(path, function, ignore -> {});
+    }
+
+    private static long tryOnFileStore(Path path, IOToLongFunction<FileStore> function, Consumer<IOException> orElse)
+    {
+        try
+        {
+            Path ancestor = findExistingAncestor(path.normalize());
+            if (ancestor == null)
+            {
+                orElse.accept(new NoSuchFileException(path.toString()));
+                return 0L;
+            }
+            return function.apply(Files.getFileStore(ancestor));
+        }
+        catch (IOException e)
+        {
+            orElse.accept(e);
+            return 0L;
+        }
+    }
+
+    /**
+     * Returns the number of bytes (determined by the provided MethodHandle) on the specified partition.
+     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of available bytes
+     * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
+     *
+     * @param path the partition (or a file within it)
+     */
+    public static long tryGetSpace(Path path, IOToLongFunction<FileStore> getSpace)
+    {
+        return handleLargeFileSystem(tryOnFileStore(path, getSpace));
+    }
+
+    public static long tryGetSpace(Path path, IOToLongFunction<FileStore> getSpace, Consumer<IOException> orElse)
+    {
+        return handleLargeFileSystem(tryOnFileStore(path, getSpace, orElse));
+    }
+
+    /**
+     * Handle large file system by returning {@code Long.MAX_VALUE} when the size overflows.
+     * @param size returned by the Java's FileStore methods
+     * @return the size or {@code Long.MAX_VALUE} if the size was bigger than {@code Long.MAX_VALUE}
+     */
+    private static long handleLargeFileSystem(long size)
+    {
+        return size < 0 ? Long.MAX_VALUE : size;
+    }
+
+    /**
+     * Private constructor as the class contains only static methods.
+     */
+    private PathUtils()
+    {
+    }
+
+    /**
+     * propagate an IOException as an FSWriteError, FSReadError or UncheckedIOException
+     */
+    public static RuntimeException propagateUnchecked(IOException ioe, Path path, boolean write)
+    {
+        if (ioe instanceof FileAlreadyExistsException
+            || ioe instanceof NoSuchFileException
+            || ioe instanceof AtomicMoveNotSupportedException
+            || ioe instanceof java.nio.file.DirectoryNotEmptyException
+            || ioe instanceof java.nio.file.FileSystemLoopException
+            || ioe instanceof java.nio.file.NotDirectoryException
+            || ioe instanceof java.nio.file.NotLinkException)
+            throw new UncheckedIOException(ioe);
+
+        if (write) throw new FSWriteError(ioe, path);
+        else throw new FSReadError(ioe, path);
+    }
+
+    /**
+     * propagate an IOException as an FSWriteError, FSReadError or UncheckedIOException - except for NoSuchFileException
+     */
+    public static NoSuchFileException propagateUncheckedOrNoSuchFileException(IOException ioe, Path path, boolean write) throws NoSuchFileException
+    {
+        if (ioe instanceof NoSuchFileException)
+            throw (NoSuchFileException) ioe;
+
+        throw propagateUnchecked(ioe, path, write);
+    }
+
+    /**
+     * propagate an IOException either as itself or an FSWriteError or FSReadError
+     */
+    public static <E extends IOException> E propagate(E ioe, Path path, boolean write) throws E
+    {
+        if (ioe instanceof FileAlreadyExistsException
+            || ioe instanceof NoSuchFileException
+            || ioe instanceof AtomicMoveNotSupportedException
+            || ioe instanceof java.nio.file.DirectoryNotEmptyException
+            || ioe instanceof java.nio.file.FileSystemLoopException
+            || ioe instanceof java.nio.file.NotDirectoryException
+            || ioe instanceof java.nio.file.NotLinkException)
+            throw ioe;
+
+        if (write) throw new FSWriteError(ioe, path);
+        else throw new FSReadError(ioe, path);
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
index 33d0127..4118bb3 100644
--- a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
+++ b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.io.util;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteOrder;
 
@@ -26,6 +25,7 @@ import javax.annotation.concurrent.NotThreadSafe;
 import com.google.common.primitives.Ints;
 
 import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.io.util.Rebufferer.BufferHolder;
 
 @NotThreadSafe
diff --git a/src/java/org/apache/cassandra/io/util/ReaderFileProxy.java b/src/java/org/apache/cassandra/io/util/ReaderFileProxy.java
index 3ddb143..5d38e80 100644
--- a/src/java/org/apache/cassandra/io/util/ReaderFileProxy.java
+++ b/src/java/org/apache/cassandra/io/util/ReaderFileProxy.java
@@ -33,4 +33,4 @@ public interface ReaderFileProxy extends AutoCloseable
      * Needed for tests. Returns the table's CRC check chance, which is only set for compressed tables.
      */
     double getCrcCheckChance();
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/io/util/Rebufferer.java b/src/java/org/apache/cassandra/io/util/Rebufferer.java
index 2fc7ffa..9920de9 100644
--- a/src/java/org/apache/cassandra/io/util/Rebufferer.java
+++ b/src/java/org/apache/cassandra/io/util/Rebufferer.java
@@ -81,4 +81,4 @@ public interface Rebufferer extends ReaderFileProxy
             // nothing to do
         }
     };
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/io/util/RewindableDataInput.java b/src/java/org/apache/cassandra/io/util/RewindableDataInput.java
index c202f60..0a0eee4 100644
--- a/src/java/org/apache/cassandra/io/util/RewindableDataInput.java
+++ b/src/java/org/apache/cassandra/io/util/RewindableDataInput.java
@@ -27,4 +27,4 @@ public interface RewindableDataInput extends DataInputPlus
     void reset(DataPosition mark) throws IOException;
 
     long bytesPastMark(DataPosition mark);
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java b/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java
deleted file mode 100644
index a1842bc..0000000
--- a/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java
+++ /dev/null
@@ -1,571 +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.io.util;
-
-import java.io.Closeable;
-import java.io.File;
-import java.io.FilterInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.RandomAccessFile;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static org.apache.cassandra.utils.Throwables.maybeFail;
-import static org.apache.cassandra.utils.Throwables.merge;
-
-/**
- * Adds mark/reset functionality to another input stream by caching read bytes to a memory buffer and
- * spilling to disk if necessary.
- *
- * When the stream is marked via {@link #mark()} or {@link #mark(int)}, up to
- * <code>maxMemBufferSize</code> will be cached in memory (heap). If more than
- * <code>maxMemBufferSize</code> bytes are read while the stream is marked, the
- * following bytes are cached on the <code>spillFile</code> for up to <code>maxDiskBufferSize</code>.
- *
- * Please note that successive calls to {@link #mark()} and {@link #reset()} will write
- * sequentially to the same <code>spillFile</code> until <code>maxDiskBufferSize</code> is reached.
- * At this point, if less than <code>maxDiskBufferSize</code> bytes are currently cached on the
- * <code>spillFile</code>, the remaining bytes are written to the beginning of the file,
- * treating the <code>spillFile</code> as a circular buffer.
- *
- * If more than <code>maxMemBufferSize + maxDiskBufferSize</code> are cached while the stream is marked,
- * the following {@link #reset()} invocation will throw a {@link IllegalStateException}.
- *
- */
-public class RewindableDataInputStreamPlus extends FilterInputStream implements RewindableDataInput, Closeable
-{
-    private boolean marked = false;
-    private boolean exhausted = false;
-    private AtomicBoolean closed = new AtomicBoolean(false);
-
-    protected int memAvailable = 0;
-    protected int diskTailAvailable = 0;
-    protected int diskHeadAvailable = 0;
-
-    private final File spillFile;
-    private final int initialMemBufferSize;
-    private final int maxMemBufferSize;
-    private final int maxDiskBufferSize;
-
-    private volatile byte memBuffer[];
-    private int memBufferSize;
-    private RandomAccessFile spillBuffer;
-
-    private final DataInputPlus dataReader;
-
-    public RewindableDataInputStreamPlus(InputStream in, int initialMemBufferSize, int maxMemBufferSize,
-                                         File spillFile, int maxDiskBufferSize)
-    {
-        super(in);
-        dataReader = new DataInputStreamPlus(this);
-        this.initialMemBufferSize = initialMemBufferSize;
-        this.maxMemBufferSize = maxMemBufferSize;
-        this.spillFile = spillFile;
-        this.maxDiskBufferSize = maxDiskBufferSize;
-    }
-
-    /* RewindableDataInput methods */
-
-    /**
-     * Marks the current position of a stream to return to this position later via the {@link #reset(DataPosition)} method.
-     * @return An empty @link{DataPosition} object
-     */
-    public DataPosition mark()
-    {
-        mark(0);
-        return new RewindableDataInputPlusMark();
-    }
-
-    /**
-     * Rewinds to the previously marked position via the {@link #mark()} method.
-     * @param mark it's not possible to return to a custom position, so this parameter is ignored.
-     * @throws IOException if an error ocurs while resetting
-     */
-    public void reset(DataPosition mark) throws IOException
-    {
-        reset();
-    }
-
-    public long bytesPastMark(DataPosition mark)
-    {
-        return maxMemBufferSize - memAvailable + (diskTailAvailable == -1? 0 : maxDiskBufferSize - diskHeadAvailable - diskTailAvailable);
-    }
-
-
-    protected static class RewindableDataInputPlusMark implements DataPosition
-    {
-    }
-
-    /* InputStream methods */
-
-    public boolean markSupported()
-    {
-        return true;
-    }
-
-    /**
-     * Marks the current position of a stream to return to this position
-     * later via the {@link #reset()} method.
-     * @param readlimit the maximum amount of bytes to cache
-     */
-    public synchronized void mark(int readlimit)
-    {
-        if (marked)
-            throw new IllegalStateException("Cannot mark already marked stream.");
-
-        if (memAvailable > 0 || diskHeadAvailable > 0 || diskTailAvailable > 0)
-            throw new IllegalStateException("Can only mark stream after reading previously marked data.");
-
-        marked = true;
-        memAvailable = maxMemBufferSize;
-        diskHeadAvailable = -1;
-        diskTailAvailable = -1;
-    }
-
-    public synchronized void reset() throws IOException
-    {
-        if (!marked)
-            throw new IOException("Must call mark() before calling reset().");
-
-        if (exhausted)
-            throw new IOException(String.format("Read more than capacity: %d bytes.", maxMemBufferSize + maxDiskBufferSize));
-
-        memAvailable = maxMemBufferSize - memAvailable;
-        memBufferSize = memAvailable;
-
-        if (diskTailAvailable == -1)
-        {
-            diskHeadAvailable = 0;
-            diskTailAvailable = 0;
-        }
-        else
-        {
-            int initialPos = diskTailAvailable > 0 ? 0 : (int)getIfNotClosed(spillBuffer).getFilePointer();
-            int diskMarkpos = initialPos + diskHeadAvailable;
-            getIfNotClosed(spillBuffer).seek(diskMarkpos);
-
-            diskHeadAvailable = diskMarkpos - diskHeadAvailable;
-            diskTailAvailable = (maxDiskBufferSize - diskTailAvailable) - diskMarkpos;
-        }
-
-        marked = false;
-    }
-
-    public int available() throws IOException
-    {
-
-        return super.available() + (marked? 0 : memAvailable + diskHeadAvailable + diskTailAvailable);
-    }
-
-    public int read() throws IOException
-    {
-        int read = readOne();
-        if (read == -1)
-            return read;
-
-        if (marked)
-        {
-            //mark exhausted
-            if (isExhausted(1))
-            {
-                exhausted = true;
-                return read;
-            }
-
-            writeOne(read);
-        }
-
-        return read;
-    }
-
-    public int read(byte[] b, int off, int len) throws IOException
-    {
-        int readBytes = readMulti(b, off, len);
-        if (readBytes == -1)
-            return readBytes;
-
-        if (marked)
-        {
-            //check we have space on buffer
-            if (isExhausted(readBytes))
-            {
-                exhausted = true;
-                return readBytes;
-            }
-
-            writeMulti(b, off, readBytes);
-        }
-
-        return readBytes;
-    }
-
-    private void maybeCreateDiskBuffer() throws IOException
-    {
-        if (spillBuffer == null)
-        {
-            if (!spillFile.getParentFile().exists())
-                spillFile.getParentFile().mkdirs();
-            spillFile.createNewFile();
-
-            this.spillBuffer = new RandomAccessFile(spillFile, "rw");
-        }
-    }
-
-
-    private int readOne() throws IOException
-    {
-        if (!marked)
-        {
-            if (memAvailable > 0)
-            {
-                int pos = memBufferSize - memAvailable;
-                memAvailable--;
-                return getIfNotClosed(memBuffer)[pos] & 0xff;
-            }
-
-            if (diskTailAvailable > 0 || diskHeadAvailable > 0)
-            {
-                int read = getIfNotClosed(spillBuffer).read();
-                if (diskTailAvailable > 0)
-                    diskTailAvailable--;
-                else if (diskHeadAvailable > 0)
-                    diskHeadAvailable++;
-                if (diskTailAvailable == 0)
-                    spillBuffer.seek(0);
-                return read;
-            }
-        }
-
-        return getIfNotClosed(in).read();
-    }
-
-    private boolean isExhausted(int readBytes)
-    {
-        return exhausted || readBytes > memAvailable + (long)(diskTailAvailable == -1? maxDiskBufferSize : diskTailAvailable + diskHeadAvailable);
-    }
-
-    private int readMulti(byte[] b, int off, int len) throws IOException
-    {
-        int readBytes = 0;
-        if (!marked)
-        {
-            if (memAvailable > 0)
-            {
-                readBytes += memAvailable < len ? memAvailable : len;
-                int pos = memBufferSize - memAvailable;
-                System.arraycopy(memBuffer, pos, b, off, readBytes);
-                memAvailable -= readBytes;
-                off += readBytes;
-                len -= readBytes;
-            }
-            if (len > 0 && diskTailAvailable > 0)
-            {
-                int readFromTail = diskTailAvailable < len? diskTailAvailable : len;
-                readFromTail = getIfNotClosed(spillBuffer).read(b, off, readFromTail);
-                readBytes += readFromTail;
-                diskTailAvailable -= readFromTail;
-                off += readFromTail;
-                len -= readFromTail;
-                if (diskTailAvailable == 0)
-                    spillBuffer.seek(0);
-            }
-            if (len > 0 && diskHeadAvailable > 0)
-            {
-                int readFromHead = diskHeadAvailable < len? diskHeadAvailable : len;
-                readFromHead = getIfNotClosed(spillBuffer).read(b, off, readFromHead);
-                readBytes += readFromHead;
-                diskHeadAvailable -= readFromHead;
-                off += readFromHead;
-                len -= readFromHead;
-            }
-        }
-
-        if (len > 0)
-            readBytes += getIfNotClosed(in).read(b, off, len);
-
-        return readBytes;
-    }
-
-    private void writeMulti(byte[] b, int off, int len) throws IOException
-    {
-        if (memAvailable > 0)
-        {
-            if (memBuffer == null)
-                memBuffer = new byte[initialMemBufferSize];
-            int pos = maxMemBufferSize - memAvailable;
-            int memWritten = memAvailable < len? memAvailable : len;
-            if (pos + memWritten >= getIfNotClosed(memBuffer).length)
-                growMemBuffer(pos, memWritten);
-            System.arraycopy(b, off, memBuffer, pos, memWritten);
-            off += memWritten;
-            len -= memWritten;
-            memAvailable -= memWritten;
-        }
-
-        if (len > 0)
-        {
-            if (diskTailAvailable == -1)
-            {
-                maybeCreateDiskBuffer();
-                diskHeadAvailable = (int)spillBuffer.getFilePointer();
-                diskTailAvailable = maxDiskBufferSize - diskHeadAvailable;
-            }
-
-            if (len > 0 && diskTailAvailable > 0)
-            {
-                int diskTailWritten = diskTailAvailable < len? diskTailAvailable : len;
-                getIfNotClosed(spillBuffer).write(b, off, diskTailWritten);
-                off += diskTailWritten;
-                len -= diskTailWritten;
-                diskTailAvailable -= diskTailWritten;
-                if (diskTailAvailable == 0)
-                    spillBuffer.seek(0);
-            }
-
-            if (len > 0 && diskTailAvailable > 0)
-            {
-                int diskHeadWritten = diskHeadAvailable < len? diskHeadAvailable : len;
-                getIfNotClosed(spillBuffer).write(b, off, diskHeadWritten);
-            }
-        }
-    }
-
-    private void writeOne(int value) throws IOException
-    {
-        if (memAvailable > 0)
-        {
-            if (memBuffer == null)
-                memBuffer = new byte[initialMemBufferSize];
-            int pos = maxMemBufferSize - memAvailable;
-            if (pos == getIfNotClosed(memBuffer).length)
-                growMemBuffer(pos, 1);
-            getIfNotClosed(memBuffer)[pos] = (byte)value;
-            memAvailable--;
-            return;
-        }
-
-        if (diskTailAvailable == -1)
-        {
-            maybeCreateDiskBuffer();
-            diskHeadAvailable = (int)spillBuffer.getFilePointer();
-            diskTailAvailable = maxDiskBufferSize - diskHeadAvailable;
-        }
-
-        if (diskTailAvailable > 0 || diskHeadAvailable > 0)
-        {
-            getIfNotClosed(spillBuffer).write(value);
-            if (diskTailAvailable > 0)
-                diskTailAvailable--;
-            else if (diskHeadAvailable > 0)
-                diskHeadAvailable--;
-            if (diskTailAvailable == 0)
-                spillBuffer.seek(0);
-            return;
-        }
-    }
-
-    public int read(byte[] b) throws IOException
-    {
-        return read(b, 0, b.length);
-    }
-
-    private void growMemBuffer(int pos, int writeSize)
-    {
-        int newSize = Math.min(2 * (pos + writeSize), maxMemBufferSize);
-        byte newBuffer[] = new byte[newSize];
-        System.arraycopy(memBuffer, 0, newBuffer, 0, pos);
-        memBuffer = newBuffer;
-    }
-
-    public long skip(long n) throws IOException
-    {
-        long skipped = 0;
-
-        if (marked)
-        {
-            //if marked, we need to cache skipped bytes
-            while (n-- > 0 && read() != -1)
-            {
-                skipped++;
-            }
-            return skipped;
-        }
-
-        if (memAvailable > 0)
-        {
-            skipped += memAvailable < n ? memAvailable : n;
-            memAvailable -= skipped;
-            n -= skipped;
-        }
-        if (n > 0 && diskTailAvailable > 0)
-        {
-            int skipFromTail = diskTailAvailable < n? diskTailAvailable : (int)n;
-            getIfNotClosed(spillBuffer).skipBytes(skipFromTail);
-            diskTailAvailable -= skipFromTail;
-            skipped += skipFromTail;
-            n -= skipFromTail;
-            if (diskTailAvailable == 0)
-                spillBuffer.seek(0);
-        }
-        if (n > 0 && diskHeadAvailable > 0)
-        {
-            int skipFromHead = diskHeadAvailable < n? diskHeadAvailable : (int)n;
-            getIfNotClosed(spillBuffer).skipBytes(skipFromHead);
-            diskHeadAvailable -= skipFromHead;
-            skipped += skipFromHead;
-            n -= skipFromHead;
-        }
-
-        if (n > 0)
-            skipped += getIfNotClosed(in).skip(n);
-
-        return skipped;
-    }
-
-    private <T> T getIfNotClosed(T in) throws IOException
-    {
-        if (closed.get())
-            throw new IOException("Stream closed");
-        return in;
-    }
-
-    public void close() throws IOException
-    {
-        close(true);
-    }
-
-    public void close(boolean closeUnderlying) throws IOException
-    {
-        if (closed.compareAndSet(false, true))
-        {
-            Throwable fail = null;
-            if (closeUnderlying)
-            {
-                try
-                {
-                    super.close();
-                }
-                catch (IOException e)
-                {
-                    fail = merge(fail, e);
-                }
-            }
-            try
-            {
-                if (spillBuffer != null)
-                {
-                    this.spillBuffer.close();
-                    this.spillBuffer = null;
-                }
-            } catch (IOException e)
-            {
-                fail = merge(fail, e);
-            }
-            try
-            {
-                if (spillFile.exists())
-                {
-                    spillFile.delete();
-                }
-            }
-            catch (Throwable e)
-            {
-                fail = merge(fail, e);
-            }
-            maybeFail(fail, IOException.class);
-        }
-    }
-
-    /* DataInputPlus methods */
-
-    public void readFully(byte[] b) throws IOException
-    {
-        dataReader.readFully(b);
-    }
-
-    public void readFully(byte[] b, int off, int len) throws IOException
-    {
-        dataReader.readFully(b, off, len);
-    }
-
-    public int skipBytes(int n) throws IOException
-    {
-        return dataReader.skipBytes(n);
-    }
-
-    public boolean readBoolean() throws IOException
-    {
-        return dataReader.readBoolean();
-    }
-
-    public byte readByte() throws IOException
-    {
-        return dataReader.readByte();
-    }
-
-    public int readUnsignedByte() throws IOException
-    {
-        return dataReader.readUnsignedByte();
-    }
-
-    public short readShort() throws IOException
-    {
-        return dataReader.readShort();
-    }
-
-    public int readUnsignedShort() throws IOException
-    {
-        return dataReader.readUnsignedShort();
-    }
-
-    public char readChar() throws IOException
-    {
-        return dataReader.readChar();
-    }
-
-    public int readInt() throws IOException
-    {
-        return dataReader.readInt();
-    }
-
-    public long readLong() throws IOException
-    {
-        return dataReader.readLong();
-    }
-
-    public float readFloat() throws IOException
-    {
-        return dataReader.readFloat();
-    }
-
-    public double readDouble() throws IOException
-    {
-        return dataReader.readDouble();
-    }
-
-    public String readLine() throws IOException
-    {
-        return dataReader.readLine();
-    }
-
-    public String readUTF() throws IOException
-    {
-        return dataReader.readUTF();
-    }
-}
diff --git a/src/java/org/apache/cassandra/io/util/SequentialWriter.java b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
index 9ad944b..431ece3 100644
--- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.io.util;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
@@ -114,7 +113,7 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr
                 FileChannel channel = FileChannel.open(file.toPath(), StandardOpenOption.READ, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW);
                 try
                 {
-                    SyncUtil.trySyncDir(file.getParentFile());
+                    SyncUtil.trySyncDir(file.parent());
                 }
                 catch (Throwable t)
                 {
@@ -163,7 +162,7 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr
         this.strictFlushing = strictFlushing;
         this.fchannel = (FileChannel)channel;
 
-        this.filePath = file.getAbsolutePath();
+        this.filePath = file.absolutePath();
 
         this.option = option;
     }
diff --git a/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java b/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java
index bc1a529..05fdb6b 100644
--- a/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java
+++ b/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java
@@ -69,4 +69,4 @@ class SimpleChunkReader extends AbstractReaderFileProxy implements ChunkReader
                              bufferSize,
                              fileLength());
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/locator/CloudstackSnitch.java b/src/java/org/apache/cassandra/locator/CloudstackSnitch.java
index be6d3c4..d857953 100644
--- a/src/java/org/apache/cassandra/locator/CloudstackSnitch.java
+++ b/src/java/org/apache/cassandra/locator/CloudstackSnitch.java
@@ -20,9 +20,7 @@ package org.apache.cassandra.locator;
 import java.io.DataInputStream;
 import java.io.BufferedInputStream;
 import java.io.BufferedReader;
-import java.io.FileReader;
 import java.io.IOException;
-import java.io.File;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.net.URI;
@@ -31,6 +29,8 @@ import java.util.Map;
 import java.util.regex.Pattern;
 import java.util.regex.Matcher;
 
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.cassandra.db.SystemKeyspace;
diff --git a/src/java/org/apache/cassandra/locator/RangesByEndpoint.java b/src/java/org/apache/cassandra/locator/RangesByEndpoint.java
index cbf5a63..023d7ee 100644
--- a/src/java/org/apache/cassandra/locator/RangesByEndpoint.java
+++ b/src/java/org/apache/cassandra/locator/RangesByEndpoint.java
@@ -53,4 +53,4 @@ public class RangesByEndpoint extends ReplicaMultimap<InetAddressAndPort, Ranges
         }
     }
 
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/net/AsyncChannelOutputPlus.java b/src/java/org/apache/cassandra/net/AsyncChannelOutputPlus.java
index 163981c..7fcdc05 100644
--- a/src/java/org/apache/cassandra/net/AsyncChannelOutputPlus.java
+++ b/src/java/org/apache/cassandra/net/AsyncChannelOutputPlus.java
@@ -265,4 +265,4 @@ public abstract class AsyncChannelOutputPlus extends BufferedDataOutputStreamPlu
         throw new UnsupportedOperationException();
     }
 
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/net/AsyncMessageOutputPlus.java b/src/java/org/apache/cassandra/net/AsyncMessageOutputPlus.java
index 8ef0a8f..e1bcfed 100644
--- a/src/java/org/apache/cassandra/net/AsyncMessageOutputPlus.java
+++ b/src/java/org/apache/cassandra/net/AsyncMessageOutputPlus.java
@@ -128,4 +128,4 @@ public class AsyncMessageOutputPlus extends AsyncChannelOutputPlus
             buffer = null;
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java b/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
index 3a9c075..9c0f265 100644
--- a/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
+++ b/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
@@ -267,4 +267,4 @@ public class AsyncStreamingOutputPlus extends AsyncChannelOutputPlus
             buffer = null;
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/net/Verb.java b/src/java/org/apache/cassandra/net/Verb.java
index dd7fab3..9e3f5f6 100644
--- a/src/java/org/apache/cassandra/net/Verb.java
+++ b/src/java/org/apache/cassandra/net/Verb.java
@@ -448,4 +448,4 @@ class VerbTimeouts
     static final ToLongFunction<TimeUnit> pingTimeout     = DatabaseDescriptor::getPingTimeout;
     static final ToLongFunction<TimeUnit> longTimeout     = units -> Math.max(DatabaseDescriptor.getRpcTimeout(units), units.convert(5L, TimeUnit.MINUTES));
     static final ToLongFunction<TimeUnit> noTimeout       = units -> { throw new IllegalStateException(); };
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/repair/asymmetric/HostDifferences.java b/src/java/org/apache/cassandra/repair/asymmetric/HostDifferences.java
index 8b123a7..76ff5aa 100644
--- a/src/java/org/apache/cassandra/repair/asymmetric/HostDifferences.java
+++ b/src/java/org/apache/cassandra/repair/asymmetric/HostDifferences.java
@@ -99,4 +99,4 @@ public class HostDifferences
                "perHostDifferences=" + perHostDifferences +
                '}';
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/repair/asymmetric/PreferedNodeFilter.java b/src/java/org/apache/cassandra/repair/asymmetric/PreferedNodeFilter.java
index e8ca85d..b2622ef 100644
--- a/src/java/org/apache/cassandra/repair/asymmetric/PreferedNodeFilter.java
+++ b/src/java/org/apache/cassandra/repair/asymmetric/PreferedNodeFilter.java
@@ -25,4 +25,4 @@ import org.apache.cassandra.locator.InetAddressAndPort;
 public interface PreferedNodeFilter
 {
     public Set<InetAddressAndPort> apply(InetAddressAndPort streamingNode, Set<InetAddressAndPort> toStream);
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/schema/SchemaPullVerbHandler.java b/src/java/org/apache/cassandra/schema/SchemaPullVerbHandler.java
index ed30792..863c68c 100644
--- a/src/java/org/apache/cassandra/schema/SchemaPullVerbHandler.java
+++ b/src/java/org/apache/cassandra/schema/SchemaPullVerbHandler.java
@@ -44,4 +44,4 @@ public final class SchemaPullVerbHandler implements IVerbHandler<NoPayload>
         Message<Collection<Mutation>> response = message.responseWith(SchemaKeyspace.convertSchemaToMutations());
         MessagingService.instance().send(response, message.from());
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/schema/TableMetadataRef.java b/src/java/org/apache/cassandra/schema/TableMetadataRef.java
index 3325510..dc4ff1d 100644
--- a/src/java/org/apache/cassandra/schema/TableMetadataRef.java
+++ b/src/java/org/apache/cassandra/schema/TableMetadataRef.java
@@ -96,4 +96,4 @@ public final class TableMetadataRef
     {
         return get().toString();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/schema/Views.java b/src/java/org/apache/cassandra/schema/Views.java
index f926c07..15d13f3 100644
--- a/src/java/org/apache/cassandra/schema/Views.java
+++ b/src/java/org/apache/cassandra/schema/Views.java
@@ -252,4 +252,4 @@ public final class Views implements Iterable<ViewMetadata>
             return new ViewsDiff(created, dropped, altered.build());
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/security/FileBasedSslContextFactory.java b/src/java/org/apache/cassandra/security/FileBasedSslContextFactory.java
index 618af51..4912364 100644
--- a/src/java/org/apache/cassandra/security/FileBasedSslContextFactory.java
+++ b/src/java/org/apache/cassandra/security/FileBasedSslContextFactory.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.security;
 
-import java.io.File;
 import java.io.InputStream;
 import java.nio.file.Files;
 import java.nio.file.Paths;
@@ -37,6 +36,8 @@ import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.io.util.File;
+
 /**
  * Abstract implementation for {@link ISslContextFactory} using file based, standard keystore format with the ability
  * to hot-reload the files upon file changes (detected by the {@code last modified timestamp}).
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 4a53aee..16c2c2b 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.service;
 
-import java.io.File;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryPoolMXBean;
@@ -37,6 +36,7 @@ import javax.management.StandardMBean;
 import javax.management.remote.JMXConnectorServer;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.cassandra.io.util.File;
 import com.google.common.collect.ImmutableList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/src/java/org/apache/cassandra/service/DefaultFSErrorHandler.java b/src/java/org/apache/cassandra/service/DefaultFSErrorHandler.java
index d5e3e53..04cb11c 100644
--- a/src/java/org/apache/cassandra/service/DefaultFSErrorHandler.java
+++ b/src/java/org/apache/cassandra/service/DefaultFSErrorHandler.java
@@ -18,8 +18,8 @@
 
 package org.apache.cassandra.service;
 
-import java.io.File;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -78,10 +78,10 @@ public class DefaultFSErrorHandler implements FSErrorHandler
                 }
 
                 // for both read and write errors mark the path as unwritable.
-                DisallowedDirectories.maybeMarkUnwritable(e.path);
+                DisallowedDirectories.maybeMarkUnwritable(new File(e.path));
                 if (e instanceof FSReadError)
                 {
-                    File directory = DisallowedDirectories.maybeMarkUnreadable(e.path);
+                    File directory = DisallowedDirectories.maybeMarkUnreadable(new File(e.path));
                     if (directory != null)
                         Keyspace.removeUnreadableSSTables(directory);
                 }
diff --git a/src/java/org/apache/cassandra/service/FileSystemOwnershipCheck.java b/src/java/org/apache/cassandra/service/FileSystemOwnershipCheck.java
index dd68d34..fe4427c 100644
--- a/src/java/org/apache/cassandra/service/FileSystemOwnershipCheck.java
+++ b/src/java/org/apache/cassandra/service/FileSystemOwnershipCheck.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.service;
 
 import java.io.BufferedReader;
-import java.io.File;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -36,6 +35,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.StartupException;
+import org.apache.cassandra.io.util.File;
 
 /**
  * Ownership markers on disk are compatible with the java property file format.
@@ -103,7 +103,7 @@ public class FileSystemOwnershipCheck implements StartupCheck
         this(() -> Iterables.concat(Arrays.asList(DatabaseDescriptor.getAllDataFileLocations()),
                                     Arrays.asList(DatabaseDescriptor.getCommitLogLocation(),
                                                   DatabaseDescriptor.getSavedCachesLocation(),
-                                                  DatabaseDescriptor.getHintsDirectory().getAbsolutePath())));
+                                                  DatabaseDescriptor.getHintsDirectory().absolutePath())));
     }
 
     @VisibleForTesting
@@ -251,7 +251,7 @@ public class FileSystemOwnershipCheck implements StartupCheck
     {
         try
         {
-            return dir.resolve(filename).toFile();
+            return new File(dir.resolve(filename));
         }
         catch (Exception e)
         {
diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java b/src/java/org/apache/cassandra/service/StartupChecks.java
index 4f9b82f..5cb938b 100644
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.service;
 
 import java.io.BufferedReader;
-import java.io.File;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.management.RuntimeMXBean;
@@ -32,12 +31,14 @@ import com.google.common.base.Joiner;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import net.jpountz.lz4.LZ4Factory;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.io.util.PathUtils;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -339,8 +340,7 @@ public class StartupChecks
         Iterable<String> dirs = Iterables.concat(Arrays.asList(DatabaseDescriptor.getAllDataFileLocations()),
                                                  Arrays.asList(DatabaseDescriptor.getCommitLogLocation(),
                                                                DatabaseDescriptor.getSavedCachesLocation(),
-                                                               DatabaseDescriptor.getHintsDirectory().getAbsolutePath()));
-
+                                                               DatabaseDescriptor.getHintsDirectory().absolutePath()));
         for (String dataDir : dirs)
         {
             logger.debug("Checking directory {}", dataDir);
@@ -351,7 +351,7 @@ public class StartupChecks
             {
                 logger.warn("Directory {} doesn't exist", dataDir);
                 // if they don't, failing their creation, stop cassandra.
-                if (!dir.mkdirs())
+                if (!dir.tryCreateDirectories())
                     throw new StartupException(StartupException.ERR_WRONG_DISK_STATE,
                                                "Has no permission to create directory "+ dataDir);
             }
@@ -377,7 +377,7 @@ public class StartupChecks
             {
                 public FileVisitResult visitFile(Path path, BasicFileAttributes attrs)
                 {
-                    File file = path.toFile();
+                    File file = new File(path);
                     if (!Descriptor.isValidFile(file))
                         return FileVisitResult.CONTINUE;
 
@@ -398,7 +398,7 @@ public class StartupChecks
                     String name = dir.getFileName().toString();
                     return (name.equals(Directories.SNAPSHOT_SUBDIR)
                             || name.equals(Directories.BACKUPS_SUBDIR)
-                            || nonSSTablePaths.contains(dir.toFile().getCanonicalPath()))
+                            || nonSSTablePaths.contains(PathUtils.toCanonicalPath(dir).toString()))
                            ? FileVisitResult.SKIP_SUBTREE
                            : FileVisitResult.CONTINUE;
                 }
@@ -408,7 +408,7 @@ public class StartupChecks
             {
                 try
                 {
-                    Files.walkFileTree(Paths.get(dataDir), sstableVisitor);
+                    Files.walkFileTree(new File(dataDir).toPath(), sstableVisitor);
                 }
                 catch (IOException e)
                 {
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index b41ebea..06c685c 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -17,7 +17,11 @@
  */
 package org.apache.cassandra.service;
 
-import java.io.*;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOError;
+import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
@@ -61,6 +65,7 @@ import org.apache.cassandra.dht.RangeStreamer.FetchReplica;
 import org.apache.cassandra.fql.FullQueryLogger;
 import org.apache.cassandra.fql.FullQueryLoggerOptions;
 import org.apache.cassandra.fql.FullQueryLoggerOptionsCompositeData;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict;
 import org.apache.cassandra.utils.concurrent.Future;
 import org.apache.cassandra.utils.concurrent.ImmediateFuture;
@@ -3990,7 +3995,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         Set<String> keyspaces = new HashSet<>();
         for (String dataDir : DatabaseDescriptor.getAllDataFileLocations())
         {
-            for(String keyspaceDir : new File(dataDir).list())
+            for(String keyspaceDir : new File(dataDir).tryListNames())
             {
                 // Only add a ks if it has been specified as a param, assuming params were actually provided.
                 if (keyspaceNames.length > 0 && !Arrays.asList(keyspaceNames).contains(keyspaceDir))
diff --git a/src/java/org/apache/cassandra/service/pager/MultiPartitionPager.java b/src/java/org/apache/cassandra/service/pager/MultiPartitionPager.java
index 7e64e03..bc2c79e 100644
--- a/src/java/org/apache/cassandra/service/pager/MultiPartitionPager.java
+++ b/src/java/org/apache/cassandra/service/pager/MultiPartitionPager.java
@@ -232,4 +232,4 @@ public class MultiPartitionPager<T extends SinglePartitionReadQuery> implements
     {
         return remaining;
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java
index 554bd51..7f1c4e9 100644
--- a/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java
+++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.service.snapshot;
 
 
-import java.io.File;
 import java.time.Instant;
 import java.util.Collection;
 import java.util.Comparator;
@@ -42,6 +41,7 @@ import java.util.stream.StreamSupport;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.utils.ExecutorUtils;
 import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 
diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotManifest.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotManifest.java
index 5d44acb..d8d900a 100644
--- a/src/java/org/apache/cassandra/service/snapshot/SnapshotManifest.java
+++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotManifest.java
@@ -18,8 +18,7 @@
 
 package org.apache.cassandra.service.snapshot;
 
-import java.io.File;
-import java.io.IOException;
+import java.io.*;
 import java.time.Instant;
 import java.util.List;
 import java.util.Objects;
@@ -30,6 +29,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.cassandra.config.Duration;
 import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule;
 import com.fasterxml.jackson.databind.DeserializationFeature;
+import org.apache.cassandra.io.util.File;
 
 // Only serialize fields
 @JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.ANY,
@@ -84,12 +84,12 @@ public class SnapshotManifest
 
     public void serializeToJsonFile(File outputFile) throws IOException
     {
-        mapper.writeValue(outputFile, this);
+        mapper.writeValue(outputFile.toJavaIOFile(), this);
     }
 
     public static SnapshotManifest deserializeFromJsonFile(File file) throws IOException
     {
-        return mapper.readValue(file, SnapshotManifest.class);
+        return mapper.readValue(file.toJavaIOFile(), SnapshotManifest.class);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java b/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java
index 7e852ec..185cd45 100644
--- a/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java
+++ b/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.service.snapshot;
 
-import java.io.File;
 import java.time.Instant;
 import java.util.Collection;
 import java.util.Map;
@@ -26,6 +25,7 @@ import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.io.util.FileUtils;
 
 public class TableSnapshot
diff --git a/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java b/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java
index e378af7..bce47a1 100644
--- a/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java
@@ -23,6 +23,7 @@ import java.util.Collection;
 
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.streaming.StreamRequest;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.streaming.StreamSummary;
diff --git a/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java b/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
index ff2cdec..a1dd03c 100644
--- a/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
@@ -21,6 +21,7 @@ import java.io.*;
 
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.streaming.StreamSession;
 
diff --git a/src/java/org/apache/cassandra/tools/AuditLogViewer.java b/src/java/org/apache/cassandra/tools/AuditLogViewer.java
index dd0e839..f226aa2 100644
--- a/src/java/org/apache/cassandra/tools/AuditLogViewer.java
+++ b/src/java/org/apache/cassandra/tools/AuditLogViewer.java
@@ -17,12 +17,12 @@
  */
 package org.apache.cassandra.tools;
 
-import java.io.File;
 import java.util.Arrays;
 import java.util.List;
 import java.util.function.Consumer;
 import java.util.stream.Collectors;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -76,7 +76,7 @@ public class AuditLogViewer
         Pauser pauser = Pauser.millis(100);
         List<ExcerptTailer> tailers = pathList.stream()
                                               .distinct()
-                                              .map(path -> SingleChronicleQueueBuilder.single(new File(path)).readOnly(true).rollCycle(RollCycles.valueOf(rollCycle)).build())
+                                              .map(path -> SingleChronicleQueueBuilder.single(new File(path).toJavaIOFile()).readOnly(true).rollCycle(RollCycles.valueOf(rollCycle)).build())
                                               .map(SingleChronicleQueue::createTailer)
                                               .collect(Collectors.toList());
         boolean hadWork = true;
diff --git a/src/java/org/apache/cassandra/tools/BulkLoader.java b/src/java/org/apache/cassandra/tools/BulkLoader.java
index bb29bbe..43df49c 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoader.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoader.java
@@ -58,7 +58,7 @@ public class BulkLoader
         DatabaseDescriptor.toolInitialization();
         OutputHandler handler = new OutputHandler.SystemOutput(options.verbose, options.debug);
         SSTableLoader loader = new SSTableLoader(
-                options.directory.getAbsoluteFile(),
+                options.directory.toAbsolute(),
                 new ExternalClient(
                         options.hosts,
                         options.storagePort,
diff --git a/src/java/org/apache/cassandra/tools/JMXTool.java b/src/java/org/apache/cassandra/tools/JMXTool.java
index e917179..d054716 100644
--- a/src/java/org/apache/cassandra/tools/JMXTool.java
+++ b/src/java/org/apache/cassandra/tools/JMXTool.java
@@ -18,8 +18,6 @@
 
 package org.apache.cassandra.tools;
 
-import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -71,6 +69,8 @@ import io.airlift.airline.Command;
 import io.airlift.airline.Help;
 import io.airlift.airline.HelpOption;
 import io.airlift.airline.Option;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
 import org.yaml.snakeyaml.TypeDescription;
 import org.yaml.snakeyaml.Yaml;
 import org.yaml.snakeyaml.constructor.Constructor;
@@ -229,8 +229,8 @@ public class JMXTool
             Preconditions.checkArgument(files.size() == 2, "files requires 2 arguments but given %s", files);
             Map<String, Info> left;
             Map<String, Info> right;
-            try (FileInputStream leftStream = new FileInputStream(files.get(0));
-                 FileInputStream rightStream = new FileInputStream(files.get(1)))
+            try (FileInputStreamPlus leftStream = new FileInputStreamPlus(files.get(0));
+                 FileInputStreamPlus rightStream = new FileInputStreamPlus(files.get(1)))
             {
                 left = format.load(leftStream);
                 right = format.load(rightStream);
diff --git a/src/java/org/apache/cassandra/tools/LoaderOptions.java b/src/java/org/apache/cassandra/tools/LoaderOptions.java
index ca1bd40..62f5046 100644
--- a/src/java/org/apache/cassandra/tools/LoaderOptions.java
+++ b/src/java/org/apache/cassandra/tools/LoaderOptions.java
@@ -20,7 +20,6 @@
  */
 package org.apache.cassandra.tools;
 
-import java.io.File;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.net.*;
@@ -32,6 +31,7 @@ import com.google.common.net.HostAndPort;
 
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.tools.BulkLoader.CmdLineOptions;
 
@@ -376,7 +376,7 @@ public class LoaderOptions
                     {
                         errorMsg("Config file not found", options);
                     }
-                    config = new YamlConfigurationLoader().loadConfig(configFile.toURI().toURL());
+                    config = new YamlConfigurationLoader().loadConfig(configFile.toPath().toUri().toURL());
                 }
                 else
                 {
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index 91a008e..7d3a3ca 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -22,18 +22,20 @@ import static com.google.common.collect.Iterables.toArray;
 import static com.google.common.collect.Lists.newArrayList;
 import static java.lang.Integer.parseInt;
 import static java.lang.String.format;
+import static org.apache.cassandra.io.util.File.WriteMode.APPEND;
 import static org.apache.commons.lang3.ArrayUtils.EMPTY_STRING_ARRAY;
 import static org.apache.commons.lang3.StringUtils.EMPTY;
 import static org.apache.commons.lang3.StringUtils.isEmpty;
 import static org.apache.commons.lang3.StringUtils.isNotEmpty;
 
 import java.io.Console;
-import java.io.File;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileWriter;
 import java.io.FileNotFoundException;
-import java.io.FileWriter;
 import java.io.IOError;
 import java.io.IOException;
 import java.net.UnknownHostException;
+import java.nio.file.NoSuchFileException;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -278,7 +280,7 @@ public class NodeTool
         String cmdLine = Joiner.on(" ").skipNulls().join(args);
         cmdLine = cmdLine.replaceFirst("(?<=(-pw|--password))\\s+\\S+", " <hidden>");
 
-        try (FileWriter writer = new FileWriter(new File(FBUtilities.getToolsOutputDirectory(), HISTORYFILE), true))
+        try (FileWriter writer = new File(FBUtilities.getToolsOutputDirectory(), HISTORYFILE).newWriter(APPEND))
         {
             SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS");
             writer.append(sdf.format(new Date())).append(": ").append(cmdLine).append(System.lineSeparator());
@@ -374,7 +376,7 @@ public class NodeTool
             String password = EMPTY;
 
             File passwordFile = new File(passwordFilePath);
-            try (Scanner scanner = new Scanner(passwordFile).useDelimiter("\\s+"))
+            try (Scanner scanner = new Scanner(passwordFile.toJavaIOFile()).useDelimiter("\\s+"))
             {
                 while (scanner.hasNextLine())
                 {
@@ -389,7 +391,8 @@ public class NodeTool
                     }
                     scanner.nextLine();
                 }
-            } catch (FileNotFoundException e)
+            }
+            catch (FileNotFoundException e)
             {
                 throw new RuntimeException(e);
             }
diff --git a/src/java/org/apache/cassandra/tools/SSTableExport.java b/src/java/org/apache/cassandra/tools/SSTableExport.java
index 5be67d7..b3000d0 100644
--- a/src/java/org/apache/cassandra/tools/SSTableExport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableExport.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.tools;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashSet;
@@ -26,6 +25,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.DecoratedKey;
@@ -135,7 +135,7 @@ public class SSTableExport
             printUsage();
             System.exit(1);
         }
-        String ssTableFileName = new File(cmd.getArgs()[0]).getAbsolutePath();
+        String ssTableFileName = new File(cmd.getArgs()[0]).absolutePath();
 
         if (!new File(ssTableFileName).exists())
         {
diff --git a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
old mode 100755
new mode 100644
index 8c1f5db..b7164e8
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@ -25,7 +25,6 @@ import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
 import static org.apache.commons.lang3.time.DurationFormatUtils.formatDurationWords;
 
 import java.io.DataInputStream;
-import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintWriter;
@@ -60,6 +59,7 @@ import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
 import org.apache.cassandra.io.sstable.metadata.MetadataType;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
 import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.tools.Util.TermHistogram;
@@ -544,7 +544,7 @@ public class SSTableMetadataViewer
             File sstable = new File(fname);
             if (sstable.exists())
             {
-                metawriter.printSStableMetadata(sstable.getAbsolutePath(), fullScan);
+                metawriter.printSStableMetadata(sstable.absolutePath(), fullScan);
             }
             else
             {
diff --git a/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java b/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java
index 79fec81..72c1e99 100644
--- a/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java
+++ b/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.tools;
 
-import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
@@ -35,6 +34,7 @@ import com.google.common.collect.Multimap;
 import com.google.common.collect.SetMultimap;
 
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
diff --git a/src/java/org/apache/cassandra/tools/SSTableRepairedAtSetter.java b/src/java/org/apache/cassandra/tools/SSTableRepairedAtSetter.java
index 31d80fa..62dd76e 100644
--- a/src/java/org/apache/cassandra/tools/SSTableRepairedAtSetter.java
+++ b/src/java/org/apache/cassandra/tools/SSTableRepairedAtSetter.java
@@ -17,7 +17,9 @@
  */
 package org.apache.cassandra.tools;
 
-import java.io.*;
+
+import java.io.IOException;
+import java.io.PrintStream;
 import java.nio.charset.Charset;
 import java.nio.file.Files;
 import java.nio.file.Paths;
@@ -39,6 +41,8 @@ import org.apache.cassandra.io.sstable.Descriptor;
  * sstablerepairset --is-repaired -f <(find /var/lib/cassandra/data/.../ -iname "*Data.db*" -mtime +14)
  * }
  */
+import org.apache.cassandra.io.util.File;
+
 public class SSTableRepairedAtSetter
 {
     /**
diff --git a/src/java/org/apache/cassandra/tools/StandaloneSSTableUtil.java b/src/java/org/apache/cassandra/tools/StandaloneSSTableUtil.java
index cca48fc..06618b3 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneSSTableUtil.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneSSTableUtil.java
@@ -25,10 +25,10 @@ import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.utils.OutputHandler;
 import org.apache.commons.cli.*;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.function.BiPredicate;
 
+import org.apache.cassandra.io.util.File;
 import static org.apache.cassandra.tools.BulkLoader.CmdLineOptions;
 
 public class StandaloneSSTableUtil
@@ -87,7 +87,7 @@ public class StandaloneSSTableUtil
         for (File dir : directories.getCFDirectories())
         {
             for (File file : LifecycleTransaction.getFiles(dir.toPath(), getFilter(options), Directories.OnTxnErr.THROW))
-                handler.output(file.getCanonicalPath());
+                handler.output(file.canonicalPath());
         }
     }
 
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index 4dfa4ab..6ee320e 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -18,7 +18,6 @@
  */
 package org.apache.cassandra.tools;
 
-import java.io.File;
 import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -27,6 +26,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -127,7 +127,7 @@ public class StandaloneScrubber
                 listResult.add(Pair.create(descriptor, components));
 
                 File snapshotDirectory = Directories.getSnapshotDirectory(descriptor, snapshotName);
-                SSTableReader.createLinks(descriptor, components, snapshotDirectory.getPath());
+                SSTableReader.createLinks(descriptor, components, snapshotDirectory.path());
             }
             System.out.println(String.format("Pre-scrub sstables snapshotted into snapshot %s", snapshotName));
 
diff --git a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
index e3c80f1..cd3affa 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
@@ -18,10 +18,10 @@
  */
 package org.apache.cassandra.tools;
 
-import java.io.File;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.cli.*;
@@ -133,7 +133,7 @@ public class StandaloneSplitter
 
                     if (options.snapshot) {
                         File snapshotDirectory = Directories.getSnapshotDirectory(sstable.descriptor, snapshotName);
-                        sstable.createLinks(snapshotDirectory.getPath());
+                        sstable.createLinks(snapshotDirectory.path());
                     }
 
                 }
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Assassinate.java b/src/java/org/apache/cassandra/tools/nodetool/Assassinate.java
index a075ded..2639ec8 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Assassinate.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Assassinate.java
@@ -44,4 +44,4 @@ public class Assassinate extends NodeToolCmd
             throw new RuntimeException(e);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/CfHistograms.java b/src/java/org/apache/cassandra/tools/nodetool/CfHistograms.java
index 8fdf803..6a06fd4 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/CfHistograms.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/CfHistograms.java
@@ -26,4 +26,4 @@ import io.airlift.airline.Command;
 @Deprecated
 public class CfHistograms extends TableHistograms
 {
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/CfStats.java b/src/java/org/apache/cassandra/tools/nodetool/CfStats.java
index 2d27ea0..42e2bc3 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/CfStats.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/CfStats.java
@@ -26,4 +26,4 @@ import io.airlift.airline.Command;
 @Deprecated
 public class CfStats extends TableStats
 {
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Compact.java b/src/java/org/apache/cassandra/tools/nodetool/Compact.java
index 7278ead..ca560cd 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Compact.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Compact.java
@@ -94,4 +94,4 @@ public class Compact extends NodeToolCmd
             }
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Decommission.java b/src/java/org/apache/cassandra/tools/nodetool/Decommission.java
index 0e58687..98b6d58 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Decommission.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Decommission.java
@@ -46,4 +46,4 @@ public class Decommission extends NodeToolCmd
             throw new IllegalStateException("Unsupported operation: " + e.getMessage(), e);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableAuditLog.java b/src/java/org/apache/cassandra/tools/nodetool/DisableAuditLog.java
index 35653ae..6d878a0 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/DisableAuditLog.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableAuditLog.java
@@ -30,4 +30,4 @@ public class DisableAuditLog extends NodeToolCmd
     {
         probe.disableAuditLog();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableAutoCompaction.java b/src/java/org/apache/cassandra/tools/nodetool/DisableAutoCompaction.java
index b9fc7d6..39a4c76 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/DisableAutoCompaction.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableAutoCompaction.java
@@ -50,4 +50,4 @@ public class DisableAutoCompaction extends NodeToolCmd
             }
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableBackup.java b/src/java/org/apache/cassandra/tools/nodetool/DisableBackup.java
index 4b0bfbe..4ee6340 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/DisableBackup.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableBackup.java
@@ -30,4 +30,4 @@ public class DisableBackup extends NodeToolCmd
     {
         probe.setIncrementalBackupsEnabled(false);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableBinary.java b/src/java/org/apache/cassandra/tools/nodetool/DisableBinary.java
index 463f2b0..79b9219 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/DisableBinary.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableBinary.java
@@ -30,4 +30,4 @@ public class DisableBinary extends NodeToolCmd
     {
         probe.stopNativeTransport();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableFullQueryLog.java b/src/java/org/apache/cassandra/tools/nodetool/DisableFullQueryLog.java
index 8820e5f..aa5d0b7 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/DisableFullQueryLog.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableFullQueryLog.java
@@ -30,4 +30,4 @@ public class DisableFullQueryLog extends NodeToolCmd
     {
         probe.stopFullQueryLogger();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableGossip.java b/src/java/org/apache/cassandra/tools/nodetool/DisableGossip.java
index 6f950bb..7b6c348 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/DisableGossip.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableGossip.java
@@ -30,4 +30,4 @@ public class DisableGossip extends NodeToolCmd
     {
         probe.stopGossiping();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableHandoff.java b/src/java/org/apache/cassandra/tools/nodetool/DisableHandoff.java
index d7ec35f..62465a3 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/DisableHandoff.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableHandoff.java
@@ -30,4 +30,4 @@ public class DisableHandoff extends NodeToolCmd
     {
         probe.disableHintedHandoff();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableHintsForDC.java b/src/java/org/apache/cassandra/tools/nodetool/DisableHintsForDC.java
index d65c70b..3615a99 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/DisableHintsForDC.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableHintsForDC.java
@@ -39,4 +39,4 @@ public class DisableHintsForDC extends NodeTool.NodeToolCmd
 
         probe.disableHintsForDC(args.get(0));
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableOldProtocolVersions.java b/src/java/org/apache/cassandra/tools/nodetool/DisableOldProtocolVersions.java
index 2083062..8756471 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/DisableOldProtocolVersions.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableOldProtocolVersions.java
@@ -30,4 +30,4 @@ public class DisableOldProtocolVersions extends NodeTool.NodeToolCmd
     {
         probe.disableOldProtocolVersions();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Drain.java b/src/java/org/apache/cassandra/tools/nodetool/Drain.java
index eaa537a..a152057 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Drain.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Drain.java
@@ -39,4 +39,4 @@ public class Drain extends NodeToolCmd
             throw new RuntimeException("Error occurred during flushing", e);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableAuditLog.java b/src/java/org/apache/cassandra/tools/nodetool/EnableAuditLog.java
index 5149876..ae0bb42 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/EnableAuditLog.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/EnableAuditLog.java
@@ -82,4 +82,4 @@ public class EnableAuditLog extends NodeToolCmd
         probe.enableAuditLog(logger, Collections.EMPTY_MAP, included_keyspaces, excluded_keyspaces, included_categories, excluded_categories, included_users, excluded_users,
                              archiveRetries, bblocking, rollCycle, maxLogSize, maxQueueWeight, archiveCommand);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableAutoCompaction.java b/src/java/org/apache/cassandra/tools/nodetool/EnableAutoCompaction.java
index 795ab13..f8b98ff 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/EnableAutoCompaction.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/EnableAutoCompaction.java
@@ -50,4 +50,4 @@ public class EnableAutoCompaction extends NodeToolCmd
             }
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableBackup.java b/src/java/org/apache/cassandra/tools/nodetool/EnableBackup.java
index d1773d9..7ebad8a 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/EnableBackup.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/EnableBackup.java
@@ -30,4 +30,4 @@ public class EnableBackup extends NodeToolCmd
     {
         probe.setIncrementalBackupsEnabled(true);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableBinary.java b/src/java/org/apache/cassandra/tools/nodetool/EnableBinary.java
index 506945f..2e37e6f 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/EnableBinary.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/EnableBinary.java
@@ -30,4 +30,4 @@ public class EnableBinary extends NodeToolCmd
     {
         probe.startNativeTransport();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableFullQueryLog.java b/src/java/org/apache/cassandra/tools/nodetool/EnableFullQueryLog.java
index 9873e5a..5084894 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/EnableFullQueryLog.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/EnableFullQueryLog.java
@@ -61,4 +61,4 @@ public class EnableFullQueryLog extends NodeToolCmd
         }
         probe.enableFullQueryLogger(path, rollCycle, bblocking, maxQueueWeight, maxLogSize, archiveCommand, archiveRetries);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableGossip.java b/src/java/org/apache/cassandra/tools/nodetool/EnableGossip.java
index 900c427..3433c3e 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/EnableGossip.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/EnableGossip.java
@@ -30,4 +30,4 @@ public class EnableGossip extends NodeToolCmd
     {
         probe.startGossiping();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableHandoff.java b/src/java/org/apache/cassandra/tools/nodetool/EnableHandoff.java
index bccf7e7..be64e12 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/EnableHandoff.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/EnableHandoff.java
@@ -30,4 +30,4 @@ public class EnableHandoff extends NodeToolCmd
     {
         probe.enableHintedHandoff();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableOldProtocolVersions.java b/src/java/org/apache/cassandra/tools/nodetool/EnableOldProtocolVersions.java
index f6d5be5..06c9f8d 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/EnableOldProtocolVersions.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/EnableOldProtocolVersions.java
@@ -31,4 +31,4 @@ public class EnableOldProtocolVersions extends NodeTool.NodeToolCmd
     {
         probe.enableOldProtocolVersions();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Flush.java b/src/java/org/apache/cassandra/tools/nodetool/Flush.java
index c83e420..fb2446d 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Flush.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Flush.java
@@ -49,4 +49,4 @@ public class Flush extends NodeToolCmd
             }
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetSnapshotThrottle.java b/src/java/org/apache/cassandra/tools/nodetool/GetSnapshotThrottle.java
index 0e9bdc1..bd98d34 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/GetSnapshotThrottle.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetSnapshotThrottle.java
@@ -33,4 +33,4 @@ public class GetSnapshotThrottle extends NodeToolCmd
         else
             System.out.println("Snapshot throttle is disabled");
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/HostStat.java b/src/java/org/apache/cassandra/tools/nodetool/HostStat.java
index 19c0448..56c46ee 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/HostStat.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/HostStat.java
@@ -38,4 +38,4 @@ public class HostStat
     {
         return resolveIp ? endpoint.getHostName() : endpoint.getHostAddress();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/InvalidateCounterCache.java b/src/java/org/apache/cassandra/tools/nodetool/InvalidateCounterCache.java
index aef77bd..3cba8e0 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/InvalidateCounterCache.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/InvalidateCounterCache.java
@@ -30,4 +30,4 @@ public class InvalidateCounterCache extends NodeToolCmd
     {
         probe.invalidateCounterCache();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/InvalidateKeyCache.java b/src/java/org/apache/cassandra/tools/nodetool/InvalidateKeyCache.java
index cfe7d2f..4414b42 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/InvalidateKeyCache.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/InvalidateKeyCache.java
@@ -30,4 +30,4 @@ public class InvalidateKeyCache extends NodeToolCmd
     {
         probe.invalidateKeyCache();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/InvalidateRowCache.java b/src/java/org/apache/cassandra/tools/nodetool/InvalidateRowCache.java
index 7357e27..1a10ed0 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/InvalidateRowCache.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/InvalidateRowCache.java
@@ -30,4 +30,4 @@ public class InvalidateRowCache extends NodeToolCmd
     {
         probe.invalidateRowCache();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Move.java b/src/java/org/apache/cassandra/tools/nodetool/Move.java
index 8654d25..075e008 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Move.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Move.java
@@ -43,4 +43,4 @@ public class Move extends NodeToolCmd
             throw new RuntimeException("Error during moving node", e);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/PauseHandoff.java b/src/java/org/apache/cassandra/tools/nodetool/PauseHandoff.java
index 4ec70d8..fde9eef 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/PauseHandoff.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/PauseHandoff.java
@@ -30,4 +30,4 @@ public class PauseHandoff extends NodeToolCmd
     {
         probe.pauseHintsDelivery();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Rebuild.java b/src/java/org/apache/cassandra/tools/nodetool/Rebuild.java
index a083cde..a16e8f2 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Rebuild.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Rebuild.java
@@ -57,4 +57,4 @@ public class Rebuild extends NodeToolCmd
 
         probe.rebuild(sourceDataCenterName, keyspace, tokens, specificSources);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/RebuildIndex.java b/src/java/org/apache/cassandra/tools/nodetool/RebuildIndex.java
index 4a6b071..f7a3b6f 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/RebuildIndex.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/RebuildIndex.java
@@ -40,4 +40,4 @@ public class RebuildIndex extends NodeToolCmd
         checkArgument(args.size() >= 3, "rebuild_index requires ks, cf and idx args");
         probe.rebuildIndex(args.get(0), args.get(1), toArray(args.subList(2, args.size()), String.class));
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/ReloadTriggers.java b/src/java/org/apache/cassandra/tools/nodetool/ReloadTriggers.java
index 6ca90fb..8727a61 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/ReloadTriggers.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/ReloadTriggers.java
@@ -30,4 +30,4 @@ public class ReloadTriggers extends NodeToolCmd
     {
         probe.reloadTriggers();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/ResetFullQueryLog.java b/src/java/org/apache/cassandra/tools/nodetool/ResetFullQueryLog.java
index 786852d..d7ac301 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/ResetFullQueryLog.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/ResetFullQueryLog.java
@@ -30,4 +30,4 @@ public class ResetFullQueryLog extends NodeToolCmd
     {
         probe.resetFullQueryLogger();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/ResetLocalSchema.java b/src/java/org/apache/cassandra/tools/nodetool/ResetLocalSchema.java
index 708636f..62775a4 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/ResetLocalSchema.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/ResetLocalSchema.java
@@ -38,4 +38,4 @@ public class ResetLocalSchema extends NodeToolCmd
             throw new RuntimeException(e);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/ResumeHandoff.java b/src/java/org/apache/cassandra/tools/nodetool/ResumeHandoff.java
index a3984f8..bda98aa 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/ResumeHandoff.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/ResumeHandoff.java
@@ -30,4 +30,4 @@ public class ResumeHandoff extends NodeToolCmd
     {
         probe.resumeHintsDelivery();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetBatchlogReplayThrottle.java b/src/java/org/apache/cassandra/tools/nodetool/SetBatchlogReplayThrottle.java
index 65bb8f5..3c6370c 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/SetBatchlogReplayThrottle.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetBatchlogReplayThrottle.java
@@ -34,4 +34,4 @@ public class SetBatchlogReplayThrottle extends NodeToolCmd
     {
         probe.setBatchlogReplayThrottle(batchlogReplayThrottle);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetCacheCapacity.java b/src/java/org/apache/cassandra/tools/nodetool/SetCacheCapacity.java
index 461f6ae..b07eb9e 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/SetCacheCapacity.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetCacheCapacity.java
@@ -42,4 +42,4 @@ public class SetCacheCapacity extends NodeToolCmd
         checkArgument(args.size() == 3, "setcachecapacity requires key-cache-capacity, row-cache-capacity, and counter-cache-capacity args.");
         probe.setCacheCapacities(args.get(0), args.get(1), args.get(2));
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetCacheKeysToSave.java b/src/java/org/apache/cassandra/tools/nodetool/SetCacheKeysToSave.java
index 18197e6..de9bab5 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/SetCacheKeysToSave.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetCacheKeysToSave.java
@@ -42,4 +42,4 @@ public class SetCacheKeysToSave extends NodeToolCmd
         checkArgument(args.size() == 3, "setcachekeystosave requires key-cache-keys-to-save, row-cache-keys-to-save, and counter-cache-keys-to-save args.");
         probe.setCacheKeysToSave(args.get(0), args.get(1), args.get(2));
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetCompactionThreshold.java b/src/java/org/apache/cassandra/tools/nodetool/SetCompactionThreshold.java
index 56e558f..52bb5bc 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/SetCompactionThreshold.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetCompactionThreshold.java
@@ -47,4 +47,4 @@ public class SetCompactionThreshold extends NodeToolCmd
 
         probe.setCompactionThreshold(args.get(0), args.get(1), minthreshold, maxthreshold);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetCompactionThroughput.java b/src/java/org/apache/cassandra/tools/nodetool/SetCompactionThroughput.java
index 80e7222..4d01f61 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/SetCompactionThroughput.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetCompactionThroughput.java
@@ -34,4 +34,4 @@ public class SetCompactionThroughput extends NodeToolCmd
     {
         probe.setCompactionThroughput(compactionThroughput);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetHintedHandoffThrottleInKB.java b/src/java/org/apache/cassandra/tools/nodetool/SetHintedHandoffThrottleInKB.java
index feb945b..96f1bdf 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/SetHintedHandoffThrottleInKB.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetHintedHandoffThrottleInKB.java
@@ -34,4 +34,4 @@ public class SetHintedHandoffThrottleInKB extends NodeToolCmd
     {
         probe.setHintedHandoffThrottleInKB(throttleInKB);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetHostStat.java b/src/java/org/apache/cassandra/tools/nodetool/SetHostStat.java
index c43abe1..1160876 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/SetHostStat.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetHostStat.java
@@ -51,4 +51,4 @@ public class SetHostStat implements Iterable<HostStat>
         Float owns = ownerships.get(endpoint);
         hostStats.add(new HostStat(token, endpoint, resolveIp, owns));
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetLoggingLevel.java b/src/java/org/apache/cassandra/tools/nodetool/SetLoggingLevel.java
index 8d9ad90..66d6283 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/SetLoggingLevel.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetLoggingLevel.java
@@ -100,4 +100,4 @@ public class SetLoggingLevel extends NodeToolCmd
         for (String classQualifier : classQualifiers)
             probe.setLoggingLevel(classQualifier, level);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetSnapshotThrottle.java b/src/java/org/apache/cassandra/tools/nodetool/SetSnapshotThrottle.java
index a4c49b8..045ccc1 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/SetSnapshotThrottle.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetSnapshotThrottle.java
@@ -33,4 +33,4 @@ public class SetSnapshotThrottle extends NodeToolCmd
     {
         probe.setSnapshotLinksPerSecond(snapshotThrottle);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetStreamThroughput.java b/src/java/org/apache/cassandra/tools/nodetool/SetStreamThroughput.java
index 469ec95..c462845 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/SetStreamThroughput.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetStreamThroughput.java
@@ -35,4 +35,4 @@ public class SetStreamThroughput extends NodeToolCmd
     {
         probe.setStreamThroughput(streamThroughput);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetTraceProbability.java b/src/java/org/apache/cassandra/tools/nodetool/SetTraceProbability.java
index e081980..ef9f498 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/SetTraceProbability.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetTraceProbability.java
@@ -36,4 +36,4 @@ public class SetTraceProbability extends NodeToolCmd
         checkArgument(traceProbability >= 0 && traceProbability <= 1, "Trace probability must be between 0 and 1");
         probe.setTraceProbability(traceProbability);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Sjk.java b/src/java/org/apache/cassandra/tools/nodetool/Sjk.java
index 3ad2c94..d7f7a04 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Sjk.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Sjk.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.tools.nodetool;
 
-import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.lang.reflect.Field;
@@ -57,6 +56,7 @@ import com.beust.jcommander.ParameterDescription;
 import com.beust.jcommander.Parameterized;
 import io.airlift.airline.Arguments;
 import io.airlift.airline.Command;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.tools.Output;
 import org.gridkit.jvmtool.JmxConnectionInfo;
 import org.gridkit.jvmtool.cli.CommandLauncher;
@@ -464,15 +464,15 @@ public class Sjk extends NodeToolCmd
             {
                 // loop through files in classpath
                 File dir = new File(packageURL.getFile());
-                String cp = dir.getCanonicalPath();
+                String cp = dir.canonicalPath();
                 File root = dir;
                 while (true)
                 {
-                    if (cp.equals(new File(root, path).getCanonicalPath()))
+                    if (cp.equals(new File(root, path).canonicalPath()))
                     {
                         break;
                     }
-                    root = root.getParentFile();
+                    root = root.parent();
                 }
                 listFiles(results, root, dir);
             }
@@ -480,10 +480,10 @@ public class Sjk extends NodeToolCmd
 
         static void listFiles(List<String> names, File root, File dir)
         {
-            String rootPath = root.getAbsolutePath();
+            String rootPath = root.absolutePath();
             if (dir.exists() && dir.isDirectory())
             {
-                for (File file : dir.listFiles())
+                for (File file : dir.tryList())
                 {
                     if (file.isDirectory())
                     {
@@ -491,7 +491,7 @@ public class Sjk extends NodeToolCmd
                     }
                     else
                     {
-                        String name = file.getAbsolutePath().substring(rootPath.length() + 1);
+                        String name = file.absolutePath().substring(rootPath.length() + 1);
                         name = name.replace('\\', '/');
                         names.add(name);
                     }
diff --git a/src/java/org/apache/cassandra/tools/nodetool/TruncateHints.java b/src/java/org/apache/cassandra/tools/nodetool/TruncateHints.java
index a3a0049..2a19d3a 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/TruncateHints.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/TruncateHints.java
@@ -38,4 +38,4 @@ public class TruncateHints extends NodeToolCmd
         else
             probe.truncateHints(endpoint);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsHolder.java b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsHolder.java
index c35e1fe..a345ce0 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsHolder.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsHolder.java
@@ -26,4 +26,4 @@ import java.util.Map;
 public interface StatsHolder
 {
     public Map<String, Object> convert2Map();
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsKeyspace.java b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsKeyspace.java
index dc15332..89d7705 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsKeyspace.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsKeyspace.java
@@ -75,4 +75,4 @@ public class StatsKeyspace
                ? totalWriteTime / writeCount / 1000
                : Double.NaN;
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsPrinter.java b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsPrinter.java
index 389efba..037227b 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsPrinter.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsPrinter.java
@@ -66,4 +66,4 @@ public interface StatsPrinter<T extends StatsHolder>
             out.println(yaml.dump(data.convert2Map()));
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/triggers/CustomClassLoader.java b/src/java/org/apache/cassandra/triggers/CustomClassLoader.java
index 6948c2d..16b182e 100644
--- a/src/java/org/apache/cassandra/triggers/CustomClassLoader.java
+++ b/src/java/org/apache/cassandra/triggers/CustomClassLoader.java
@@ -21,22 +21,22 @@ package org.apache.cassandra.triggers;
  */
 
 
-import java.io.File;
-import java.io.FilenameFilter;
 import java.io.IOException;
 import java.net.URL;
 import java.net.URLClassLoader;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiPredicate;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.io.Files;
-
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 
+import static java.nio.file.Files.*;
+
 /**
  * Custom class loader will load the classes from the class path, CCL will load
  * the classes from the the URL first, if it cannot find the required class it
@@ -69,28 +69,22 @@ public class CustomClassLoader extends URLClassLoader
     {
         if (dir == null || !dir.exists())
             return;
-        FilenameFilter filter = new FilenameFilter()
-        {
-            public boolean accept(File dir, String name)
-            {
-                return name.endsWith(".jar");
-            }
-        };
-        for (File inputJar : dir.listFiles(filter))
+        BiPredicate<File, String> filter = (ignore, name) -> name.endsWith(".jar");
+        for (File inputJar : dir.tryList(filter))
         {
             File lib = new File(FileUtils.getTempDir(), "lib");
             if (!lib.exists())
             {
-                lib.mkdir();
+                lib.tryCreateDirectory();
                 lib.deleteOnExit();
             }
             File out = FileUtils.createTempFile("cassandra-", ".jar", lib);
             out.deleteOnExit();
-            logger.info("Loading new jar {}", inputJar.getAbsolutePath());
+            logger.info("Loading new jar {}", inputJar.absolutePath());
             try
             {
-                Files.copy(inputJar, out);
-                addURL(out.toURI().toURL());
+                copy(inputJar.toPath(), out.toPath());
+                addURL(out.toPath().toUri().toURL());
             }
             catch (IOException ex)
             {
diff --git a/src/java/org/apache/cassandra/triggers/TriggerExecutor.java b/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
index 295003f..298ac56 100644
--- a/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
+++ b/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
@@ -18,7 +18,6 @@
  */
 package org.apache.cassandra.triggers;
 
-import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.*;
 
@@ -33,6 +32,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.exceptions.CassandraException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TriggerMetadata;
 import org.apache.cassandra.schema.Triggers;
diff --git a/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java b/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
index d3c08b5..8506ce5 100644
--- a/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
+++ b/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
@@ -17,10 +17,13 @@
  */
 package org.apache.cassandra.utils;
 
+import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.IOException;
+import java.io.InputStream;
 
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.obs.IBitSet;
 import org.apache.cassandra.utils.obs.OffHeapBitSet;
@@ -38,7 +41,7 @@ public final class BloomFilterSerializer
     }
 
     @SuppressWarnings("resource")
-    public static BloomFilter deserialize(DataInputStream in, boolean oldBfFormat) throws IOException
+    public static <I extends InputStream & DataInput> BloomFilter deserialize(I in, boolean oldBfFormat) throws IOException
     {
         int hashes = in.readInt();
         IBitSet bs = OffHeapBitSet.deserialize(in, oldBfFormat);
diff --git a/src/java/org/apache/cassandra/utils/ByteArrayUtil.java b/src/java/org/apache/cassandra/utils/ByteArrayUtil.java
index 75734ad..3673f67 100644
--- a/src/java/org/apache/cassandra/utils/ByteArrayUtil.java
+++ b/src/java/org/apache/cassandra/utils/ByteArrayUtil.java
@@ -253,4 +253,4 @@ public class ByteArrayUtil
     {
         FastByteOperations.copy(src, srcPos, dst, dstPos, length);
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/utils/DirectorySizeCalculator.java b/src/java/org/apache/cassandra/utils/DirectorySizeCalculator.java
index c1fb6e0..97fc22e 100644
--- a/src/java/org/apache/cassandra/utils/DirectorySizeCalculator.java
+++ b/src/java/org/apache/cassandra/utils/DirectorySizeCalculator.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.utils;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.file.FileVisitResult;
 import java.nio.file.Path;
@@ -28,6 +27,8 @@ import java.nio.file.attribute.BasicFileAttributes;
 /**
  * Walks directory recursively, summing up total contents of files within.
  */
+import org.apache.cassandra.io.util.File;
+
 public class DirectorySizeCalculator extends SimpleFileVisitor<Path>
 {
     protected volatile long size = 0;
diff --git a/src/java/org/apache/cassandra/utils/ExecutorUtils.java b/src/java/org/apache/cassandra/utils/ExecutorUtils.java
index e26d776..5bb841f 100644
--- a/src/java/org/apache/cassandra/utils/ExecutorUtils.java
+++ b/src/java/org/apache/cassandra/utils/ExecutorUtils.java
@@ -152,4 +152,4 @@ public class ExecutorUtils
     {
         shutdownNowAndWait(timeout, unit, Arrays.asList(executors));
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java
index 66ac00e..e8e8066 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -17,7 +17,14 @@
  */
 package org.apache.cassandra.utils;
 
-import java.io.*;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
 import java.lang.reflect.Field;
 import java.math.BigInteger;
 import java.net.*;
@@ -38,6 +45,7 @@ import javax.annotation.Nullable;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Strings;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.utils.concurrent.*;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
@@ -365,7 +373,7 @@ public class FBUtilities
         if (scpurl == null)
             throw new ConfigurationException("unable to locate " + filename);
 
-        return new File(scpurl.getFile()).getAbsolutePath();
+        return new File(scpurl.getFile()).absolutePath();
     }
 
     public static File cassandraTriggerDir()
diff --git a/src/java/org/apache/cassandra/utils/HeapUtils.java b/src/java/org/apache/cassandra/utils/HeapUtils.java
index 4dd0d46..c0910d8 100644
--- a/src/java/org/apache/cassandra/utils/HeapUtils.java
+++ b/src/java/org/apache/cassandra/utils/HeapUtils.java
@@ -17,9 +17,13 @@
  */
 package org.apache.cassandra.utils;
 
-import java.io.*;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
 import java.lang.management.ManagementFactory;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.text.StrBuilder;
 
@@ -81,14 +85,8 @@ public final class HeapUtils
         if (javaHome == null)
             return null;
         File javaBinDirectory = new File(javaHome, "bin");
-        File[] files = javaBinDirectory.listFiles(new FilenameFilter()
-        {
-            public boolean accept(File dir, String name)
-            {
-                return name.startsWith("jcmd");
-            }
-        });
-        return ArrayUtils.isEmpty(files) ? null : files[0].getPath();
+        File[] files = javaBinDirectory.tryList((dir, name) -> name.startsWith("jcmd"));
+        return ArrayUtils.isEmpty(files) ? null : files[0].path();
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/utils/IndexedSearchIterator.java b/src/java/org/apache/cassandra/utils/IndexedSearchIterator.java
index 597e5bb..bd2f70a 100644
--- a/src/java/org/apache/cassandra/utils/IndexedSearchIterator.java
+++ b/src/java/org/apache/cassandra/utils/IndexedSearchIterator.java
@@ -35,4 +35,4 @@ public interface IndexedSearchIterator<K, V> extends SearchIterator<K, V>
      * @throws java.util.NoSuchElementException if next() returned null
      */
     public int indexOfCurrent();
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
index 158baf2..6470f1f 100644
--- a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
+++ b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.utils;
 
 import java.io.FileNotFoundException;
 import java.net.SocketException;
+import java.nio.file.FileSystemException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -141,7 +142,7 @@ public final class JVMStabilityInspector
         fn.accept(t);
 
         // Check for file handle exhaustion
-        if (t instanceof FileNotFoundException || t instanceof SocketException)
+        if (t instanceof FileNotFoundException || t instanceof FileSystemException || t instanceof SocketException)
             if (t.getMessage() != null && t.getMessage().contains("Too many open files"))
                 isUnstable = true;
 
diff --git a/src/java/org/apache/cassandra/utils/NativeLibrary.java b/src/java/org/apache/cassandra/utils/NativeLibrary.java
index e5b5da7..01225aa 100644
--- a/src/java/org/apache/cassandra/utils/NativeLibrary.java
+++ b/src/java/org/apache/cassandra/utils/NativeLibrary.java
@@ -17,14 +17,14 @@
  */
 package org.apache.cassandra.utils;
 
-import java.io.File;
 import java.io.FileDescriptor;
-import java.io.FileInputStream;
 import java.io.IOException;
 import java.lang.reflect.Field;
 import java.nio.channels.FileChannel;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -216,7 +216,7 @@ public final class NativeLibrary
         if (!f.exists())
             return;
 
-        try (FileInputStream fis = new FileInputStream(f))
+        try (FileInputStreamPlus fis = new FileInputStreamPlus(f))
         {
             trySkipCache(getfd(fis.getChannel()), offset, len, path);
         }
diff --git a/src/java/org/apache/cassandra/utils/ResourceWatcher.java b/src/java/org/apache/cassandra/utils/ResourceWatcher.java
index 5e7cbdd..e8dcb85 100644
--- a/src/java/org/apache/cassandra/utils/ResourceWatcher.java
+++ b/src/java/org/apache/cassandra/utils/ResourceWatcher.java
@@ -17,9 +17,9 @@
  */
 package org.apache.cassandra.utils;
 
-import java.io.File;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/utils/SyncUtil.java b/src/java/org/apache/cassandra/utils/SyncUtil.java
index b4a4bee..6055859 100644
--- a/src/java/org/apache/cassandra/utils/SyncUtil.java
+++ b/src/java/org/apache/cassandra/utils/SyncUtil.java
@@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.cassandra.config.Config;
 
 import com.google.common.base.Preconditions;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -172,12 +173,6 @@ public class SyncUtil
         }
     }
 
-    public static void sync(RandomAccessFile ras) throws IOException
-    {
-        Preconditions.checkNotNull(ras);
-        sync(ras.getFD());
-    }
-
     public static void sync(FileOutputStream fos) throws IOException
     {
         Preconditions.checkNotNull(fos);
@@ -197,7 +192,7 @@ public class SyncUtil
         if (SKIP_SYNC)
             return;
 
-        int directoryFD = NativeLibrary.tryOpenDirectory(dir.getPath());
+        int directoryFD = NativeLibrary.tryOpenDirectory(dir.path());
         try
         {
             trySync(directoryFD);
diff --git a/src/java/org/apache/cassandra/utils/Throwables.java b/src/java/org/apache/cassandra/utils/Throwables.java
index 73e7d24..7012132 100644
--- a/src/java/org/apache/cassandra/utils/Throwables.java
+++ b/src/java/org/apache/cassandra/utils/Throwables.java
@@ -18,7 +18,7 @@
 */
 package org.apache.cassandra.utils;
 
-import java.io.File;
+import org.apache.cassandra.io.util.File;
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.util.Arrays;
@@ -152,7 +152,7 @@ public final class Throwables
     @SafeVarargs
     public static void perform(File against, FileOpType opType, DiscreteAction<? extends IOException> ... actions)
     {
-        perform(against.getPath(), opType, actions);
+        perform(against.path(), opType, actions);
     }
 
     @SafeVarargs
diff --git a/src/java/org/apache/cassandra/utils/binlog/BinLog.java b/src/java/org/apache/cassandra/utils/binlog/BinLog.java
index 8b8588a..a9bb55a 100644
--- a/src/java/org/apache/cassandra/utils/binlog/BinLog.java
+++ b/src/java/org/apache/cassandra/utils/binlog/BinLog.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.utils.binlog;
 
-import java.io.File;
 import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -31,6 +30,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -363,11 +363,11 @@ public class BinLog implements Runnable
         public Builder path(Path path)
         {
             Preconditions.checkNotNull(path, "path was null");
-            File pathAsFile = path.toFile();
+            File pathAsFile = new File(path);
             //Exists and is a directory or can be created
             Preconditions.checkArgument(!pathAsFile.toString().isEmpty(), "you might have forgotten to specify a directory to save logs");
-            Preconditions.checkArgument((pathAsFile.exists() && pathAsFile.isDirectory()) || (!pathAsFile.exists() && pathAsFile.mkdirs()), "path exists and is not a directory or couldn't be created");
-            Preconditions.checkArgument(pathAsFile.canRead() && pathAsFile.canWrite() && pathAsFile.canExecute(), "path is not readable, writable, and executable");
+            Preconditions.checkArgument((pathAsFile.exists() && pathAsFile.isDirectory()) || (!pathAsFile.exists() && pathAsFile.tryCreateDirectories()), "path exists and is not a directory or couldn't be created");
+            Preconditions.checkArgument(pathAsFile.isReadable() && pathAsFile.isWritable() && pathAsFile.isExecutable(), "path is not readable, writable, and executable");
             this.path = path;
             return this;
         }
@@ -432,7 +432,7 @@ public class BinLog implements Runnable
                     logger.info("Cleaning directory: {} as requested", path);
                     if (path.toFile().exists())
                     {
-                        Throwable error = cleanDirectory(path.toFile(), null);
+                        Throwable error = cleanDirectory(new File(path), null);
                         if (error != null)
                         {
                             throw new RuntimeException(error);
@@ -471,7 +471,7 @@ public class BinLog implements Runnable
         {
             return Throwables.merge(accumulate, new RuntimeException(String.format("%s is not a directory", directory)));
         }
-        for (File f : directory.listFiles())
+        for (File f : directory.tryList())
         {
             accumulate = deleteRecursively(f, accumulate);
         }
@@ -486,7 +486,7 @@ public class BinLog implements Runnable
     {
         if (fileOrDirectory.isDirectory())
         {
-            for (File f : fileOrDirectory.listFiles())
+            for (File f : fileOrDirectory.tryList())
             {
                 accumulate = FileUtils.deleteWithConfirm(f, accumulate);
             }
diff --git a/src/java/org/apache/cassandra/utils/btree/LeafBTreeSearchIterator.java b/src/java/org/apache/cassandra/utils/btree/LeafBTreeSearchIterator.java
index a23f460..3c1991a 100644
--- a/src/java/org/apache/cassandra/utils/btree/LeafBTreeSearchIterator.java
+++ b/src/java/org/apache/cassandra/utils/btree/LeafBTreeSearchIterator.java
@@ -133,4 +133,4 @@ public class LeafBTreeSearchIterator<K, V> implements BTreeSearchIterator<K, V>
         int current = forwards ? nextPos - 1 : nextPos + 1;
         return forwards ? current - lowerBound : upperBound - current;
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/utils/btree/NodeCursor.java b/src/java/org/apache/cassandra/utils/btree/NodeCursor.java
index e9fa89e..4c7e993 100644
--- a/src/java/org/apache/cassandra/utils/btree/NodeCursor.java
+++ b/src/java/org/apache/cassandra/utils/btree/NodeCursor.java
@@ -195,4 +195,4 @@ class NodeCursor<K>
     {
         return (K) node[position];
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java b/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
index 8d118e7..8b0550f 100644
--- a/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
+++ b/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.utils.obs;
 
-import java.io.DataInputStream;
+import java.io.*;
 import java.io.IOException;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -143,7 +143,7 @@ public class OffHeapBitSet implements IBitSet
     }
 
     @SuppressWarnings("resource")
-    public static OffHeapBitSet deserialize(DataInputStream in, boolean oldBfFormat) throws IOException
+    public static <I extends InputStream & DataInput> OffHeapBitSet deserialize(I in, boolean oldBfFormat) throws IOException
     {
         long byteCount = in.readInt() * 8L;
         Memory memory = Memory.allocate(byteCount);
diff --git a/src/java/org/apache/cassandra/utils/streamhist/HistogramDataConsumer.java b/src/java/org/apache/cassandra/utils/streamhist/HistogramDataConsumer.java
old mode 100755
new mode 100644
diff --git a/src/java/org/apache/cassandra/utils/streamhist/StreamingTombstoneHistogramBuilder.java b/src/java/org/apache/cassandra/utils/streamhist/StreamingTombstoneHistogramBuilder.java
old mode 100755
new mode 100644
diff --git a/src/java/org/apache/cassandra/utils/streamhist/TombstoneHistogram.java b/src/java/org/apache/cassandra/utils/streamhist/TombstoneHistogram.java
old mode 100755
new mode 100644
diff --git a/test/burn/org/apache/cassandra/net/GlobalInboundSettings.java b/test/burn/org/apache/cassandra/net/GlobalInboundSettings.java
index 9b23041..da453fe 100644
--- a/test/burn/org/apache/cassandra/net/GlobalInboundSettings.java
+++ b/test/burn/org/apache/cassandra/net/GlobalInboundSettings.java
@@ -54,4 +54,4 @@ class GlobalInboundSettings
     {
         return new GlobalInboundSettings(queueCapacity, endpointReserveLimit, globalReserveLimit, template);
     }
-}
\ No newline at end of file
+}
diff --git a/test/burn/org/apache/cassandra/transport/DriverBurnTest.java b/test/burn/org/apache/cassandra/transport/DriverBurnTest.java
index 5655867..7856b1b 100644
--- a/test/burn/org/apache/cassandra/transport/DriverBurnTest.java
+++ b/test/burn/org/apache/cassandra/transport/DriverBurnTest.java
@@ -438,4 +438,4 @@ public class DriverBurnTest extends CQLTester
         System.out.println("99p:      " + stats.getPercentile(0.99));
     }
 }
-// TODO: test disconnecting and reconnecting constantly
\ No newline at end of file
+// TODO: test disconnecting and reconnecting constantly
diff --git a/test/burn/org/apache/cassandra/transport/SimpleClientBurnTest.java b/test/burn/org/apache/cassandra/transport/SimpleClientBurnTest.java
index 7e57916..2d863cf 100644
--- a/test/burn/org/apache/cassandra/transport/SimpleClientBurnTest.java
+++ b/test/burn/org/apache/cassandra/transport/SimpleClientBurnTest.java
@@ -212,4 +212,4 @@ public class SimpleClientBurnTest
         server.stop();
     }
 
-}
\ No newline at end of file
+}
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
index 201555f..217dcd5 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.distributed.impl;
 
-import java.io.File;
 import java.lang.annotation.Annotation;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
@@ -45,7 +44,6 @@ import java.util.stream.Stream;
 import javax.annotation.concurrent.GuardedBy;
 
 import com.google.common.collect.Sets;
-import org.apache.cassandra.utils.concurrent.Condition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -78,7 +76,10 @@ import org.apache.cassandra.distributed.shared.Shared;
 import org.apache.cassandra.distributed.shared.ShutdownException;
 import org.apache.cassandra.distributed.shared.Versions;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.PathUtils;
 import org.apache.cassandra.net.Verb;
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.apache.cassandra.utils.FBUtilities;
 import org.reflections.Reflections;
 import org.reflections.util.ConfigurationBuilder;
@@ -388,7 +389,7 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
 
     protected AbstractCluster(AbstractBuilder<I, ? extends ICluster<I>, ?> builder)
     {
-        this.root = builder.getRoot();
+        this.root = new File(builder.getRoot());
         this.sharedClassLoader = builder.getSharedClassLoader();
         this.subnet = builder.getSubnet();
         this.tokenSupplier = builder.getTokenSupplier();
@@ -857,6 +858,7 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
 
         instances.clear();
         instanceMap.clear();
+        PathUtils.setDeletionListener(ignore -> {});
         // Make sure to only delete directory when threads are stopped
         if (root.exists())
             FileUtils.deleteRecursive(root);
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/FileLogAction.java b/test/distributed/org/apache/cassandra/distributed/impl/FileLogAction.java
index 0f48a23..14db561 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/FileLogAction.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/FileLogAction.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.distributed.impl;
 
-import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.RandomAccessFile;
@@ -28,6 +27,7 @@ import java.util.function.Predicate;
 
 import com.google.common.io.Closeables;
 
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.utils.AbstractIterator;
 import org.apache.cassandra.distributed.api.LogAction;
 import org.apache.cassandra.distributed.api.LineIterator;
@@ -53,7 +53,7 @@ public class FileLogAction implements LogAction
         RandomAccessFile reader;
         try
         {
-            reader = new RandomAccessFile(file, "r");
+            reader = new RandomAccessFile(file.toJavaIOFile(), "r");
         }
         catch (FileNotFoundException e)
         {
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
index d590694..730773d 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.distributed.impl;
 
 import java.io.ByteArrayOutputStream;
 import java.io.Closeable;
-import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.net.BindException;
@@ -98,6 +97,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.CassandraMetricsRegistry;
@@ -203,7 +203,7 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
         if (!f.exists())
             f = new File(String.format("build/test/logs/%s/%s/%s/system.log", tag, clusterId, instanceId));
         if (!f.exists())
-            throw new AssertionError("Unable to locate system.log under " + new File("build/test/logs").getAbsolutePath() + "; make sure ICluster.setup() is called or extend TestBaseImpl and do not define a static beforeClass function with @BeforeClass");
+            throw new AssertionError("Unable to locate system.log under " + new File("build/test/logs").absolutePath() + "; make sure ICluster.setup() is called or extend TestBaseImpl and do not define a static beforeClass function with @BeforeClass");
         return new FileLogAction(f);
     }
 
@@ -639,11 +639,11 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
 
     private void mkdirs()
     {
-        new File(config.getString("saved_caches_directory")).mkdirs();
-        new File(config.getString("hints_directory")).mkdirs();
-        new File(config.getString("commitlog_directory")).mkdirs();
+        new File(config.getString("saved_caches_directory")).tryCreateDirectories();
+        new File(config.getString("hints_directory")).tryCreateDirectories();
+        new File(config.getString("commitlog_directory")).tryCreateDirectories();
         for (String dir : (String[]) config.get("data_file_directories"))
-            new File(dir).mkdirs();
+            new File(dir).tryCreateDirectories();
     }
 
     private Config loadConfig(IInstanceConfig overrides)
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
index 1bbdd0b..81e254d 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.distributed.impl;
 
-import java.io.File;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.Collections;
@@ -37,6 +36,7 @@ import org.apache.cassandra.distributed.api.IInstanceConfig;
 import org.apache.cassandra.distributed.shared.NetworkTopology;
 import org.apache.cassandra.distributed.shared.Shared;
 import org.apache.cassandra.distributed.upgrade.UpgradeTestBase;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.SimpleSeedProvider;
 
@@ -275,7 +275,7 @@ public class InstanceConfig implements IInstanceConfig
 
     private static String[] datadirs(int datadirCount, File root, int nodeNum)
     {
-        String datadirFormat = String.format("%s/node%d/data%%d", root.getPath(), nodeNum);
+        String datadirFormat = String.format("%s/node%d/data%%d", root.path(), nodeNum);
         String [] datadirs = new String[datadirCount];
         for (int i = 0; i < datadirs.length; i++)
             datadirs[i] = String.format(datadirFormat, i);
diff --git a/test/distributed/org/apache/cassandra/distributed/shared/Byteman.java b/test/distributed/org/apache/cassandra/distributed/shared/Byteman.java
index bc27ec7..b4dd10c 100644
--- a/test/distributed/org/apache/cassandra/distributed/shared/Byteman.java
+++ b/test/distributed/org/apache/cassandra/distributed/shared/Byteman.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.distributed.shared;
 
-import java.io.File;
 import java.io.IOException;
 import java.io.UncheckedIOException;
 import java.lang.reflect.Method;
@@ -42,6 +41,7 @@ import java.util.stream.Stream;
 import com.google.common.base.StandardSystemProperty;
 import com.google.common.io.ByteStreams;
 import com.google.common.io.Files;
+import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -96,7 +96,7 @@ public final class Byteman
         List<String> texts = Stream.of(scripts).map(p -> {
             try
             {
-                return Files.toString(new File(p), StandardCharsets.UTF_8);
+                return Files.toString(new File(p).toJavaIOFile(), StandardCharsets.UTF_8);
             }
             catch (IOException e)
             {
@@ -155,11 +155,11 @@ public final class Byteman
                 if (DEBUG_TRANSFORMATIONS)
                 {
                     File f = new File(StandardSystemProperty.JAVA_IO_TMPDIR.value(), "byteman/" + details.klassPath + ".class");
-                    f.getParentFile().mkdirs();
-                    File original = new File(f.getParentFile(), "original-" + f.getName());
-                    logger.info("Writing class file for {} to {}", details.klassPath, f.getAbsolutePath());
-                    Files.asByteSink(f).write(newBytes);
-                    Files.asByteSink(original).write(details.bytes);
+                    f.parent().tryCreateDirectories();
+                    File original = new File(f.parent(), "original-" + f.name());
+                    logger.info("Writing class file for {} to {}", details.klassPath, f.absolutePath());
+                    Files.asByteSink(f.toJavaIOFile()).write(newBytes);
+                    Files.asByteSink(original.toJavaIOFile()).write(details.bytes);
                 }
             }
         }
diff --git a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java
index 382f5a7..1821f9c 100644
--- a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java
+++ b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.distributed.shared;
 
-import java.io.File;
 import java.lang.reflect.Field;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
@@ -40,6 +39,7 @@ import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 
 import com.google.common.util.concurrent.Futures;
+import org.apache.cassandra.io.util.File;
 import org.junit.Assert;
 
 import org.apache.cassandra.dht.Token;
diff --git a/test/distributed/org/apache/cassandra/distributed/shared/ShutdownException.java b/test/distributed/org/apache/cassandra/distributed/shared/ShutdownException.java
index d2b5bf7..9738357 100644
--- a/test/distributed/org/apache/cassandra/distributed/shared/ShutdownException.java
+++ b/test/distributed/org/apache/cassandra/distributed/shared/ShutdownException.java
@@ -27,4 +27,4 @@ public class ShutdownException extends RuntimeException
         super("Uncaught exceptions were thrown during test");
         uncaughtExceptions.forEach(super::addSuppressed);
     }
-}
\ No newline at end of file
+}
diff --git a/test/distributed/org/apache/cassandra/distributed/test/FrozenUDTTest.java b/test/distributed/org/apache/cassandra/distributed/test/FrozenUDTTest.java
index 2a45b86..3b54398 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/FrozenUDTTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/FrozenUDTTest.java
@@ -150,4 +150,4 @@ public class FrozenUDTTest extends TestBaseImpl
     {
         return String.format("system.fromjson('{\"foo\":\"%d\", \"bar\":\"%d\"}')", i, j);
     }
-}
\ No newline at end of file
+}
diff --git a/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java b/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java
index 0a81359..e1733ce 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java
@@ -94,4 +94,4 @@ public class LargeColumnTest extends TestBaseImpl
     {
         testLargeColumns(2, 16 << 20, 5);
     }
-}
\ No newline at end of file
+}
diff --git a/test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java b/test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
index 6ea186c..9cae1bd 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
@@ -335,4 +335,4 @@ public class MessageFiltersTest extends TestBaseImpl
             // ignore
         }
     }
... 7707 lines suppressed ...

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


[cassandra] 06/06: [CASSANDRA-17013] CEP-10 Phase 1: in-jvm-dtest-api changes and version bump

Posted by be...@apache.org.
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 5b82447098ad634900f8892297ef82083eadb954
Author: Benedict Elliott Smith <be...@apache.org>
AuthorDate: Thu Jul 29 18:06:32 2021 +0100

    [CASSANDRA-17013] CEP-10 Phase 1: in-jvm-dtest-api changes and version bump
---
 build.xml                                          |   2 +-
 .../cassandra/concurrent/ExecutorFactory.java      |   2 +-
 .../cql3/functions/JavaBasedUDFunction.java        |   1 +
 .../cassandra/cql3/functions/UDFunction.java       |   2 +-
 .../org/apache/cassandra/gms/EndpointState.java    |   5 +-
 src/java/org/apache/cassandra/gms/Gossiper.java    |  10 +
 .../org/apache/cassandra/gms/HeartBeatState.java   |   5 +-
 .../cassandra/io/sstable/format/SSTableReader.java |   2 +
 src/java/org/apache/cassandra/io/util/File.java    |   8 +-
 .../apache/cassandra/triggers/TriggerExecutor.java |   2 +-
 .../org/apache/cassandra/distributed/Cluster.java  |   1 -
 .../distributed/impl/AbstractCluster.java          | 103 +++++-
 .../cassandra/distributed/impl/Coordinator.java    |   4 +-
 .../impl/DelegatingInvokableInstance.java          |  62 ++++
 .../impl/DirectStreamingConnectionFactory.java     | 386 +++++++++++++++++++++
 .../cassandra/distributed/impl/Instance.java       | 247 ++++++-------
 .../cassandra/distributed/impl/InstanceConfig.java |  22 +-
 .../cassandra/distributed/impl/InstanceKiller.java |   1 +
 .../distributed/impl/IsolatedExecutor.java         |  95 +++--
 .../apache/cassandra/distributed/impl/Query.java   | 110 ++++++
 .../distributed/impl/UnsafeGossipHelper.java       | 265 ++++++++++++++
 .../apache/cassandra/distributed/test/CASTest.java |  32 +-
 .../distributed/upgrade/UpgradeTestBase.java       |   3 +-
 .../cassandra/db/RangeTombstoneListTest.java       |   3 +-
 .../utils/concurrent/ImmediateFutureTest.java      |   3 +-
 25 files changed, 1141 insertions(+), 235 deletions(-)

diff --git a/build.xml b/build.xml
index f5acaff..b8eabb8 100644
--- a/build.xml
+++ b/build.xml
@@ -532,7 +532,7 @@
           <dependency groupId="com.google.code.java-allocation-instrumenter" artifactId="java-allocation-instrumenter" version="${allocation-instrumenter.version}" scope="test">
             <exclusion groupId="com.google.guava" artifactId="guava"/>
           </dependency>
-          <dependency groupId="org.apache.cassandra" artifactId="dtest-api" version="0.0.9" scope="test"/>
+          <dependency groupId="org.apache.cassandra" artifactId="dtest-api" version="0.0.10" scope="test"/>
           <dependency groupId="org.reflections" artifactId="reflections" version="0.9.12" scope="test"/>
           <dependency groupId="com.puppycrawl.tools" artifactId="checkstyle" version="8.40" scope="test"/>
           <dependency groupId="org.apache.hadoop" artifactId="hadoop-core" version="1.0.3" scope="provided">
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorFactory.java b/src/java/org/apache/cassandra/concurrent/ExecutorFactory.java
index 9c7a2cf..52ba94a 100644
--- a/src/java/org/apache/cassandra/concurrent/ExecutorFactory.java
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorFactory.java
@@ -146,7 +146,7 @@ public interface ExecutorFactory extends ExecutorBuilderFactory.Jmxable<Executor
     {
         // 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);
+        private static ExecutorFactory FACTORY = new ExecutorFactory.Default(Global.class.getClassLoader(), null, JVMStabilityInspector::uncaughtException);
         public static ExecutorFactory executorFactory()
         {
             return FACTORY;
diff --git a/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java b/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
index 3200c88..10be467 100644
--- a/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
@@ -350,6 +350,7 @@ public final class JavaBasedUDFunction extends UDFunction
         catch (InvocationTargetException e)
         {
             // in case of an ITE, use the cause
+            logger.error(String.format("Could not compile function '%s' from Java source:%n%s", name, javaSource), e);
             throw new InvalidRequestException(String.format("Could not compile function '%s' from Java source: %s", name, e.getCause()));
         }
         catch (InvalidRequestException | VirtualMachineError e)
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
index 172b998..eccea3c 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@ -743,7 +743,7 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
     private static class UDFClassLoader extends ClassLoader
     {
         // insecureClassLoader is the C* class loader
-        static final ClassLoader insecureClassLoader = Thread.currentThread().getContextClassLoader();
+        static final ClassLoader insecureClassLoader = UDFClassLoader.class.getClassLoader();
 
         private UDFClassLoader()
         {
diff --git a/src/java/org/apache/cassandra/gms/EndpointState.java b/src/java/org/apache/cassandra/gms/EndpointState.java
index a48a857..b7f6bdb 100644
--- a/src/java/org/apache/cassandra/gms/EndpointState.java
+++ b/src/java/org/apache/cassandra/gms/EndpointState.java
@@ -24,6 +24,8 @@ import java.util.stream.Collectors;
 
 import javax.annotation.Nullable;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -73,7 +75,8 @@ public class EndpointState
         isAlive = true;
     }
 
-    HeartBeatState getHeartBeatState()
+    @VisibleForTesting
+    public HeartBeatState getHeartBeatState()
     {
         return hbState;
     }
diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java
index 98068db..3219145 100644
--- a/src/java/org/apache/cassandra/gms/Gossiper.java
+++ b/src/java/org/apache/cassandra/gms/Gossiper.java
@@ -662,6 +662,16 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         GossiperDiagnostics.removedEndpoint(this, endpoint);
     }
 
+    @VisibleForTesting
+    public void unsafeAnnulEndpoint(InetAddressAndPort endpoint)
+    {
+        removeEndpoint(endpoint);
+        justRemovedEndpoints.remove(endpoint);
+        endpointStateMap.remove(endpoint);
+        expireTimeEndpointMap.remove(endpoint);
+        unreachableEndpoints.remove(endpoint);
+    }
+
     /**
      * Quarantines the endpoint for QUARANTINE_DELAY
      *
diff --git a/src/java/org/apache/cassandra/gms/HeartBeatState.java b/src/java/org/apache/cassandra/gms/HeartBeatState.java
index 104d957..d0a7142 100644
--- a/src/java/org/apache/cassandra/gms/HeartBeatState.java
+++ b/src/java/org/apache/cassandra/gms/HeartBeatState.java
@@ -19,6 +19,8 @@ package org.apache.cassandra.gms;
 
 import java.io.*;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -89,7 +91,8 @@ public class HeartBeatState
         generation += 1;
     }
 
-    void forceHighestPossibleVersionUnsafe()
+    @VisibleForTesting
+    public void forceHighestPossibleVersionUnsafe()
     {
         version = Integer.MAX_VALUE;
     }
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 1748a43..ebdb2ef 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -32,6 +32,8 @@ 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.config.CassandraRelevantProperties;
 import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.slf4j.Logger;
diff --git a/src/java/org/apache/cassandra/io/util/File.java b/src/java/org/apache/cassandra/io/util/File.java
index 2b60904..5fa5c9d 100644
--- a/src/java/org/apache/cassandra/io/util/File.java
+++ b/src/java/org/apache/cassandra/io/util/File.java
@@ -48,7 +48,8 @@ import static org.apache.cassandra.utils.Throwables.maybeFail;
  */
 public class File implements Comparable<File>
 {
-    private static final FileSystem filesystem = FileSystems.getDefault();
+    private static FileSystem filesystem = FileSystems.getDefault();
+
     public enum WriteMode { OVERWRITE, APPEND }
 
     public static String pathSeparator()
@@ -604,5 +605,10 @@ public class File implements Comparable<File>
             throw new IllegalStateException("Cannot read from an empty path");
         return path;
     }
+
+    public static void unsafeSetFilesystem(FileSystem fs)
+    {
+        filesystem = fs;
+    }
 }
 
diff --git a/src/java/org/apache/cassandra/triggers/TriggerExecutor.java b/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
index 298ac56..c76c6bd 100644
--- a/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
+++ b/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
@@ -44,7 +44,7 @@ public class TriggerExecutor
     public static final TriggerExecutor instance = new TriggerExecutor();
 
     private final Map<String, ITrigger> cachedTriggers = Maps.newConcurrentMap();
-    private final ClassLoader parent = Thread.currentThread().getContextClassLoader();
+    private final ClassLoader parent = TriggerExecutor.class.getClassLoader();
     private volatile ClassLoader customClassLoader;
 
     private TriggerExecutor()
diff --git a/test/distributed/org/apache/cassandra/distributed/Cluster.java b/test/distributed/org/apache/cassandra/distributed/Cluster.java
index a613fc5..05ea799 100644
--- a/test/distributed/org/apache/cassandra/distributed/Cluster.java
+++ b/test/distributed/org/apache/cassandra/distributed/Cluster.java
@@ -34,7 +34,6 @@ import org.apache.cassandra.distributed.shared.Versions;
 @Shared
 public class Cluster extends AbstractCluster<IInvokableInstance>
 {
-
     private Cluster(Builder builder)
     {
         super(builder);
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
index eca9088..2f146bf 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
@@ -20,6 +20,9 @@ package org.apache.cassandra.distributed.impl;
 
 import java.lang.annotation.Annotation;
 import java.net.InetSocketAddress;
+import java.nio.file.FileSystem;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -29,11 +32,12 @@ import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.Executor;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.BiConsumer;
 import java.util.function.BiPredicate;
 import java.util.function.Consumer;
 import java.util.function.Predicate;
@@ -54,15 +58,18 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.distributed.Constants;
 import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.IClassTransformer;
 import org.apache.cassandra.distributed.api.ICluster;
 import org.apache.cassandra.distributed.api.ICoordinator;
 import org.apache.cassandra.distributed.api.IInstance;
 import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.IInstanceInitializer;
 import org.apache.cassandra.distributed.api.IInvokableInstance;
 import org.apache.cassandra.distributed.api.IIsolatedExecutor;
 import org.apache.cassandra.distributed.api.IListen;
 import org.apache.cassandra.distributed.api.IMessage;
 import org.apache.cassandra.distributed.api.IMessageFilters;
+import org.apache.cassandra.distributed.api.IMessageSink;
 import org.apache.cassandra.distributed.api.IUpgradeableInstance;
 import org.apache.cassandra.distributed.api.LogAction;
 import org.apache.cassandra.distributed.api.NodeToolResult;
@@ -75,8 +82,6 @@ import org.apache.cassandra.distributed.shared.NetworkTopology;
 import org.apache.cassandra.distributed.shared.Shared;
 import org.apache.cassandra.distributed.shared.ShutdownException;
 import org.apache.cassandra.distributed.shared.Versions;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.io.util.PathUtils;
 import org.apache.cassandra.net.Verb;
 import org.apache.cassandra.utils.concurrent.Condition;
@@ -125,7 +130,7 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
     // include byteman so tests can use
     private static final Set<String> SHARED_CLASSES = findClassesMarkedForSharedClassLoader();
     private static final Set<String> ISOLATED_CLASSES = findClassesMarkedForInstanceClassLoader();
-    private static final Predicate<String> SHARED_PREDICATE = s -> {
+    public static final Predicate<String> SHARED_PREDICATE = s -> {
         if (ISOLATED_CLASSES.contains(s))
             return false;
 
@@ -135,8 +140,10 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
     };
 
     private final UUID clusterId = UUID.randomUUID();
-    private final File root;
+    private final Path root;
     private final ClassLoader sharedClassLoader;
+    private final Predicate<String> sharedClassPredicate;
+    private final IClassTransformer classTransformer;
     private final int subnet;
     private final TokenSupplier tokenSupplier;
     private final Map<Integer, NetworkTopology.DcAndRack> nodeIdTopology;
@@ -152,12 +159,16 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
     // mutated by user-facing API
     private final MessageFilters filters;
     private final INodeProvisionStrategy.Strategy nodeProvisionStrategy;
-    private final BiConsumer<ClassLoader, Integer> instanceInitializer;
+    private final IInstanceInitializer instanceInitializer;
     private final int datadirCount;
     private volatile Thread.UncaughtExceptionHandler previousHandler = null;
     private volatile BiPredicate<Integer, Throwable> ignoreUncaughtThrowable = null;
     private final List<Throwable> uncaughtExceptions = new CopyOnWriteArrayList<>();
 
+    private final ThreadGroup clusterThreadGroup = new ThreadGroup(clusterId.toString());
+
+    private volatile IMessageSink messageSink;
+
     /**
      * Common builder, add methods that are applicable to both Cluster and Upgradable cluster here.
      */
@@ -169,6 +180,7 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
         public AbstractBuilder(Factory<I, C, B> factory)
         {
             super(factory);
+            withSharedClasses(SHARED_PREDICATE);
         }
 
         public B withNodeProvisionStrategy(INodeProvisionStrategy.Strategy nodeProvisionStrategy)
@@ -215,11 +227,33 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
 
         private IInvokableInstance newInstance(int generation)
         {
-            ClassLoader classLoader = new InstanceClassLoader(generation, config.num(), version.classpath, sharedClassLoader, SHARED_PREDICATE);
+            ClassLoader classLoader = new InstanceClassLoader(generation, config.num(), version.classpath, sharedClassLoader, sharedClassPredicate, classTransformer);
+            ThreadGroup threadGroup = new ThreadGroup(clusterThreadGroup, "node" + config.num() + (generation > 1 ? "_" + generation : ""));
             if (instanceInitializer != null)
-                instanceInitializer.accept(classLoader, config.num());
-            return Instance.transferAdhoc((SerializableBiFunction<IInstanceConfig, ClassLoader, Instance>)Instance::new, classLoader)
-                                        .apply(config.forVersion(version.version), classLoader);
+                instanceInitializer.initialise(classLoader, threadGroup, config.num(), generation);
+
+            IInvokableInstance instance;
+            try
+            {
+                instance = Instance.transferAdhoc((SerializableTriFunction<IInstanceConfig, ClassLoader, FileSystem, Instance>)Instance::new, classLoader)
+                                   .apply(config.forVersion(version.version), classLoader, root.getFileSystem());
+            }
+            catch (NoSuchMethodError e)
+            {
+                instance = Instance.transferAdhoc((SerializableBiFunction<IInstanceConfig, ClassLoader, Instance>)Instance::new, classLoader)
+                                   .apply(config.forVersion(version.version), classLoader);
+            }
+
+            if (instanceInitializer != null)
+                instanceInitializer.beforeStartup(instance);
+
+            return instance;
+        }
+
+        public Executor executorFor(int verb)
+        {
+            // this method must be lock-free to avoid Simulator deadlock
+            return delegate().executorFor(verb);
         }
 
         public IInstanceConfig config()
@@ -241,7 +275,9 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
         public synchronized void startup()
         {
             startup(AbstractCluster.this);
+            postStartup();
         }
+
         public synchronized void startup(ICluster cluster)
         {
             if (cluster != AbstractCluster.this)
@@ -284,6 +320,9 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
                 throw t;
             }
             updateMessagingVersions();
+
+            if (instanceInitializer != null)
+                instanceInitializer.afterStartup(this);
         }
 
         @Override
@@ -342,6 +381,14 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
         }
 
         @Override
+        public void receiveMessageWithInvokingThread(IMessage message)
+        {
+            IInvokableInstance delegate = this.delegate;
+            if (isRunning() && delegate != null) // since we sync directly on the other node, we drop messages immediately if we are shutdown
+                delegate.receiveMessageWithInvokingThread(message);
+        }
+
+        @Override
         public boolean getLogsEnabled()
         {
             return delegate().getLogsEnabled();
@@ -388,8 +435,10 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
 
     protected AbstractCluster(AbstractBuilder<I, ? extends ICluster<I>, ?> builder)
     {
-        this.root = new File(builder.getRoot());
+        this.root = builder.getRootPath();
         this.sharedClassLoader = builder.getSharedClassLoader();
+        this.sharedClassPredicate = builder.getSharedClasses();
+        this.classTransformer = builder.getClassTransformer();
         this.subnet = builder.getSubnet();
         this.tokenSupplier = builder.getTokenSupplier();
         this.nodeIdTopology = builder.getNodeIdTopology();
@@ -400,7 +449,7 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
         this.instanceMap = new ConcurrentHashMap<>();
         this.initialVersion = builder.getVersion();
         this.filters = new MessageFilters();
-        this.instanceInitializer = builder.getInstanceInitializer();
+        this.instanceInitializer = builder.getInstanceInitializer2();
         this.datadirCount = builder.getDatadirCount();
 
         int generation = GENERATION.incrementAndGet();
@@ -526,7 +575,7 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
                                               i.config().localRack().equals(rackName));
     }
 
-    public void run(Consumer<? super I> action,  Predicate<I> filter)
+    public void run(Consumer<? super I> action, Predicate<I> filter)
     {
         run(Collections.singletonList(action), filter);
     }
@@ -594,6 +643,20 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
         return filters;
     }
 
+    public synchronized void setMessageSink(IMessageSink sink)
+    {
+        if (messageSink != null && sink != null)
+            throw new IllegalStateException();
+        this.messageSink = sink;
+    }
+
+    public void deliverMessage(InetSocketAddress to, IMessage message)
+    {
+        IMessageSink sink = messageSink;
+        if (sink == null) get(to).receiveMessage(message);
+        else sink.accept(to, message);
+    }
+
     public IMessageFilters.Builder verbs(Verb... verbs)
     {
         int[] ids = new int[verbs.length];
@@ -815,8 +878,14 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
                     startParallel.add(instance);
             }
 
-            forEach(startSequentially, I::startup);
-            parallelForEach(startParallel, I::startup, 0, null);
+            forEach(startSequentially, i -> {
+                i.startup(this);
+                i.postStartup();
+            });
+            parallelForEach(startParallel, i -> {
+                i.startup(this);
+                i.postStartup();
+            }, 0, null);
             monitor.waitForCompletion();
         }
     }
@@ -859,8 +928,8 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
         instanceMap.clear();
         PathUtils.setDeletionListener(ignore -> {});
         // Make sure to only delete directory when threads are stopped
-        if (root.exists())
-            FileUtils.deleteRecursive(root);
+        if (Files.exists(root))
+            PathUtils.deleteRecursive(root);
         Thread.setDefaultUncaughtExceptionHandler(previousHandler);
         previousHandler = null;
         checkAndResetUncaughtExceptions();
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java b/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java
index 9c4f255..9fdebed 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java
@@ -80,7 +80,7 @@ public class Coordinator implements ICoordinator
         }).call();
     }
 
-    protected org.apache.cassandra.db.ConsistencyLevel toCassandraCL(ConsistencyLevel cl)
+    static org.apache.cassandra.db.ConsistencyLevel toCassandraCL(ConsistencyLevel cl)
     {
         return org.apache.cassandra.db.ConsistencyLevel.fromCode(cl.ordinal());
     }
@@ -210,7 +210,7 @@ public class Coordinator implements ICoordinator
         }).call();
     }
 
-    private static final ClientState makeFakeClientState()
+    static ClientState makeFakeClientState()
     {
         return ClientState.forExternalCalls(new InetSocketAddress(FBUtilities.getJustLocalAddress(), 9042));
     }
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/DelegatingInvokableInstance.java b/test/distributed/org/apache/cassandra/distributed/impl/DelegatingInvokableInstance.java
index a6bd0f6..79bf946 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/DelegatingInvokableInstance.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/DelegatingInvokableInstance.java
@@ -21,6 +21,8 @@ package org.apache.cassandra.distributed.impl;
 import java.io.Serializable;
 import java.net.InetSocketAddress;
 import java.util.UUID;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.function.BiConsumer;
 import java.util.function.BiFunction;
@@ -31,6 +33,7 @@ import org.apache.cassandra.distributed.api.ICluster;
 import org.apache.cassandra.distributed.api.ICoordinator;
 import org.apache.cassandra.distributed.api.IInstanceConfig;
 import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor;
 import org.apache.cassandra.distributed.api.IListen;
 import org.apache.cassandra.distributed.api.IMessage;
 import org.apache.cassandra.distributed.api.SimpleQueryResult;
@@ -134,18 +137,42 @@ public abstract class DelegatingInvokableInstance implements IInvokableInstance
     }
 
     @Override
+    public IIsolatedExecutor with(ExecutorService executor)
+    {
+        return delegate().with(executor);
+    }
+
+    @Override
+    public Executor executor()
+    {
+        return delegate().executor();
+    }
+
+    @Override
     public void startup(ICluster cluster)
     {
         delegateForStartup().startup(cluster);
     }
 
     @Override
+    public void postStartup()
+    {
+        delegateForStartup().postStartup();
+    }
+
+    @Override
     public void receiveMessage(IMessage message)
     {
         delegate().receiveMessage(message);
     }
 
     @Override
+    public void receiveMessageWithInvokingThread(IMessage message)
+    {
+        delegate().receiveMessageWithInvokingThread(message);
+    }
+
+    @Override
     public <O> CallableNoExcept<Future<O>> async(CallableNoExcept<O> call)
     {
         return delegate().async(call);
@@ -194,6 +221,18 @@ public abstract class DelegatingInvokableInstance implements IInvokableInstance
     }
 
     @Override
+    public <I1, I2, I3> TriFunction<I1, I2, I3, Future<?>> async(TriConsumer<I1, I2, I3> consumer)
+    {
+        return delegate().async(consumer);
+    }
+
+    @Override
+    public <I1, I2, I3> TriConsumer<I1, I2, I3> sync(TriConsumer<I1, I2, I3> consumer)
+    {
+        return delegate().sync(consumer);
+    }
+
+    @Override
     public <I, O> Function<I, Future<O>> async(Function<I, O> f)
     {
         return delegate().async(f);
@@ -229,4 +268,27 @@ public abstract class DelegatingInvokableInstance implements IInvokableInstance
         return delegate().sync(f);
     }
 
+    @Override
+    public <I1, I2, I3, I4, O> QuadFunction<I1, I2, I3, I4, Future<O>> async(QuadFunction<I1, I2, I3, I4, O> f)
+    {
+        return delegate().async(f);
+    }
+
+    @Override
+    public <I1, I2, I3, I4, O> QuadFunction<I1, I2, I3, I4, O> sync(QuadFunction<I1, I2, I3, I4, O> f)
+    {
+        return delegate().sync(f);
+    }
+
+    @Override
+    public <I1, I2, I3, I4, I5, O> QuintFunction<I1, I2, I3, I4, I5, Future<O>> async(QuintFunction<I1, I2, I3, I4, I5, O> f)
+    {
+        return delegate().async(f);
+    }
+
+    @Override
+    public <I1, I2, I3, I4, I5, O> QuintFunction<I1, I2, I3, I4, I5, O> sync(QuintFunction<I1, I2, I3, I4, I5, O> f)
+    {
+        return delegate().sync(f);
+    }
 }
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/DirectStreamingConnectionFactory.java b/test/distributed/org/apache/cassandra/distributed/impl/DirectStreamingConnectionFactory.java
new file mode 100644
index 0000000..e598dc1
--- /dev/null
+++ b/test/distributed/org/apache/cassandra/distributed/impl/DirectStreamingConnectionFactory.java
@@ -0,0 +1,386 @@
+/*
+ * 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.distributed.impl;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.nio.channels.FileChannel;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.function.IntFunction;
+
+import io.netty.util.concurrent.Future;
+import org.apache.cassandra.distributed.api.ICluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
+import org.apache.cassandra.io.util.RebufferingInputStream;
+import org.apache.cassandra.streaming.StreamDeserializingTask;
+import org.apache.cassandra.streaming.StreamingDataInputPlus;
+import org.apache.cassandra.streaming.StreamingDataOutputPlus;
+import org.apache.cassandra.streaming.StreamingChannel;
+import org.apache.cassandra.streaming.StreamingDataOutputPlusFixed;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
+import org.apache.cassandra.utils.concurrent.NotScheduledFuture;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
+
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
+import static org.apache.cassandra.net.MessagingService.*;
+
+public class DirectStreamingConnectionFactory
+{
+    static class DirectConnection
+    {
+        private static final AtomicInteger nextId = new AtomicInteger();
+
+        final int protocolVersion;
+
+        // TODO rename
+        private static class Buffer
+        {
+            boolean isClosed;
+            byte[] pending;
+        }
+
+        @SuppressWarnings({"InnerClassMayBeStatic","unused"}) // helpful for debugging
+        class DirectStreamingChannel implements StreamingChannel
+        {
+            class Out extends BufferedDataOutputStreamPlus implements StreamingDataOutputPlus
+            {
+                private final Buffer out;
+                private Thread thread;
+                private boolean inUse;
+
+                Out(Buffer out)
+                {
+                    super(ByteBuffer.allocate(16 << 10));
+                    this.out = out;
+                }
+
+                protected void doFlush(int count) throws IOException
+                {
+                    if (buffer.position() == 0)
+                        return;
+
+                    try
+                    {
+                        synchronized (out)
+                        {
+                            while (out.pending != null && !out.isClosed)
+                                out.wait();
+
+                            if (out.isClosed)
+                                throw new ClosedChannelException();
+
+                            buffer.flip();
+                            out.pending = ByteBufferUtil.getArray(buffer);
+                            buffer.clear();
+
+                            out.notify();
+                        }
+                    }
+                    catch (InterruptedException e)
+                    {
+                        throw new UncheckedInterruptedException(e);
+                    }
+                }
+
+                public synchronized Out acquire()
+                {
+                    if (inUse)
+                        throw new IllegalStateException();
+                    inUse = true;
+                    thread = Thread.currentThread();
+                    return this;
+                }
+
+                public synchronized void close() throws IOException
+                {
+                    flush();
+                    inUse = false;
+                }
+
+                void realClose()
+                {
+                    synchronized (out)
+                    {
+                        out.isClosed = true;
+                        out.notifyAll();
+                    }
+                }
+
+                @Override
+                public int writeToChannel(Write write, RateLimiter limiter) throws IOException
+                {
+                    class Holder
+                    {
+                        ByteBuffer buffer;
+                    }
+                    Holder holder = new Holder();
+
+                    write.write(size -> {
+                        if (holder.buffer != null)
+                            throw new IllegalStateException("Can only allocate one ByteBuffer");
+                        holder.buffer = ByteBuffer.allocate(size);
+                        return holder.buffer;
+                    });
+
+                    ByteBuffer buffer = holder.buffer;
+                    int length = buffer.limit();
+                    write(buffer);
+                    return length;
+                }
+
+                // TODO (future): support RateLimiter
+                @Override
+                public long writeFileToChannel(FileChannel file, RateLimiter limiter) throws IOException
+                {
+                    long count = 0;
+                    while (file.read(buffer) >= 0)
+                    {
+                        count += buffer.position();
+                        doFlush(0);
+                    }
+                    return count;
+                }
+            }
+
+            class In extends RebufferingInputStream implements StreamingDataInputPlus
+            {
+                private final Buffer in;
+                private Thread thread;
+
+                In(Buffer in)
+                {
+                    super(ByteBuffer.allocate(0));
+                    this.in = in;
+                }
+
+                protected void reBuffer() throws IOException
+                {
+                    try
+                    {
+                        synchronized (in)
+                        {
+                            byte[] bytes;
+                            while ((bytes = in.pending) == null && !in.isClosed)
+                                in.wait();
+
+                            if (bytes == null)
+                                throw new ClosedChannelException();
+
+                            in.pending = null;
+                            buffer = ByteBuffer.wrap(bytes);
+
+                            in.notify();
+                        }
+                    }
+                    catch (InterruptedException e)
+                    {
+                        throw new UncheckedInterruptedException(e);
+                    }
+                }
+
+                public void close()
+                {
+                    DirectStreamingChannel.this.close();
+                }
+
+                public void realClose()
+                {
+                    synchronized (in)
+                    {
+                        in.isClosed = true;
+                        in.notifyAll();
+                    }
+                }
+            }
+
+            final InetSocketAddress remoteAddress;
+
+            private final In in;
+            private final Out out;
+            private final Integer id = nextId.incrementAndGet();
+            Runnable onClose;
+            boolean isClosed;
+
+            DirectStreamingChannel(InetSocketAddress remoteAddress, Buffer outBuffer, Buffer inBuffer)
+            {
+                this.remoteAddress = remoteAddress;
+                this.in = new In(inBuffer);
+                this.out = new Out(outBuffer);
+            }
+
+            public StreamingDataOutputPlus acquireOut()
+            {
+                return out.acquire();
+            }
+
+            @Override
+            public synchronized Future<?> send(Send send) throws IOException
+            {
+                class Factory implements IntFunction<StreamingDataOutputPlus>
+                {
+                    ByteBuffer buffer;
+                    @Override
+                    public StreamingDataOutputPlus apply(int size)
+                    {
+                        buffer = ByteBuffer.allocate(size);
+                        return new StreamingDataOutputPlusFixed(buffer);
+                    }
+                }
+                Factory factory = new Factory();
+                send.send(factory);
+                factory.buffer.flip();
+                try (StreamingDataOutputPlus out = acquireOut())
+                {
+                    out.write(factory.buffer);
+                }
+                return ImmediateFuture.success(true);
+            }
+
+            @Override
+            public Object id()
+            {
+                return id;
+            }
+
+            @Override
+            public String description()
+            {
+                return remoteAddress.getAddress().getHostAddress() + "/in@" + id;
+            }
+
+            public StreamingDataInputPlus in()
+            {
+                in.thread = Thread.currentThread();
+                return in;
+            }
+
+            public InetSocketAddress peer()
+            {
+                return remoteAddress;
+            }
+
+            @Override
+            public InetSocketAddress connectedTo()
+            {
+                return remoteAddress;
+            }
+
+            @Override
+            public boolean connected()
+            {
+                return true;
+            }
+
+            @Override
+            public Future<?> close()
+            {
+                in.realClose();
+                out.realClose();
+                synchronized (this)
+                {
+                    if (!isClosed)
+                    {
+                        isClosed = true;
+                        if (onClose != null)
+                            onClose.run();
+                    }
+                }
+                return ImmediateFuture.success(null);
+            }
+
+            @Override
+            public synchronized void onClose(Runnable runOnClose)
+            {
+                if (isClosed) runOnClose.run();
+                else if (onClose == null) onClose = runOnClose;
+                else { Runnable tmp = onClose; onClose = () -> { tmp.run(); runOnClose.run(); }; }
+            }
+        }
+
+        private final DirectStreamingChannel outToRecipient, outToOriginator;
+
+        DirectConnection(int protocolVersion, InetSocketAddress originator, InetSocketAddress recipient)
+        {
+            this.protocolVersion = protocolVersion;
+            Buffer buffer1 = new Buffer(), buffer2 = new Buffer();
+            outToRecipient = new DirectStreamingChannel(recipient, buffer1, buffer2);
+            outToOriginator = new DirectStreamingChannel(originator, buffer2, buffer1);
+        }
+
+        StreamingChannel get(InetSocketAddress remoteAddress)
+        {
+            if (remoteAddress.equals(outToOriginator.remoteAddress)) return outToOriginator;
+            else if (remoteAddress.equals(outToRecipient.remoteAddress)) return outToRecipient;
+            else throw new IllegalArgumentException();
+        }
+    }
+
+    public class Factory implements StreamingChannel.Factory
+    {
+        final InetSocketAddress from;
+        Factory(InetSocketAddress from)
+        {
+            this.from = from;
+        }
+
+        @Override
+        public StreamingChannel create(InetSocketAddress to, int messagingVersion, StreamingChannel.Kind kind)
+        {
+            DirectConnection connection = new DirectConnection(messagingVersion, from, to);
+            IInvokableInstance instance = cluster.get(to);
+            instance.unsafeAcceptOnThisThread((channel, version) -> executorFactory().startThread(channel.description(), new StreamDeserializingTask(null, channel, version)),
+                         connection.get(from), messagingVersion);
+            return connection.get(to);
+        }
+    }
+
+    final ICluster<IInvokableInstance> cluster;
+    final int protocolVersion;
+
+    private DirectStreamingConnectionFactory(ICluster<IInvokableInstance> cluster)
+    {
+        this.cluster = cluster;
+        // we don't invoke this on the host ClassLoader as it initiates state like DatabaseDescriptor,
+        // potentially leading to resource leaks on the hosts (particularly in validateHeader which runs on the host threads)
+        this.protocolVersion = current_version;
+    }
+
+    public static Function<IInvokableInstance, StreamingChannel.Factory> create(ICluster<IInvokableInstance> cluster)
+    {
+        return cluster.get(1).unsafeApplyOnThisThread(c -> new DirectStreamingConnectionFactory(c)::get, cluster);
+    }
+
+    public static void setup(ICluster<IInvokableInstance> cluster)
+    {
+        Function<IInvokableInstance, StreamingChannel.Factory> streamingConnectionFactory = create(cluster);
+        cluster.stream().forEach(i -> i.unsafeAcceptOnThisThread(StreamingChannel.Factory.Global::unsafeSet, streamingConnectionFactory.apply(i)));
+    }
+
+    public Factory get(IInvokableInstance instance)
+    {
+        return new Factory(instance.config().broadcastAddress());
+    }
+}
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
index 3f67b2f..18c1573 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
@@ -19,12 +19,12 @@
 package org.apache.cassandra.distributed.impl;
 
 import java.io.ByteArrayOutputStream;
-import java.io.Closeable;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
+import java.nio.file.FileSystem;
 import java.security.Permission;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -49,6 +49,7 @@ import org.apache.cassandra.auth.AuthCache;
 import org.apache.cassandra.batchlog.Batch;
 import org.apache.cassandra.batchlog.BatchlogManager;
 import org.apache.cassandra.concurrent.ExecutorLocals;
+import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.concurrent.SharedExecutorPool;
 import org.apache.cassandra.concurrent.Stage;
@@ -67,8 +68,6 @@ import org.apache.cassandra.db.SystemKeyspaceMigrator40;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.compaction.CompactionLogger;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.distributed.Constants;
 import org.apache.cassandra.distributed.action.GossipHelper;
@@ -85,9 +84,8 @@ import org.apache.cassandra.distributed.api.SimpleQueryResult;
 import org.apache.cassandra.distributed.mock.nodetool.InternalNodeProbe;
 import org.apache.cassandra.distributed.mock.nodetool.InternalNodeProbeFactory;
 import org.apache.cassandra.distributed.shared.Metrics;
-import org.apache.cassandra.gms.ApplicationState;
+import org.apache.cassandra.distributed.shared.ThrowingRunnable;
 import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.gms.VersionedValue;
 import org.apache.cassandra.hints.DTestSerializer;
 import org.apache.cassandra.hints.HintsService;
 import org.apache.cassandra.index.SecondaryIndexManager;
@@ -130,6 +128,7 @@ import org.apache.cassandra.tracing.TraceState;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteArrayUtil;
+import org.apache.cassandra.utils.Closeable;
 import org.apache.cassandra.utils.DiagnosticSnapshotService;
 import org.apache.cassandra.utils.ExecutorUtils;
 import org.apache.cassandra.utils.FBUtilities;
@@ -141,6 +140,7 @@ import org.apache.cassandra.utils.progress.jmx.JMXBroadcastExecutor;
 import org.apache.cassandra.utils.memory.BufferPools;
 
 import static java.util.concurrent.TimeUnit.MINUTES;
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
 import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
 import static org.apache.cassandra.distributed.api.Feature.NETWORK;
@@ -149,18 +149,27 @@ import static org.apache.cassandra.distributed.impl.DistributedTestSnitch.toCass
 import static org.apache.cassandra.net.Verb.BATCH_STORE_REQ;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
 
+/**
+ * This class is instantiated on the relevant classloader, so its methods invoke the correct target classes automatically
+ */
 public class Instance extends IsolatedExecutor implements IInvokableInstance
 {
     public final IInstanceConfig config;
     private volatile boolean initialized = false;
     private final long startedAt;
 
-    // should never be invoked directly, so that it is instantiated on other class loader;
-    // only visible for inheritance
+    @Deprecated
     Instance(IInstanceConfig config, ClassLoader classLoader)
     {
-        super("node" + config.num(), classLoader);
+        this(config, classLoader, null);
+    }
+
+    Instance(IInstanceConfig config, ClassLoader classLoader, FileSystem fileSystem)
+    {
+        super("node" + config.num(), classLoader, executorFactory().pooled("isolatedExecutor", Integer.MAX_VALUE));
         this.config = config;
+        if (fileSystem != null)
+            File.unsafeSetFilesystem(fileSystem);
         Object clusterId = Objects.requireNonNull(config.get(Constants.KEY_DTEST_API_CLUSTER_ID), "cluster_id is not defined");
         ClusterIDDefiner.setId("cluster-" + clusterId);
         InstanceIDDefiner.setInstanceId(config.num());
@@ -230,31 +239,34 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
     @Override
     public SimpleQueryResult executeInternalWithResult(String query, Object... args)
     {
-        return sync(() -> {
-            ClientWarn.instance.captureWarnings();
-            CoordinatorWarnings.init();
-            try
-            {
-                QueryHandler.Prepared prepared = QueryProcessor.prepareInternal(query);
-                ResultMessage result = prepared.statement.executeLocally(QueryProcessor.internalQueryState(),
-                                                           QueryProcessor.makeInternalOptions(prepared.statement, args));
-                CoordinatorWarnings.done();
-
-                if (result != null)
-                    result.setWarnings(ClientWarn.instance.getWarnings());
-                return RowUtil.toQueryResult(result);
-            }
-            catch (Exception | Error e)
-            {
-                CoordinatorWarnings.done();
-                throw e;
-            }
-            finally
-            {
-                CoordinatorWarnings.reset();
-                ClientWarn.instance.resetWarnings();
-            }
-        }).call();
+        return sync(() -> unsafeExecuteInternalWithResult(query, args)).call();
+    }
+
+    public static SimpleQueryResult unsafeExecuteInternalWithResult(String query, Object ... args)
+    {
+        ClientWarn.instance.captureWarnings();
+        CoordinatorWarnings.init();
+        try
+        {
+            QueryHandler.Prepared prepared = QueryProcessor.prepareInternal(query);
+            ResultMessage result = prepared.statement.executeLocally(QueryProcessor.internalQueryState(),
+                                                                     QueryProcessor.makeInternalOptions(prepared.statement, args));
+            CoordinatorWarnings.done();
+
+            if (result != null)
+                result.setWarnings(ClientWarn.instance.getWarnings());
+            return RowUtil.toQueryResult(result);
+        }
+        catch (Exception | Error e)
+        {
+            CoordinatorWarnings.done();
+            throw e;
+        }
+        finally
+        {
+            CoordinatorWarnings.reset();
+            ClientWarn.instance.resetWarnings();
+        }
     }
 
     @Override
@@ -297,20 +309,19 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
         }).run();
     }
 
-    private void registerMockMessaging(ICluster cluster)
+    private void registerMockMessaging(ICluster<?> cluster)
     {
         MessagingService.instance().outboundSink.add((message, to) -> {
-            InetSocketAddress toAddr = fromCassandraInetAddressAndPort(to);
-            IInstance toInstance = cluster.get(toAddr);
-            if (toInstance != null)
-                toInstance.receiveMessage(serializeMessage(message.from(), to, message));
+            cluster.deliverMessage(to, serializeMessage(message.from(), to, message));
             return false;
         });
     }
 
-    private void registerInboundFilter(ICluster cluster)
+    private void registerInboundFilter(ICluster<?> cluster)
     {
         MessagingService.instance().inboundSink.add(message -> {
+            if (!cluster.filters().hasInbound())
+                return true;
             if (isShutdown())
                 return false;
             IMessage serialized = serializeMessage(message.from(), toCassandraInetAddressAndPort(broadcastAddress()), message);
@@ -343,7 +354,7 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
         sync(JVMStabilityInspector::uncaughtException).accept(thread, throwable);
     }
 
-    private static IMessage serializeMessage(InetAddressAndPort from, InetAddressAndPort to, Message<?> messageOut)
+    public static IMessage serializeMessage(InetAddressAndPort from, InetAddressAndPort to, Message<?> messageOut)
     {
         int fromVersion = MessagingService.instance().versions.get(from);
         int toVersion = MessagingService.instance().versions.get(to);
@@ -434,7 +445,20 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
     @Override
     public void receiveMessage(IMessage message)
     {
-        sync(() -> {
+        sync(receiveMessageRunnable(message)).accept(false);
+    }
+
+    @Override
+    public void receiveMessageWithInvokingThread(IMessage message)
+    {
+        if (classLoader != Thread.currentThread().getContextClassLoader())
+            throw new IllegalStateException("Must be invoked by a Thread utilising the node's class loader");
+        receiveMessageRunnable(message).accept(true);
+    }
+
+    private SerializableConsumer<Boolean> receiveMessageRunnable(IMessage message)
+    {
+        return runOnCaller -> {
             if (message.version() > MessagingService.current_version)
             {
                 throw new IllegalStateException(String.format("Node%d received message version %d but current version is %d",
@@ -446,15 +470,26 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
             Message<?> messageIn = deserializeMessage(message);
             Message.Header header = messageIn.header;
             TraceState state = Tracing.instance.initializeFromMessage(header);
-            if (state != null) state.trace("{} message received from {}", header.verb, header.from);
-            header.verb.stage.execute(ExecutorLocals.create(state), () -> MessagingService.instance().inboundSink.accept(messageIn)
-            );
-        }).run();
+            if (state != null)
+                state.trace("{} message received from {}", header.verb, header.from);
+
+            if (runOnCaller)
+            {
+                try (Closeable close = ExecutorLocals.create(state))
+                {
+                    MessagingService.instance().inboundSink.accept(messageIn);
+                }
+            }
+            else
+            {
+                header.verb.stage.executor().execute(ExecutorLocals.create(state), () -> MessagingService.instance().inboundSink.accept(messageIn));
+            }
+        };
     }
 
     public int getMessagingVersion()
     {
-        return callsOnInstance(() -> MessagingService.current_version).call();
+        return MessagingService.current_version;
     }
 
     @Override
@@ -466,26 +501,29 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
     @Override
     public String getReleaseVersionString()
     {
-        return callsOnInstance(() -> FBUtilities.getReleaseVersionString()).call();
+        return FBUtilities.getReleaseVersionString();
     }
 
     public void flush(String keyspace)
     {
-        runOnInstance(() -> FBUtilities.waitOnFutures(Keyspace.open(keyspace).flush()));
+        FBUtilities.waitOnFutures(Keyspace.open(keyspace).flush());
     }
 
     public void forceCompact(String keyspace, String table)
     {
-        runOnInstance(() -> {
-            try
-            {
-                Keyspace.open(keyspace).getColumnFamilyStore(table).forceMajorCompaction();
-            }
-            catch (Exception e)
-            {
-                throw new RuntimeException(e);
-            }
-        });
+        try
+        {
+            Keyspace.open(keyspace).getColumnFamilyStore(table).forceMajorCompaction();
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public ExecutorPlus executorFor(int verbId)
+    {
+        return Verb.fromId(verbId).stage.executor();
     }
 
     @Override
@@ -602,6 +640,7 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
 
                     StorageService.instance.setUpDistributedSystemKeyspaces();
                     StorageService.instance.setNormalModeUnsafe();
+                    Gossiper.instance.register(StorageService.instance);
                 }
 
                 // Populate tokenMetadata for the second time,
@@ -610,7 +649,6 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
 
                 SystemKeyspace.finishStartup();
 
-                StorageService.instance.doAuthSetup(false);
                 CassandraDaemon.getInstanceForTesting().completeSetup();
 
                 if (config.has(NATIVE_PROTOCOL))
@@ -624,6 +662,7 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
                     throw new IllegalStateException(String.format("%s != %s", FBUtilities.getBroadcastAddressAndPort(), broadcastAddress()));
 
                 ActiveRepairService.instance.start();
+                CassandraDaemon.getInstanceForTesting().completeSetup();
             }
             catch (Throwable t)
             {
@@ -636,6 +675,11 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
         initialized = true;
     }
 
+    @Override
+    public void postStartup()
+    {
+        StorageService.instance.doAuthSetup(false);
+    }
 
     private void mkdirs()
     {
@@ -648,90 +692,13 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
 
     private Config loadConfig(IInstanceConfig overrides)
     {
-        Map<String,Object> params = ((InstanceConfig) overrides).getParams();
+        Map<String, Object> params = overrides.getParams();
         boolean check = true;
         if (overrides.get(Constants.KEY_DTEST_API_CONFIG_CHECK) != null)
             check = (boolean) overrides.get(Constants.KEY_DTEST_API_CONFIG_CHECK);
         return YamlConfigurationLoader.fromMap(params, check, Config.class);
     }
 
-    public static void addToRing(boolean bootstrapping, IInstance peer)
-    {
-        try
-        {
-            IInstanceConfig config = peer.config();
-            IPartitioner partitioner = FBUtilities.newPartitioner(config.getString("partitioner"));
-            Token token = partitioner.getTokenFactory().fromString(config.getString("initial_token"));
-            InetAddressAndPort addressAndPort = toCassandraInetAddressAndPort(peer.broadcastAddress());
-
-            UUID hostId = config.hostId();
-            Gossiper.runInGossipStageBlocking(() -> {
-                Gossiper.instance.initializeNodeUnsafe(addressAndPort, hostId, 1);
-                Gossiper.instance.injectApplicationState(addressAndPort,
-                                                         ApplicationState.TOKENS,
-                                                         new VersionedValue.VersionedValueFactory(partitioner).tokens(Collections.singleton(token)));
-                StorageService.instance.onChange(addressAndPort,
-                                                 ApplicationState.STATUS,
-                                                 bootstrapping
-                                                 ? new VersionedValue.VersionedValueFactory(partitioner).bootstrapping(Collections.singleton(token))
-                                                 : new VersionedValue.VersionedValueFactory(partitioner).normal(Collections.singleton(token)));
-                Gossiper.instance.realMarkAlive(addressAndPort, Gossiper.instance.getEndpointStateForEndpoint(addressAndPort));
-            });
-            int messagingVersion = peer.isShutdown()
-                    ? MessagingService.current_version
-                    : Math.min(MessagingService.current_version, peer.getMessagingVersion());
-            MessagingService.instance().versions.set(addressAndPort, messagingVersion);
-
-            assert bootstrapping || StorageService.instance.getTokenMetadata().isMember(addressAndPort);
-            PendingRangeCalculatorService.instance.blockUntilFinished();
-        }
-        catch (Throwable e) // UnknownHostException
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public static void removeFromRing(IInstance peer)
-    {
-        try
-        {
-            IInstanceConfig config = peer.config();
-            IPartitioner partitioner = FBUtilities.newPartitioner(config.getString("partitioner"));
-            Token token = partitioner.getTokenFactory().fromString(config.getString("initial_token"));
-            InetAddressAndPort addressAndPort = toCassandraInetAddressAndPort(peer.broadcastAddress());
-
-            Gossiper.runInGossipStageBlocking(() -> {
-                StorageService.instance.onChange(addressAndPort,
-                        ApplicationState.STATUS,
-                        new VersionedValue.VersionedValueFactory(partitioner).left(Collections.singleton(token), 0L));
-            });
-        }
-        catch (Throwable e) // UnknownHostException
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public static void addToRingNormal(IInstance peer)
-    {
-        addToRing(false, peer);
-        assert StorageService.instance.getTokenMetadata().isMember(toCassandraInetAddressAndPort(peer.broadcastAddress()));
-    }
-
-    public static void addToRingBootstrapping(IInstance peer)
-    {
-        addToRing(true, peer);
-    }
-
-    private static void initializeRing(ICluster cluster)
-    {
-        for (int i = 1 ; i <= cluster.size() ; ++i)
-            addToRing(false, cluster.get(i));
-
-        for (int i = 1; i <= cluster.size(); ++i)
-            assert StorageService.instance.getTokenMetadata().isMember(toCassandraInetAddressAndPort(cluster.get(i).broadcastAddress()));
-    }
-
     public Future<Void> shutdown()
     {
         return shutdown(true);
@@ -769,7 +736,6 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
                                 () -> SecondaryIndexManager.shutdownAndWait(1L, MINUTES),
                                 () -> IndexSummaryManager.instance.shutdownAndWait(1L, MINUTES),
                                 () -> ColumnFamilyStore.shutdownExecutorsAndWait(1L, MINUTES),
-                                () -> PendingRangeCalculatorService.instance.shutdownAndWait(1L, MINUTES),
                                 () -> BufferPools.shutdownLocalCleaner(1L, MINUTES),
                                 () -> Ref.shutdownReferenceReaper(1L, MINUTES),
                                 () -> Memtable.MEMORY_POOL.shutdownAndWait(1L, MINUTES),
@@ -786,11 +752,12 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
                                 (IgnoreThrowingRunnable) () -> MessagingService.instance().shutdown(1L, MINUTES, false, true)
             );
             error = parallelRun(error, executor,
-                                () -> GlobalEventExecutor.INSTANCE.awaitInactivity(1l, MINUTES),
+                                () -> { try { GlobalEventExecutor.INSTANCE.awaitInactivity(1L, MINUTES); } catch (IllegalStateException ignore) {} },
                                 () -> Stage.shutdownAndWait(1L, MINUTES),
                                 () -> SharedExecutorPool.SHARED.shutdownAndWait(1L, MINUTES)
             );
             error = parallelRun(error, executor,
+                                () -> PendingRangeCalculatorService.instance.shutdownAndWait(1L, MINUTES),
                                 () -> shutdownAndWait(Collections.singletonList(JMXBroadcastExecutor.executor))
             );
 
@@ -817,7 +784,7 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
     @Override
     public Metrics metrics()
     {
-        return callOnInstance(() -> new InstanceMetrics(CassandraMetricsRegistry.Metrics));
+        return new InstanceMetrics(CassandraMetricsRegistry.Metrics);
     }
 
     @Override
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
index 81e254d..9978697 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.distributed.impl;
 
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
+import java.nio.file.Path;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.Map;
@@ -28,24 +29,18 @@ import java.util.UUID;
 import java.util.function.Function;
 
 import com.vdurmont.semver4j.Semver;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.distributed.api.Feature;
 import org.apache.cassandra.distributed.api.IInstanceConfig;
 import org.apache.cassandra.distributed.shared.NetworkTopology;
 import org.apache.cassandra.distributed.shared.Shared;
 import org.apache.cassandra.distributed.upgrade.UpgradeTestBase;
-import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.SimpleSeedProvider;
 
 @Shared
 public class InstanceConfig implements IInstanceConfig
 {
-    private static final Object NULL = new Object();
-    private static final Logger logger = LoggerFactory.getLogger(InstanceConfig.class);
-
     public final int num;
     public int num() { return num; }
 
@@ -130,7 +125,6 @@ public class InstanceConfig implements IInstanceConfig
         this.broadcastAddressAndPort = copy.broadcastAddressAndPort;
     }
 
-
     @Override
     public InetSocketAddress broadcastAddress()
     {
@@ -193,18 +187,12 @@ public class InstanceConfig implements IInstanceConfig
 
     public InstanceConfig set(String fieldName, Object value)
     {
-        if (value == null)
-            value = NULL;
         getParams(fieldName).put(fieldName, value);
         return this;
     }
 
-    private InstanceConfig forceSet(String fieldName, Object value)
+    public InstanceConfig forceSet(String fieldName, Object value)
     {
-        if (value == null)
-            value = NULL;
-
-        // test value
         getParams(fieldName).put(fieldName, value);
         return this;
     }
@@ -251,7 +239,7 @@ public class InstanceConfig implements IInstanceConfig
     public static InstanceConfig generate(int nodeNum,
                                           INodeProvisionStrategy provisionStrategy,
                                           NetworkTopology networkTopology,
-                                          File root,
+                                          Path root,
                                           String token,
                                           int datadirCount)
     {
@@ -273,9 +261,9 @@ public class InstanceConfig implements IInstanceConfig
                                   provisionStrategy.nativeTransportPort(nodeNum));
     }
 
-    private static String[] datadirs(int datadirCount, File root, int nodeNum)
+    private static String[] datadirs(int datadirCount, Path root, int nodeNum)
     {
-        String datadirFormat = String.format("%s/node%d/data%%d", root.path(), nodeNum);
+        String datadirFormat = String.format("%s/node%d/data%%d", root, nodeNum);
         String [] datadirs = new String[datadirCount];
         for (int i = 0; i < datadirs.length; i++)
             datadirs[i] = String.format(datadirFormat, i);
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/InstanceKiller.java b/test/distributed/org/apache/cassandra/distributed/impl/InstanceKiller.java
index e7ca49b..99fc75d 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/InstanceKiller.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/InstanceKiller.java
@@ -43,6 +43,7 @@ public class InstanceKiller extends JVMStabilityInspector.Killer
         // the bad part is that System.exit kills the JVM, so all code which calls kill won't hit the
         // next line; yet in in-JVM dtests System.exit is not desirable, so need to rely on a runtime exception
         // as a means to try to stop execution
+        // TODO (now): this is only used for one dtest, and can cause infinite loops with Simulator in e.g. AbstractCommitLogSegmentManager (failing its first assert, invoking the handler, throwing this exception, restarting)
         throw new InstanceShutdown();
     }
 
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java
index a612781..2779110 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java
@@ -28,8 +28,8 @@ import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.net.URLClassLoader;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadFactory;
@@ -44,7 +44,8 @@ import java.util.function.Supplier;
 import org.slf4j.LoggerFactory;
 
 import ch.qos.logback.classic.LoggerContext;
-import org.apache.cassandra.concurrent.NamedThreadFactory;
+import io.netty.util.concurrent.FastThreadLocal;
+import org.apache.cassandra.concurrent.ExecutorFactory;
 import org.apache.cassandra.distributed.api.IIsolatedExecutor;
 import org.apache.cassandra.utils.ExecutorUtils;
 import org.apache.cassandra.utils.Throwables;
@@ -55,15 +56,33 @@ public class IsolatedExecutor implements IIsolatedExecutor
 {
     final ExecutorService isolatedExecutor;
     private final String name;
-    private final ClassLoader classLoader;
-    private final Method deserializeOnInstance;
+    final ClassLoader classLoader;
+    private final DynamicFunction<Serializable> transfer;
 
-    IsolatedExecutor(String name, ClassLoader classLoader)
+    IsolatedExecutor(String name, ClassLoader classLoader, ExecutorFactory executorFactory)
+    {
+        this(name, classLoader, executorFactory.pooled("isolatedExecutor", Integer.MAX_VALUE));
+    }
+
+    IsolatedExecutor(String name, ClassLoader classLoader, ExecutorService executorService)
     {
         this.name = name;
-        this.isolatedExecutor = Executors.newCachedThreadPool(new NamedThreadFactory("isolatedExecutor", Thread.NORM_PRIORITY, classLoader, new ThreadGroup(name)));
+        this.isolatedExecutor = executorService;
         this.classLoader = classLoader;
-        this.deserializeOnInstance = lookupDeserializeOneObject(classLoader);
+        this.transfer = transferTo(classLoader);
+    }
+
+    protected IsolatedExecutor(IsolatedExecutor from, ExecutorService executor)
+    {
+        this.name = from.name;
+        this.isolatedExecutor = executor;
+        this.classLoader = from.classLoader;
+        this.transfer = from.transfer;
+    }
+
+    public IIsolatedExecutor with(ExecutorService executor)
+    {
+        return new IsolatedExecutor(this, executor);
     }
 
     public Future<Void> shutdown()
@@ -93,6 +112,8 @@ public class IsolatedExecutor implements IIsolatedExecutor
                 LoggerContext loggerContext = (LoggerContext) LoggerFactory.getILoggerFactory();
                 loggerContext.stop();
 
+                FastThreadLocal.destroy();
+
                 // Close the instance class loader after shutting down the isolatedExecutor and logging
                 // in case error handling triggers loading additional classes
                 ((URLClassLoader) classLoader).close();
@@ -120,6 +141,9 @@ public class IsolatedExecutor implements IIsolatedExecutor
     public <I1, I2> BiFunction<I1, I2, Future<?>> async(BiConsumer<I1, I2> consumer) { return (a, b) -> isolatedExecutor.submit(() -> consumer.accept(a, b)); }
     public <I1, I2> BiConsumer<I1, I2> sync(BiConsumer<I1, I2> consumer) { return (a, b) -> waitOn(async(consumer).apply(a, b)); }
 
+    public <I1, I2, I3> TriFunction<I1, I2, I3, Future<?>> async(TriConsumer<I1, I2, I3> consumer) { return (a, b, c) -> isolatedExecutor.submit(() -> consumer.accept(a, b, c)); }
+    public <I1, I2, I3> TriConsumer<I1, I2, I3> sync(TriConsumer<I1, I2, I3> consumer) { return (a, b, c) -> waitOn(async(consumer).apply(a, b, c)); }
+
     public <I, O> Function<I, Future<O>> async(Function<I, O> f) { return (a) -> isolatedExecutor.submit(() -> f.apply(a)); }
     public <I, O> Function<I, O> sync(Function<I, O> f) { return (a) -> waitOn(async(f).apply(a)); }
 
@@ -129,17 +153,42 @@ public class IsolatedExecutor implements IIsolatedExecutor
     public <I1, I2, I3, O> TriFunction<I1, I2, I3, Future<O>> async(TriFunction<I1, I2, I3, O> f) { return (a, b, c) -> isolatedExecutor.submit(() -> f.apply(a, b, c)); }
     public <I1, I2, I3, O> TriFunction<I1, I2, I3, O> sync(TriFunction<I1, I2, I3, O> f) { return (a, b, c) -> waitOn(async(f).apply(a, b, c)); }
 
-    public <E extends Serializable> E transfer(E object)
+    public <I1, I2, I3, I4, O> QuadFunction<I1, I2, I3, I4, Future<O>> async(QuadFunction<I1, I2, I3, I4, O> f) { return (a, b, c, d) -> isolatedExecutor.submit(() -> f.apply(a, b, c, d)); }
+    public <I1, I2, I3, I4, O> QuadFunction<I1, I2, I3, I4, O> sync(QuadFunction<I1, I2, I3, I4, O> f) { return (a, b, c, d) -> waitOn(async(f).apply(a, b, c, d)); }
+
+    public <I1, I2, I3, I4, I5, O> QuintFunction<I1, I2, I3, I4, I5, Future<O>> async(QuintFunction<I1, I2, I3, I4, I5, O> f) { return (a, b, c, d, e) -> isolatedExecutor.submit(() -> f.apply(a, b, c, d, e)); }
+    public <I1, I2, I3, I4, I5, O> QuintFunction<I1, I2, I3, I4, I5, O> sync(QuintFunction<I1, I2, I3, I4, I5, O> f) { return (a, b, c, d,e ) -> waitOn(async(f).apply(a, b, c, d, e)); }
+
+    public Executor executor()
+    {
+        return isolatedExecutor;
+    }
+
+    public <T extends Serializable> T transfer(T in)
     {
-        return (E) transferOneObject(object, classLoader, deserializeOnInstance);
+        return transfer.apply(in);
     }
 
-    static <E extends Serializable> E transferAdhoc(E object, ClassLoader classLoader)
+    public static <T extends Serializable> T transferAdhoc(T object, ClassLoader classLoader)
     {
-        return transferOneObject(object, classLoader, lookupDeserializeOneObject(classLoader));
+        return transferOneObjectAdhoc(object, classLoader, lookupDeserializeOneObject(classLoader));
     }
 
-    private static <E extends Serializable> E transferOneObject(E object, ClassLoader classLoader, Method deserializeOnInstance)
+    private static final SerializableFunction<byte[], Object> DESERIALIZE_ONE_OBJECT = IsolatedExecutor::deserializeOneObject;
+
+    public static DynamicFunction<Serializable> transferTo(ClassLoader classLoader)
+    {
+        SerializableFunction<byte[], Object> deserializeOneObject = transferAdhoc(DESERIALIZE_ONE_OBJECT, classLoader);
+        return new DynamicFunction<Serializable>()
+        {
+            public <T extends Serializable> T apply(T in)
+            {
+                return (T) deserializeOneObject.apply(serializeOneObject(in));
+            }
+        };
+    }
+
+    private static <T extends Serializable> T transferOneObjectAdhoc(T object, ClassLoader classLoader, Method deserializeOnInstance)
     {
         byte[] bytes = serializeOneObject(object);
         try
@@ -148,11 +197,11 @@ public class IsolatedExecutor implements IIsolatedExecutor
             if (onInstance.getClass().getClassLoader() != classLoader)
                 throw new IllegalStateException(onInstance + " seemingly from wrong class loader: " + onInstance.getClass().getClassLoader() + ", but expected " + classLoader);
 
-            return (E) onInstance;
+            return (T) onInstance;
         }
         catch (IllegalAccessException | InvocationTargetException e)
         {
-            throw new RuntimeException("Error while transfering object to " + classLoader, e);
+            throw new RuntimeException(e);
         }
     }
 
@@ -214,22 +263,4 @@ public class IsolatedExecutor implements IIsolatedExecutor
         }
     }
 
-    public interface ThrowingRunnable
-    {
-        public void run() throws Throwable;
-
-        public static Runnable toRunnable(ThrowingRunnable runnable)
-        {
-            return () -> {
-                try
-                {
-                    runnable.run();
-                }
-                catch (Throwable throwable)
-                {
-                    throw new RuntimeException(throwable);
-                }
-            };
-        }
-    }
 }
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Query.java b/test/distributed/org/apache/cassandra/distributed/impl/Query.java
new file mode 100644
index 0000000..1b03996
--- /dev/null
+++ b/test/distributed/org/apache/cassandra/distributed/impl/Query.java
@@ -0,0 +1,110 @@
+/*
+ * 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.distributed.impl;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.distributed.impl.Coordinator.toCassandraCL;
+
+// TODO: maybe just keep with Simulator?
+public class Query implements IIsolatedExecutor.SerializableCallable<Object[][]>
+{
+    final String query;
+    final long timestamp;
+    final org.apache.cassandra.distributed.api.ConsistencyLevel commitConsistencyOrigin;
+    final org.apache.cassandra.distributed.api.ConsistencyLevel serialConsistencyOrigin;
+    final Object[] boundValues;
+
+    public Query(String query, long timestamp, org.apache.cassandra.distributed.api.ConsistencyLevel commitConsistencyOrigin, org.apache.cassandra.distributed.api.ConsistencyLevel serialConsistencyOrigin, Object[] boundValues)
+    {
+        this.query = query;
+        this.timestamp = timestamp;
+        this.commitConsistencyOrigin = commitConsistencyOrigin;
+        this.serialConsistencyOrigin = serialConsistencyOrigin;
+        this.boundValues = boundValues;
+    }
+
+    public Object[][] call()
+    {
+        ConsistencyLevel commitConsistency = toCassandraCL(commitConsistencyOrigin);
+        ConsistencyLevel serialConsistency = serialConsistencyOrigin == null ? null : toCassandraCL(serialConsistencyOrigin);
+        ClientState clientState = Coordinator.makeFakeClientState();
+        CQLStatement prepared = QueryProcessor.getStatement(query, clientState);
+        List<ByteBuffer> boundBBValues = new ArrayList<>();
+        for (Object boundValue : boundValues)
+            boundBBValues.add(ByteBufferUtil.objectToBytes(boundValue));
+
+        prepared.validate(QueryState.forInternalCalls().getClientState());
+
+        // Start capturing warnings on this thread. Note that this will implicitly clear out any previous
+        // warnings as it sets a new State instance on the ThreadLocal.
+        ClientWarn.instance.captureWarnings();
+
+        ResultMessage res = prepared.execute(QueryState.forInternalCalls(),
+                                             QueryOptions.create(commitConsistency,
+                                                                 boundBBValues,
+                                                                 false,
+                                                                 Integer.MAX_VALUE,
+                                                                 null,
+                                                                 serialConsistency,
+                                                                 ProtocolVersion.V4,
+                                                                 null,
+                                                                 timestamp,
+                                                                 FBUtilities.nowInSeconds()),
+                                             System.nanoTime());
+
+        // Collect warnings reported during the query.
+        if (res != null)
+            res.setWarnings(ClientWarn.instance.getWarnings());
+
+        return RowUtil.toQueryResult(res).toObjectArrays();
+    }
+
+    public String toString()
+    {
+        return String.format(query.replaceAll("\\?", "%s") + " AT " + commitConsistencyOrigin, boundValues);
+    }
+
+    static org.apache.cassandra.db.ConsistencyLevel toCassandraCL(org.apache.cassandra.distributed.api.ConsistencyLevel cl)
+    {
+        return org.apache.cassandra.db.ConsistencyLevel.fromCode(cl.ordinal());
+    }
+
+    static final org.apache.cassandra.distributed.api.ConsistencyLevel[] API_CLs = org.apache.cassandra.distributed.api.ConsistencyLevel.values();
+    static org.apache.cassandra.distributed.api.ConsistencyLevel fromCassandraCL(org.apache.cassandra.db.ConsistencyLevel cl)
+    {
+        return API_CLs[cl.ordinal()];
+    }
+
+}
\ No newline at end of file
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/UnsafeGossipHelper.java b/test/distributed/org/apache/cassandra/distributed/impl/UnsafeGossipHelper.java
new file mode 100644
index 0000000..fd166f5
--- /dev/null
+++ b/test/distributed/org/apache/cassandra/distributed/impl/UnsafeGossipHelper.java
@@ -0,0 +1,265 @@
+/*
+ * 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.distributed.impl;
+
+import java.io.Serializable;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.UUID;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.distributed.api.IInstance;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor;
+import org.apache.cassandra.gms.ApplicationState;
+import org.apache.cassandra.gms.EndpointState;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.gms.VersionedValue;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.service.PendingRangeCalculatorService;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static java.util.Collections.singleton;
+import static org.apache.cassandra.distributed.impl.DistributedTestSnitch.toCassandraInetAddressAndPort;
+import static org.apache.cassandra.locator.InetAddressAndPort.getByAddress;
+
+public class UnsafeGossipHelper
+{
+    public static class HostInfo implements Serializable
+    {
+        final InetSocketAddress address;
+        final UUID hostId;
+        final String tokenString;
+        final int messagingVersion;
+        final boolean isShutdown;
+
+        private HostInfo(InetSocketAddress address, UUID hostId, String tokenString, int messagingVersion, boolean isShutdown)
+        {
+            this.address = address;
+            this.hostId = hostId;
+            this.tokenString = tokenString;
+            this.messagingVersion = messagingVersion;
+            this.isShutdown = isShutdown;
+        }
+
+        private HostInfo(IInstance instance)
+        {
+            this(instance, instance.config().hostId(), instance.config().getString("initial_token"));
+        }
+
+        private HostInfo(IInstance instance, UUID hostId, String tokenString)
+        {
+            this(instance.broadcastAddress(), hostId, tokenString, instance.getMessagingVersion(), instance.isShutdown());
+        }
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable addToRingRunner(IIsolatedExecutor.SerializableBiFunction<VersionedValue.VersionedValueFactory, Collection<Token>, VersionedValue> statusFactory, InetSocketAddress address, UUID hostId, String tokenString, int messagingVersion, boolean isShutdown)
+    {
+        return () -> {
+            try
+            {
+                IPartitioner partitioner = DatabaseDescriptor.getPartitioner();
+                InetAddressAndPort addressAndPort = getByAddress(address);
+                Token token;
+                if (FBUtilities.getBroadcastAddressAndPort().equals(addressAndPort))
+                {
+                    if (tokenString == null)
+                    {
+                        token = Iterables.getOnlyElement(SystemKeyspace.getSavedTokens());
+                    }
+                    else
+                    {
+                        token = DatabaseDescriptor.getPartitioner().getTokenFactory().fromString(tokenString);
+                        SystemKeyspace.setLocalHostId(hostId);
+                        SystemKeyspace.updateTokens(singleton(token));
+                    }
+                }
+                else
+                {
+                    if (tokenString == null)
+                        throw new IllegalArgumentException();
+
+                    token = DatabaseDescriptor.getPartitioner().getTokenFactory().fromString(tokenString);
+                }
+
+                Gossiper.runInGossipStageBlocking(() -> {
+                    EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(addressAndPort);
+                    if (state == null)
+                    {
+                        Gossiper.instance.initializeNodeUnsafe(addressAndPort, hostId, 1);
+                        state = Gossiper.instance.getEndpointStateForEndpoint(addressAndPort);
+                        Gossiper.instance.realMarkAlive(addressAndPort, state);
+                    }
+
+                    state.addApplicationState(ApplicationState.TOKENS, new VersionedValue.VersionedValueFactory(partitioner).tokens(singleton(token)));
+                    VersionedValue status = statusFactory.apply(new VersionedValue.VersionedValueFactory(partitioner), singleton(token));
+                    state.addApplicationState(ApplicationState.STATUS_WITH_PORT, status);
+                    StorageService.instance.onChange(addressAndPort, ApplicationState.STATUS_WITH_PORT, status);
+                });
+
+                int setMessagingVersion = isShutdown
+                                          ? MessagingService.current_version
+                                          : Math.min(MessagingService.current_version, messagingVersion);
+                MessagingService.instance().versions.set(addressAndPort, setMessagingVersion);
+
+                PendingRangeCalculatorService.instance.blockUntilFinished();
+            }
+            catch (Throwable e) // UnknownHostException
+            {
+                throw new RuntimeException(e);
+            }
+        };
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable addToRingNormalRunner(IInstance peer)
+    {
+        return addToRingNormalRunner(new HostInfo(peer));
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable addToRingNormalRunner(IInstance peer, UUID hostId, String tokenString)
+    {
+        return addToRingNormalRunner(new HostInfo(peer, hostId, tokenString));
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable addToRingNormalRunner(HostInfo info)
+    {
+        return addToRingNormalRunner(info.address, info.hostId, info.tokenString, info.messagingVersion, info.isShutdown);
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable addToRingNormalRunner(InetSocketAddress address, UUID hostId, String tokenString, int messagingVersion, boolean isShutdown)
+    {
+        return addToRingRunner(VersionedValue.VersionedValueFactory::normal, address, hostId, tokenString, messagingVersion, isShutdown);
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable addToRingRunner(IIsolatedExecutor.SerializableBiFunction<VersionedValue.VersionedValueFactory, Collection<Token>, VersionedValue> statusFactory, HostInfo info)
+    {
+        return addToRingRunner(statusFactory, info.address, info.hostId, info.tokenString, info.messagingVersion, info.isShutdown);
+    }
+
+    // reset gossip state so we know of the node being alive only
+    public static IIsolatedExecutor.SerializableRunnable removeFromRingRunner(IInstance instance)
+    {
+        return removeFromRingRunner(new HostInfo(instance));
+    }
+
+    // reset gossip state so we know of the node being alive only
+    public static IIsolatedExecutor.SerializableRunnable removeFromRingRunner(HostInfo info)
+    {
+        return removeFromRingRunner(info.address, info.hostId, info.tokenString);
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable removeFromRingRunner(InetSocketAddress address, UUID hostId, String tokenString)
+    {
+        return () -> {
+
+            try
+            {
+                IPartitioner partitioner = DatabaseDescriptor.getPartitioner();
+                Token token = partitioner.getTokenFactory().fromString(tokenString);
+                InetAddressAndPort addressAndPort = toCassandraInetAddressAndPort(address);
+
+                Gossiper.runInGossipStageBlocking(() -> {
+                    StorageService.instance.onChange(addressAndPort,
+                                                     ApplicationState.STATUS,
+                                                     new VersionedValue.VersionedValueFactory(partitioner).left(singleton(token), 0L));
+                    Gossiper.instance.unsafeAnnulEndpoint(addressAndPort);
+                    Gossiper.instance.initializeNodeUnsafe(addressAndPort, hostId, 1);
+                    Gossiper.instance.realMarkAlive(addressAndPort, Gossiper.instance.getEndpointStateForEndpoint(addressAndPort));
+                });
+                PendingRangeCalculatorService.instance.blockUntilFinished();
+            }
+            catch (Throwable e) // UnknownHostException
+            {
+                throw new RuntimeException(e);
+            }
+        };
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable addToRingBootstrappingRunner(IInstance peer)
+    {
+        return addToRingRunner(VersionedValue.VersionedValueFactory::bootstrapping, new HostInfo(peer));
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable addToRingBootstrapReplacingRunner(IInstance peer, IInvokableInstance replacing, UUID hostId, String tokenString)
+    {
+        return addToRingBootstrapReplacingRunner(peer, replacing.broadcastAddress(), hostId, tokenString);
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable addToRingBootstrapReplacingRunner(IInstance peer, InetSocketAddress replacingAddress, UUID hostId, String tokenString)
+    {
+        return addToRingRunner((factory, ignore) -> factory.bootReplacingWithPort(getByAddress(replacingAddress)), new HostInfo(peer, hostId, tokenString));
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable addToRingNormalReplacedRunner(IInstance peer, IInstance replaced)
+    {
+        return addToRingNormalReplacedRunner(peer, replaced.broadcastAddress());
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable addToRingNormalReplacedRunner(IInstance peer, InetSocketAddress replacedAddress)
+    {
+        return addToRingRunner((factory, ignore) -> factory.bootReplacingWithPort(getByAddress(replacedAddress)), new HostInfo(peer, null, null));
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable addToRingLeavingRunner(IInstance peer)
+    {
+        return addToRingRunner(VersionedValue.VersionedValueFactory::leaving, new HostInfo(peer, null, null));
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable addToRingLeftRunner(IInstance peer)
+    {
+        return addToRingRunner((factory, tokens) -> factory.left(tokens, Long.MAX_VALUE), new HostInfo(peer, null, null));
+    }
+
+    public static void removeFromRing(IInstance peer)
+    {
+        removeFromRingRunner(peer).run();
+    }
+
+    public static void addToRingNormal(IInstance peer)
+    {
+        addToRingNormalRunner(peer).run();
+        assert StorageService.instance.getTokenMetadata().isMember(toCassandraInetAddressAndPort(peer.broadcastAddress()));
+    }
+
+    public static void addToRingBootstrapping(IInstance peer)
+    {
+        addToRingBootstrappingRunner(peer).run();
+    }
+
+    public static IIsolatedExecutor.SerializableRunnable markShutdownRunner(InetSocketAddress address)
+    {
+        return () -> {
+            IPartitioner partitioner = DatabaseDescriptor.getPartitioner();
+            Gossiper.runInGossipStageBlocking(() -> {
+                EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(getByAddress(address));
+                VersionedValue status = new VersionedValue.VersionedValueFactory(partitioner).shutdown(true);
+                state.addApplicationState(ApplicationState.STATUS, status);
+                state.getHeartBeatState().forceHighestPossibleVersionUnsafe();
+                StorageService.instance.onChange(getByAddress(address), ApplicationState.STATUS, status);
+            });
+        };
+    }
+}
diff --git a/test/distributed/org/apache/cassandra/distributed/test/CASTest.java b/test/distributed/org/apache/cassandra/distributed/test/CASTest.java
index 91d26f8..8dd5977 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/CASTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/CASTest.java
@@ -39,7 +39,7 @@ import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.ICoordinator;
 import org.apache.cassandra.distributed.api.IInstance;
 import org.apache.cassandra.distributed.api.IMessageFilters;
-import org.apache.cassandra.distributed.impl.Instance;
+import org.apache.cassandra.distributed.impl.UnsafeGossipHelper;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.UUIDGen;
@@ -384,7 +384,7 @@ public class CASTest extends TestBaseImpl
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v1 int, v2 int, PRIMARY KEY (pk, ck))");
 
             // make it so {1} is unaware (yet) that {4} is an owner of the token
-            cluster.get(1).acceptsOnInstance(Instance::removeFromRing).accept(cluster.get(4));
+            cluster.get(1).acceptsOnInstance(UnsafeGossipHelper::removeFromRing).accept(cluster.get(4));
 
             int pk = pk(cluster, 1, 2);
 
@@ -396,7 +396,7 @@ public class CASTest extends TestBaseImpl
                     row(true));
 
             for (int i = 1 ; i <= 3 ; ++i)
-                cluster.get(i).acceptsOnInstance(Instance::addToRingNormal).accept(cluster.get(4));
+                cluster.get(i).acceptsOnInstance(UnsafeGossipHelper::addToRingNormal).accept(cluster.get(4));
 
             // {4} reads from !{2} => {3, 4}
             cluster.filters().verbs(PAXOS_PREPARE_REQ.id, READ_REQ.id).from(4).to(2).drop();
@@ -425,7 +425,7 @@ public class CASTest extends TestBaseImpl
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v1 int, v2 int, PRIMARY KEY (pk, ck))");
 
             // make it so {1} is unaware (yet) that {4} is an owner of the token
-            cluster.get(1).acceptsOnInstance(Instance::removeFromRing).accept(cluster.get(4));
+            cluster.get(1).acceptsOnInstance(UnsafeGossipHelper::removeFromRing).accept(cluster.get(4));
 
             // {4} promises, accepts and commits on !{2} => {3, 4}
             int pk = pk(cluster, 1, 2);
@@ -466,8 +466,8 @@ public class CASTest extends TestBaseImpl
 
             // make it so {4} is bootstrapping, and this has not propagated to other nodes yet
             for (int i = 1 ; i <= 4 ; ++i)
-                cluster.get(1).acceptsOnInstance(Instance::removeFromRing).accept(cluster.get(4));
-            cluster.get(4).acceptsOnInstance(Instance::addToRingBootstrapping).accept(cluster.get(4));
+                cluster.get(1).acceptsOnInstance(UnsafeGossipHelper::removeFromRing).accept(cluster.get(4));
+            cluster.get(4).acceptsOnInstance(UnsafeGossipHelper::addToRingBootstrapping).accept(cluster.get(4));
 
             int pk = pk(cluster, 1, 2);
 
@@ -480,7 +480,7 @@ public class CASTest extends TestBaseImpl
 
             // finish topology change
             for (int i = 1 ; i <= 4 ; ++i)
-                cluster.get(i).acceptsOnInstance(Instance::addToRingNormal).accept(cluster.get(4));
+                cluster.get(i).acceptsOnInstance(UnsafeGossipHelper::addToRingNormal).accept(cluster.get(4));
 
             // {3} reads from !{2} => {3, 4}
             cluster.filters().verbs(PAXOS_PREPARE_REQ.id, READ_REQ.id).from(3).to(2).drop();
@@ -510,8 +510,8 @@ public class CASTest extends TestBaseImpl
 
             // make it so {4} is bootstrapping, and this has not propagated to other nodes yet
             for (int i = 1 ; i <= 4 ; ++i)
-                cluster.get(1).acceptsOnInstance(Instance::removeFromRing).accept(cluster.get(4));
-            cluster.get(4).acceptsOnInstance(Instance::addToRingBootstrapping).accept(cluster.get(4));
+                cluster.get(1).acceptsOnInstance(UnsafeGossipHelper::removeFromRing).accept(cluster.get(4));
+            cluster.get(4).acceptsOnInstance(UnsafeGossipHelper::addToRingBootstrapping).accept(cluster.get(4));
 
             int pk = pk(cluster, 1, 2);
 
@@ -524,7 +524,7 @@ public class CASTest extends TestBaseImpl
 
             // finish topology change
             for (int i = 1 ; i <= 4 ; ++i)
-                cluster.get(i).acceptsOnInstance(Instance::addToRingNormal).accept(cluster.get(4));
+                cluster.get(i).acceptsOnInstance(UnsafeGossipHelper::addToRingNormal).accept(cluster.get(4));
 
             // {3} reads from !{2} => {3, 4}
             cluster.filters().verbs(PAXOS_PREPARE_REQ.id, READ_REQ.id).from(3).to(2).drop();
@@ -562,8 +562,8 @@ public class CASTest extends TestBaseImpl
 
             // make it so {4} is bootstrapping, and this has not propagated to other nodes yet
             for (int i = 1 ; i <= 4 ; ++i)
-                cluster.get(1).acceptsOnInstance(Instance::removeFromRing).accept(cluster.get(4));
-            cluster.get(4).acceptsOnInstance(Instance::addToRingBootstrapping).accept(cluster.get(4));
+                cluster.get(1).acceptsOnInstance(UnsafeGossipHelper::removeFromRing).accept(cluster.get(4));
+            cluster.get(4).acceptsOnInstance(UnsafeGossipHelper::addToRingBootstrapping).accept(cluster.get(4));
 
             int pk = pk(cluster, 1, 2);
 
@@ -589,7 +589,7 @@ public class CASTest extends TestBaseImpl
 
             // finish topology change
             for (int i = 1 ; i <= 4 ; ++i)
-                cluster.get(i).acceptsOnInstance(Instance::addToRingNormal).accept(cluster.get(4));
+                cluster.get(i).acceptsOnInstance(UnsafeGossipHelper::addToRingNormal).accept(cluster.get(4));
 
             // {3} reads from !{2} => {3, 4}
             cluster.filters().verbs(PAXOS_PREPARE_REQ.id, READ_REQ.id).from(3).to(2).drop();
@@ -626,8 +626,8 @@ public class CASTest extends TestBaseImpl
 
             // make it so {4} is bootstrapping, and this has not propagated to other nodes yet
             for (int i = 1 ; i <= 4 ; ++i)
-                cluster.get(1).acceptsOnInstance(Instance::removeFromRing).accept(cluster.get(4));
-            cluster.get(4).acceptsOnInstance(Instance::addToRingBootstrapping).accept(cluster.get(4));
+                cluster.get(1).acceptsOnInstance(UnsafeGossipHelper::removeFromRing).accept(cluster.get(4));
+            cluster.get(4).acceptsOnInstance(UnsafeGossipHelper::addToRingBootstrapping).accept(cluster.get(4));
 
             int pk = pk(cluster, 1, 2);
 
@@ -653,7 +653,7 @@ public class CASTest extends TestBaseImpl
 
             // finish topology change
             for (int i = 1 ; i <= 4 ; ++i)
-                cluster.get(i).acceptsOnInstance(Instance::addToRingNormal).accept(cluster.get(4));
+                cluster.get(i).acceptsOnInstance(UnsafeGossipHelper::addToRingNormal).accept(cluster.get(4));
 
             // {3} reads from !{2} => {3, 4}
             cluster.filters().verbs(PAXOS_PREPARE_REQ.id, READ_REQ.id).from(3).to(2).drop();
diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java b/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java
index 316a5d5..6fb5805 100644
--- a/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java
+++ b/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java
@@ -39,6 +39,7 @@ import org.apache.cassandra.distributed.api.ICluster;
 import org.apache.cassandra.distributed.api.IInstanceConfig;
 import org.apache.cassandra.distributed.impl.Instance;
 import org.apache.cassandra.distributed.shared.DistributedTestBase;
+import org.apache.cassandra.distributed.shared.ThrowingRunnable;
 import org.apache.cassandra.distributed.shared.Versions;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
@@ -108,7 +109,7 @@ public class UpgradeTestBase extends DistributedTestBase
         }
     }
 
-    public static class TestCase implements Instance.ThrowingRunnable
+    public static class TestCase implements ThrowingRunnable
     {
         private final Versions versions;
         private final List<TestVersions> upgrade = new ArrayList<>();
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneListTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneListTest.java
index bbe06a8..7c9c7d4 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneListTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneListTest.java
@@ -32,6 +32,7 @@ import org.junit.Test;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.distributed.impl.IsolatedExecutor;
+import org.apache.cassandra.distributed.shared.ThrowingRunnable;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -610,7 +611,7 @@ public class RangeTombstoneListTest
         }
     }
 
-    private void assertHasException(IsolatedExecutor.ThrowingRunnable block, Consumer<Throwable> verifier)
+    private void assertHasException(ThrowingRunnable block, Consumer<Throwable> verifier)
     {
         try
         {
diff --git a/test/unit/org/apache/cassandra/utils/concurrent/ImmediateFutureTest.java b/test/unit/org/apache/cassandra/utils/concurrent/ImmediateFutureTest.java
index b042a28..f1ad6d5 100644
--- a/test/unit/org/apache/cassandra/utils/concurrent/ImmediateFutureTest.java
+++ b/test/unit/org/apache/cassandra/utils/concurrent/ImmediateFutureTest.java
@@ -28,6 +28,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.distributed.impl.IsolatedExecutor;
+import org.apache.cassandra.distributed.shared.ThrowingRunnable;
 
 public class ImmediateFutureTest
 {
@@ -88,7 +89,7 @@ public class ImmediateFutureTest
         testSimple(p, isCancelled);
     }
 
-    private static void assertFailure(IsolatedExecutor.ThrowingRunnable run, Predicate<Throwable> test)
+    private static void assertFailure(ThrowingRunnable run, Predicate<Throwable> test)
     {
         Throwable failure = null;
         try

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


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

Posted by be...@apache.org.
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 be1f050bc8c0cd695a42952e3fc84625ad48d83a
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


[cassandra] 01/06: [CASSANDRA-16924] CEP-10 Phase 1: Mockable Blocking Concurrency Primitives

Posted by be...@apache.org.
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 e5b92e108851d2be81c43cbf982564f92d28aa6b
Author: Benedict Elliott Smith <be...@apache.org>
AuthorDate: Wed Jul 28 19:24:43 2021 +0100

    [CASSANDRA-16924] CEP-10 Phase 1: Mockable Blocking Concurrency Primitives
    
    patch by Benedict; reviewed by Sam Tunnicliffe and Aleksei Zotov for CASSANDRA-16924
    
    Co-authored-by: Benedict Elliott Smith <be...@apache.org>
    Co-authored-by: Sam Tunnicliffe <sa...@apache.org>
---
 checkstyle_suppressions.xml                        |   2 +-
 .../apache/cassandra/cache/AutoSavingCache.java    |  30 +-
 .../AbstractLocalAwareExecutorService.java         |   7 +-
 .../concurrent/JMXEnabledSingleThreadExecutor.java |   7 +-
 .../cassandra/concurrent/NamedThreadFactory.java   |   2 -
 .../apache/cassandra/concurrent/SEPExecutor.java   |  18 +-
 .../org/apache/cassandra/concurrent/Stage.java     |   7 +-
 src/java/org/apache/cassandra/cql3/CQL3Type.java   |   1 -
 .../cassandra/cql3/functions/AbstractFunction.java |   1 -
 .../cql3/functions/UDFExecutorService.java         |  14 +-
 .../cassandra/cql3/functions/UDFunction.java       |   5 +-
 .../cassandra/cql3/functions/types/TypeCodec.java  |   2 -
 .../cql3/statements/ModificationStatement.java     |   2 -
 .../statements/schema/CreateFunctionStatement.java |   2 -
 .../cql3/statements/schema/DropIndexStatement.java |   1 -
 .../org/apache/cassandra/db/BufferClustering.java  |   1 -
 .../apache/cassandra/db/BufferClusteringBound.java |   1 -
 .../cassandra/db/BufferClusteringBoundary.java     |   1 -
 .../org/apache/cassandra/db/ColumnFamilyStore.java |  14 +-
 .../apache/cassandra/db/DiskBoundaryManager.java   |   1 -
 src/java/org/apache/cassandra/db/Keyspace.java     |   3 +-
 .../apache/cassandra/db/MutableDeletionInfo.java   |   1 -
 src/java/org/apache/cassandra/db/Slice.java        |   1 -
 .../org/apache/cassandra/db/SnapshotCommand.java   |   2 -
 .../org/apache/cassandra/db/SystemKeyspace.java    |   1 -
 .../cassandra/db/UnfilteredDeserializer.java       |   1 -
 .../cassandra/db/aggregation/GroupingState.java    |   1 -
 .../commitlog/AbstractCommitLogSegmentManager.java |  11 +-
 .../db/commitlog/AbstractCommitLogService.java     |   7 +-
 .../apache/cassandra/db/commitlog/CommitLog.java   |   5 +-
 .../cassandra/db/commitlog/CommitLogArchiver.java  |   4 +-
 .../cassandra/db/commitlog/CommitLogSegment.java   |   5 +-
 .../commitlog/CommitLogSegmentManagerStandard.java |   2 -
 .../cassandra/db/commitlog/EncryptedSegment.java   |   1 -
 .../db/compaction/AbstractStrategyHolder.java      |   1 -
 .../cassandra/db/compaction/CompactionManager.java |   1 -
 .../db/compaction/CompactionStrategyHolder.java    |   1 -
 .../db/compaction/PendingRepairHolder.java         |   1 -
 .../cassandra/db/compaction/SSTableSplitter.java   |   1 -
 .../apache/cassandra/db/compaction/Upgrader.java   |   1 -
 .../compaction/writers/CompactionAwareWriter.java  |   1 -
 .../db/lifecycle/ILifecycleTransaction.java        |   1 -
 .../apache/cassandra/db/marshal/DurationType.java  |   1 -
 .../cassandra/db/marshal/SimpleDateType.java       |   1 -
 .../org/apache/cassandra/db/marshal/TimeType.java  |   1 -
 .../apache/cassandra/db/marshal/TimeUUIDType.java  |   1 -
 .../apache/cassandra/db/marshal/TimestampType.java |   1 -
 .../cassandra/db/monitoring/MonitoringTask.java    |   4 +-
 .../apache/cassandra/db/partitions/Partition.java  |   1 -
 .../cassandra/db/partitions/PurgeFunction.java     |   1 -
 .../db/rows/AbstractRangeTombstoneMarker.java      |   2 +-
 .../db/rows/ThrottledUnfilteredIterator.java       |   1 -
 .../streaming/CassandraCompressedStreamReader.java |   1 -
 .../db/streaming/CassandraStreamReceiver.java      |   2 +-
 .../apache/cassandra/db/view/ViewBuilderTask.java  |   1 -
 .../org/apache/cassandra/db/view/ViewUtils.java    |   1 -
 .../cassandra/dht/ByteOrderedPartitioner.java      |   1 -
 src/java/org/apache/cassandra/dht/Datacenters.java |   1 -
 .../org/apache/cassandra/dht/StreamStateStore.java |   1 -
 .../cassandra/diag/DiagnosticEventService.java     |   3 -
 .../org/apache/cassandra/fql/FullQueryLogger.java  |   4 +-
 .../org/apache/cassandra/gms/FailureDetector.java  |   1 -
 src/java/org/apache/cassandra/gms/Gossiper.java    |  11 +-
 .../cassandra/hints/ChecksummedDataInput.java      |   1 -
 .../apache/cassandra/hints/HintsBufferPool.java    |   8 +-
 .../cassandra/hints/HintsDispatchExecutor.java     |  21 +-
 .../cassandra/hints/HintsDispatchTrigger.java      |   3 -
 .../apache/cassandra/hints/HintsDispatcher.java    |  20 +-
 .../org/apache/cassandra/hints/HintsService.java   |  13 +-
 .../apache/cassandra/hints/HintsWriteExecutor.java |   7 +-
 .../cassandra/index/SecondaryIndexManager.java     |  10 +-
 .../internal/composites/CompositesSearcher.java    |   2 -
 .../apache/cassandra/index/sasi/TermIterator.java  |  24 +-
 .../index/sasi/disk/PerSSTableIndexWriter.java     |  20 +-
 .../cassandra/io/compress/CompressionMetadata.java |   2 -
 .../io/sstable/SSTableSimpleUnsortedWriter.java    |   4 +-
 .../cassandra/io/sstable/format/SSTableReader.java |  10 +-
 .../io/util/BufferedDataOutputStreamPlus.java      |   1 -
 .../io/util/UnbufferedDataOutputStreamPlus.java    |   2 -
 .../org/apache/cassandra/locator/Endpoints.java    |   2 -
 .../cassandra/locator/EndpointsByReplica.java      |   2 -
 .../apache/cassandra/locator/InOurDcTester.java    |   2 +-
 .../apache/cassandra/locator/RangesByEndpoint.java |   2 -
 .../locator/ReconnectableSnitchHelper.java         |   2 -
 .../org/apache/cassandra/locator/ReplicaPlans.java |   1 -
 .../apache/cassandra/locator/TokenMetadata.java    |   1 +
 .../cassandra/metrics/CompactionMetrics.java       |   2 -
 .../apache/cassandra/net/AsyncChannelPromise.java  |  27 +-
 .../org/apache/cassandra/net/AsyncOneResponse.java |   7 +-
 .../org/apache/cassandra/net/AsyncPromise.java     | 489 ----------------
 .../cassandra/net/AsyncStreamingInputPlus.java     |   5 +-
 .../org/apache/cassandra/net/FrameDecoderCrc.java  |   1 -
 .../org/apache/cassandra/net/FrameEncoderCrc.java  |   1 -
 .../cassandra/net/FrameEncoderLegacyLZ4.java       |   1 -
 .../cassandra/net/FrameEncoderUnprotected.java     |   1 -
 .../org/apache/cassandra/net/FutureCombiner.java   |   1 +
 .../apache/cassandra/net/HandshakeProtocol.java    |   4 +-
 .../cassandra/net/InboundConnectionInitiator.java  |   3 -
 .../org/apache/cassandra/net/InboundSockets.java   |   1 +
 .../cassandra/net/MessagingServiceMBeanImpl.java   |   1 -
 .../apache/cassandra/net/OutboundConnection.java   |  16 +-
 .../cassandra/net/OutboundConnectionInitiator.java |   2 +-
 .../apache/cassandra/net/OutboundConnections.java  |  32 +-
 .../apache/cassandra/net/OutboundMessageQueue.java |  20 +-
 .../net/StartupClusterConnectivityChecker.java     |  18 +-
 src/java/org/apache/cassandra/net/Verb.java        |   2 -
 .../cassandra/repair/AsymmetricRemoteSyncTask.java |   4 +-
 .../org/apache/cassandra/repair/LocalSyncTask.java |   5 +-
 .../org/apache/cassandra/repair/RepairJob.java     |   7 +-
 .../org/apache/cassandra/repair/RepairJobDesc.java |   1 -
 .../apache/cassandra/repair/RepairRunnable.java    |   7 +-
 .../org/apache/cassandra/repair/RepairSession.java |  11 +-
 .../org/apache/cassandra/repair/SnapshotTask.java  |   9 +-
 .../cassandra/repair/SymmetricRemoteSyncTask.java  |   4 +-
 src/java/org/apache/cassandra/repair/SyncTask.java |   6 +-
 .../repair/ValidationPartitionIterator.java        |   1 -
 .../apache/cassandra/repair/ValidationTask.java    |  11 +-
 .../repair/consistent/ConsistentSession.java       |   1 -
 .../cassandra/repair/consistent/RepairedState.java |   6 -
 .../repair/consistent/admin/PendingStat.java       |   2 -
 .../apache/cassandra/schema/CompressionParams.java |   1 -
 src/java/org/apache/cassandra/schema/Indexes.java  |   1 -
 .../cassandra/schema/MigrationCoordinator.java     |  22 +-
 .../org/apache/cassandra/schema/SchemaEvent.java   |   1 -
 .../cassandra/schema/SchemaMigrationEvent.java     |   2 -
 .../cassandra/serializers/MapSerializer.java       |   1 -
 .../cassandra/serializers/SetSerializer.java       |   1 -
 .../service/AbstractWriteResponseHandler.java      |  54 +-
 .../cassandra/service/ActiveRepairService.java     |  27 +-
 .../apache/cassandra/service/EchoVerbHandler.java  |   1 -
 .../org/apache/cassandra/service/GCInspector.java  |   3 -
 .../service/PendingRangeCalculatorService.java     |   6 +-
 .../org/apache/cassandra/service/QueryState.java   |   1 -
 .../cassandra/service/SnapshotVerbHandler.java     |   5 -
 .../org/apache/cassandra/service/StorageProxy.java |  30 +-
 .../apache/cassandra/service/StorageService.java   |  15 +-
 .../cassandra/service/TruncateResponseHandler.java |  12 +-
 .../cassandra/service/pager/PagingState.java       |   1 -
 .../service/paxos/AbstractPaxosCallback.java       |  12 +-
 .../cassandra/service/paxos/PrepareCallback.java   |   6 +-
 .../cassandra/service/paxos/ProposeCallback.java   |  10 +-
 .../reads/HybridSpeculativeRetryPolicy.java        |   1 -
 .../cassandra/service/reads/ReadCallback.java      |  28 +-
 .../service/reads/ShortReadProtection.java         |   3 -
 .../reads/repair/BlockingPartitionRepair.java      |  18 +-
 .../repair/PartitionIteratorMergeListener.java     |   1 -
 .../cassandra/service/reads/repair/ReadRepair.java |   1 -
 .../reads/repair/ReadRepairDiagnostics.java        |   1 -
 .../service/reads/repair/ReadRepairEvent.java      |   1 -
 .../service/reads/repair/ReadRepairStrategy.java   |   1 -
 .../service/reads/repair/RepairedDataVerifier.java |  11 -
 .../apache/cassandra/streaming/SessionSummary.java |   1 -
 .../apache/cassandra/streaming/StreamManager.java  |  17 +-
 .../cassandra/streaming/StreamResultFuture.java    |  10 +-
 .../streaming/StreamingMessageSender.java          |   3 -
 .../async/NettyStreamingMessageSender.java         |  88 +--
 .../streaming/messages/StreamMessage.java          |   2 -
 .../apache/cassandra/tools/BootstrapMonitor.java   |   6 +-
 src/java/org/apache/cassandra/tools/NodeTool.java  |   1 -
 .../org/apache/cassandra/tools/RepairRunner.java   |  32 +-
 .../apache/cassandra/tools/nodetool/TpStats.java   |   2 -
 .../org/apache/cassandra/tracing/TraceState.java   |   3 +-
 src/java/org/apache/cassandra/transport/Event.java |   1 -
 .../apache/cassandra/transport/SimpleClient.java   |  12 +-
 .../transport/messages/PrepareMessage.java         |   2 -
 src/java/org/apache/cassandra/utils/Clock.java     |  13 +-
 .../cassandra/utils/DiagnosticSnapshotService.java |   1 -
 .../org/apache/cassandra/utils/FBUtilities.java    |   5 +-
 .../Intercept.java}                                |  24 +-
 .../cassandra/utils/JVMStabilityInspector.java     |   4 +
 src/java/org/apache/cassandra/utils/Mx4jTool.java  |   2 -
 .../cassandra/utils/NativeSSTableLoaderClient.java |   4 -
 src/java/org/apache/cassandra/utils/SyncUtil.java  |   1 -
 .../org/apache/cassandra/utils/Throwables.java     |   9 +-
 .../org/apache/cassandra/utils/binlog/BinLog.java  |   5 +-
 .../cassandra/utils/binlog/ExternalArchiver.java   |   7 +-
 .../org/apache/cassandra/utils/btree/BTree.java    |   1 -
 .../cassandra/utils/concurrent/AsyncFuture.java    | 444 +++++++++++++++
 .../cassandra/utils/concurrent/AsyncPromise.java   | 223 ++++++++
 .../cassandra/utils/concurrent/Awaitable.java      | 403 +++++++++++++
 .../apache/cassandra/utils/concurrent/Blocker.java |  63 ---
 .../cassandra/utils/concurrent/BlockingQueues.java | 253 +++++++++
 .../cassandra/utils/concurrent/Condition.java      | 100 ++++
 .../cassandra/utils/concurrent/CountDownLatch.java | 107 ++++
 .../apache/cassandra/utils/concurrent/Future.java  | 121 ++++
 .../utils/concurrent/NotScheduledFuture.java       |  50 +-
 .../apache/cassandra/utils/concurrent/OpOrder.java |  44 +-
 .../apache/cassandra/utils/concurrent/Promise.java | 114 ++++
 .../cassandra/utils/concurrent/Semaphore.java      | 333 +++++++++++
 .../utils/concurrent/SimpleCondition.java          | 107 ----
 .../concurrent/UncheckedInterruptedException.java} |  28 +-
 .../cassandra/utils/concurrent/WaitQueue.java      | 621 ++++++++-------------
 .../cassandra/utils/concurrent/WeightedQueue.java  |   9 +-
 .../cassandra/utils/memory/MemtableAllocator.java  |  13 +-
 .../utils/memory/MemtableCleanerThread.java        |   4 +-
 .../cassandra/utils/memory/MemtablePool.java       |   4 +-
 .../cassandra/utils/memory/NativeAllocator.java    |  10 +-
 .../apache/cassandra/utils/obs/OffHeapBitSet.java  |   2 -
 test/burn/org/apache/cassandra/net/Verifier.java   |   6 +-
 .../distributed/impl/AbstractCluster.java          |  10 +-
 .../cassandra/distributed/impl/Coordinator.java    |   3 -
 .../impl/DelegatingInvokableInstance.java          |   2 -
 .../distributed/impl/IsolatedExecutor.java         |   4 +-
 .../test/AbstractEncryptionOptionsImpl.java        |  21 +-
 .../distributed/test/GossipShutdownTest.java       |  31 +-
 .../distributed/test/IncRepairTruncationTest.java  |  12 +-
 .../distributed/test/PreviewRepairTest.java        |  86 +--
 .../cassandra/distributed/test/ReadRepairTest.java |  17 +-
 .../distributed/test/RepairBoundaryTest.java       |  15 +-
 .../cassandra/distributed/test/RepairTest.java     |  14 +-
 .../cassandra/distributed/test/StreamingTest.java  |   1 -
 .../LongLeveledCompactionStrategyTest.java         |   2 -
 .../RandomReplicationAwareTokenAllocatorTest.java  |   1 -
 .../test/microbench/BTreeSearchIteratorBench.java  |   1 -
 .../test/microbench/DirectorySizerBench.java       |   1 -
 .../test/microbench/FastThreadExecutor.java        |   7 +-
 .../test/microbench/LatencyTrackingBench.java      |   7 -
 .../test/microbench/PendingRangesBench.java        |   1 -
 test/unit/org/apache/cassandra/SchemaLoader.java   |   1 -
 .../apache/cassandra/audit/AuditLoggerTest.java    |   1 -
 .../apache/cassandra/concurrent/WaitQueueTest.java |   5 +-
 .../config/DatabaseDescriptorRefTest.java          |   1 +
 .../config/OverrideConfigurationLoader.java        |   1 -
 .../org/apache/cassandra/cql3/DurationTest.java    |   1 -
 .../unit/org/apache/cassandra/cql3/PagingTest.java |   3 -
 .../cassandra/cql3/functions/CastFctsTest.java     |   1 -
 .../cql3/validation/entities/UFTypesTest.java      |   1 -
 .../apache/cassandra/db/CleanupTransientTest.java  |   1 -
 test/unit/org/apache/cassandra/db/ColumnsTest.java |   3 +-
 .../cassandra/db/DiskBoundaryManagerTest.java      |   1 -
 .../cassandra/db/SerializationHeaderTest.java      |   1 -
 .../cassandra/db/commitlog/CommitLogTest.java      |   8 +-
 .../db/commitlog/CommitlogShutdownTest.java        |   1 -
 .../db/compaction/CancelCompactionsTest.java       |   1 -
 .../db/compaction/CompactionControllerTest.java    |   1 -
 .../db/compaction/CompactionInfoTest.java          |   2 -
 .../db/compaction/ValidationExecutorTest.java      |  13 +-
 .../org/apache/cassandra/db/filter/SliceTest.java  |   1 -
 .../cassandra/db/marshal/CollectionTypesTest.java  |   1 -
 .../db/rows/RowAndDeletionMergeIteratorTest.java   |   1 -
 .../db/rows/ThrottledUnfilteredIteratorTest.java   |   2 -
 .../cassandra/dht/RangeFetchMapCalculatorTest.java |   1 -
 test/unit/org/apache/cassandra/dht/RangeTest.java  |   1 -
 .../apache/cassandra/fql/FullQueryLoggerTest.java  |   1 -
 .../apache/cassandra/gms/EndpointStateTest.java    |   1 -
 .../cassandra/index/sasi/plan/ExpressionTest.java  |   1 -
 .../index/sasi/utils/LongIteratorTest.java         |   2 -
 .../cassandra/io/sstable/CQLSSTableWriterTest.java |   1 -
 .../cassandra/io/sstable/LegacySSTableTest.java    |  11 -
 .../cassandra/io/sstable/SSTableWriterTest.java    |   1 -
 .../io/sstable/SSTableWriterTestBase.java          |   1 -
 .../locator/DynamicEndpointSnitchTest.java         |   1 -
 .../locator/ReconnectableSnitchHelperTest.java     |   1 -
 .../cassandra/locator/ReplicaCollectionTest.java   |   6 +-
 .../apache/cassandra/metrics/BatchMetricsTest.java |   1 -
 .../DecayingEstimatedHistogramReservoirTest.java   |   2 -
 .../cassandra/net/AsyncChannelPromiseTest.java     |   3 +-
 .../unit/org/apache/cassandra/net/FramingTest.java |   2 -
 .../org/apache/cassandra/net/HandshakeTest.java    |   2 +-
 .../org/apache/cassandra/net/MockMessagingSpy.java |   7 +-
 .../net/OutboundConnectionSettingsTest.java        |   2 -
 .../repair/SymmetricRemoteSyncTaskTest.java        |   1 -
 .../org/apache/cassandra/repair/ValidatorTest.java |   2 -
 .../consistent/admin/SchemaArgsParserTest.java     |   3 -
 .../repair/messages/RepairOptionTest.java          |   2 -
 .../cassandra/service/ActiveRepairServiceTest.java |  12 +-
 .../cassandra/service/BootstrapTransientTest.java  |   3 -
 .../org/apache/cassandra/service/MoveTest.java     |   6 -
 .../service/NativeTransportServiceTest.java        |   2 -
 .../cassandra/service/StorageServiceTest.java      |   1 -
 .../service/WriteResponseHandlerTransientTest.java |   2 -
 .../reads/repair/InstrumentedReadRepair.java       |   1 -
 .../reads/repair/ReadOnlyReadRepairTest.java       |   1 -
 .../service/reads/repair/TestableReadRepair.java   |   2 -
 ...ntireSSTableStreamingCorrectFilesCountTest.java |   1 -
 .../compression/CompressedInputStreamTest.java     |   1 -
 .../apache/cassandra/tools/LoaderOptionsTest.java  |   2 -
 .../apache/cassandra/tools/TopPartitionsTest.java  |   1 -
 .../cassandra/transport/CQLConnectionTest.java     |   5 +-
 .../cassandra/transport/CQLUserAuditTest.java      |   4 +-
 .../cassandra/triggers/TriggerExecutorTest.java    |   1 -
 .../apache/cassandra/triggers/TriggersTest.java    |   2 -
 .../cassandra/utils/CassandraVersionTest.java      |   1 -
 .../utils/btree/BTreeSearchIteratorTest.java       |   2 -
 .../apache/cassandra/utils/btree/BTreeTest.java    |   2 -
 .../concurrent/AbstractTestAsyncPromise.java}      |   8 +-
 .../utils/concurrent/AbstractTestAwaitable.java    | 173 ++++++
 .../concurrent/AbstractTestPromise.java}           |  14 +-
 .../concurrent}/AsyncPromiseTest.java              |   7 +-
 .../cassandra/utils/concurrent/ConditionTest.java} |  41 +-
 .../utils/concurrent/CountDownLatchTest.java       |  61 ++
 .../cassandra/utils/concurrent/SemaphoreTest.java  |  98 ++++
 .../utils/concurrent/WeightedQueueTest.java        |  55 +-
 .../cassandra/utils/vint/VIntCodingTest.java       |   1 -
 .../org/apache/cassandra/stress/StressGraph.java   |   1 -
 .../org/apache/cassandra/stress/StressProfile.java |   1 -
 .../stress/operations/SampledOpDistribution.java   |   1 -
 .../operations/SampledOpDistributionFactory.java   |   1 -
 .../stress/operations/userdefined/CASQuery.java    |   1 -
 .../operations/userdefined/TokenRangeQuery.java    |   2 -
 .../userdefined/ValidatingSchemaQuery.java         |   1 -
 .../settings/SettingsCommandPreDefinedMixed.java   |   1 -
 .../stress/settings/SettingsCommandUser.java       |   1 -
 .../cassandra/stress/settings/SettingsGraph.java   |   1 -
 304 files changed, 3526 insertions(+), 2014 deletions(-)

diff --git a/checkstyle_suppressions.xml b/checkstyle_suppressions.xml
index 13ce561..3a2ae0c 100644
--- a/checkstyle_suppressions.xml
+++ b/checkstyle_suppressions.xml
@@ -22,5 +22,5 @@
         "https://checkstyle.org/dtds/suppressions_1_1.dtd">
 
 <suppressions>
-  <suppress checks="RegexpSinglelineJava" files="Clock\.java"/>
+  <suppress checks="RegexpSinglelineJava" files="Clock\.java|Semaphore\.java"/>
 </suppressions>
diff --git a/src/java/org/apache/cassandra/cache/AutoSavingCache.java b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
index d18baab..c7a015c 100644
--- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java
+++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.cache;
 
 import java.io.*;
 import java.util.*;
-import java.util.concurrent.Callable;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledFuture;
@@ -46,7 +45,6 @@ import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.compaction.CompactionInfo.Unit;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.*;
-import org.apache.cassandra.io.util.CorruptFileException;
 import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.JVMStabilityInspector;
@@ -162,26 +160,14 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
         final ListeningExecutorService es = MoreExecutors.listeningDecorator(Executors.newSingleThreadExecutor());
         final long start = nanoTime();
 
-        ListenableFuture<Integer> cacheLoad = es.submit(new Callable<Integer>()
-        {
-            @Override
-            public Integer call()
-            {
-                return loadSaved();
-            }
-        });
-        cacheLoad.addListener(new Runnable()
-        {
-            @Override
-            public void run()
-            {
-                if (size() > 0)
-                    logger.info("Completed loading ({} ms; {} keys) {} cache",
-                            TimeUnit.NANOSECONDS.toMillis(nanoTime() - start),
-                            CacheService.instance.keyCache.size(),
-                            cacheType);
-                es.shutdown();
-            }
+        ListenableFuture<Integer> cacheLoad = es.submit(this::loadSaved);
+        cacheLoad.addListener(() -> {
+            if (size() > 0)
+                logger.info("Completed loading ({} ms; {} keys) {} cache",
+                        TimeUnit.NANOSECONDS.toMillis(nanoTime() - start),
+                        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
index d666a36..6053b83 100644
--- a/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
+++ b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
@@ -26,10 +26,11 @@ 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.concurrent.SimpleCondition;
+
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 import static org.apache.cassandra.tracing.Tracing.isTracing;
@@ -140,7 +141,7 @@ public abstract class AbstractLocalAwareExecutorService implements LocalAwareExe
         }
     }
 
-    class FutureTask<T> extends SimpleCondition implements Future<T>, Runnable
+    class FutureTask<T> extends Condition.Async implements Future<T>, Runnable
     {
         private boolean failure;
         private Object result = this;
@@ -187,7 +188,7 @@ public abstract class AbstractLocalAwareExecutorService implements LocalAwareExe
 
         public boolean isDone()
         {
-            return isSignaled();
+            return isSignalled();
         }
 
         public T get() throws InterruptedException, ExecutionException
diff --git a/src/java/org/apache/cassandra/concurrent/JMXEnabledSingleThreadExecutor.java b/src/java/org/apache/cassandra/concurrent/JMXEnabledSingleThreadExecutor.java
index ed54b3e..1e61aa1 100644
--- a/src/java/org/apache/cassandra/concurrent/JMXEnabledSingleThreadExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/JMXEnabledSingleThreadExecutor.java
@@ -18,15 +18,16 @@
 
 package org.apache.cassandra.concurrent;
 
-import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
+
+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, TimeUnit.SECONDS, new LinkedBlockingQueue<>(), new SingleThreadFactory(threadPoolName), jmxPath);
+        super(1, Integer.MAX_VALUE, SECONDS, newBlockingQueue(), new SingleThreadFactory(threadPoolName), jmxPath);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java b/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
index bcf686f..5aadb48 100644
--- a/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
+++ b/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
@@ -22,9 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.annotations.VisibleForTesting;
 
-import io.netty.util.concurrent.FastThreadLocal;
 import io.netty.util.concurrent.FastThreadLocalThread;
-import org.apache.cassandra.utils.memory.BufferPool;
 
 /**
  * This class is an implementation of the <i>ThreadFactory</i> interface. This
diff --git a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
index 675e047..9085ee4 100644
--- a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
@@ -26,14 +26,16 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import org.apache.cassandra.utils.MBeanWrapper;
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.metrics.ThreadPoolMetrics;
-import org.apache.cassandra.utils.MBeanWrapper;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
 
+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
 {
@@ -55,7 +57,7 @@ public class SEPExecutor extends AbstractLocalAwareExecutorService implements SE
     private final AtomicLong completedTasks = new AtomicLong();
 
     volatile boolean shuttingDown = false;
-    final SimpleCondition shutdown = new SimpleCondition();
+    final Condition shutdown = newOneTimeCondition();
 
     // TODO: see if other queue implementations might improve throughput
     protected final ConcurrentLinkedQueue<FutureTask<?>> tasks = new ConcurrentLinkedQueue<>();
@@ -144,14 +146,14 @@ public class SEPExecutor extends AbstractLocalAwareExecutorService implements SE
                 // Work permits are negative when the pool is reducing in size.  Atomically
                 // adjust the number of work permits so there is no race of multiple SEPWorkers
                 // exiting.  On conflicting update, recheck.
-                result = TakeTaskPermitResult.RETURNED_WORK_PERMIT;
+                result = RETURNED_WORK_PERMIT;
                 updated = updateWorkPermits(current, workPermits + 1);
             }
             else
             {
                 if (taskPermits == 0)
-                    return TakeTaskPermitResult.NONE_AVAILABLE;
-                result = TakeTaskPermitResult.TOOK_PERMIT;
+                    return NONE_AVAILABLE;
+                result = TOOK_PERMIT;
                 updated = updateTaskPermits(current, taskPermits - 1);
             }
             if (permits.compareAndSet(current, updated))
@@ -234,7 +236,7 @@ public class SEPExecutor extends AbstractLocalAwareExecutorService implements SE
     {
         shutdown();
         List<Runnable> aborted = new ArrayList<>();
-        while (takeTaskPermit(false) == TakeTaskPermitResult.TOOK_PERMIT)
+        while (takeTaskPermit(false) == TOOK_PERMIT)
             aborted.add(tasks.poll());
         return aborted;
     }
@@ -246,7 +248,7 @@ public class SEPExecutor extends AbstractLocalAwareExecutorService implements SE
 
     public boolean isTerminated()
     {
-        return shuttingDown && shutdown.isSignaled();
+        return shuttingDown && shutdown.isSignalled();
     }
 
     public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException
diff --git a/src/java/org/apache/cassandra/concurrent/Stage.java b/src/java/org/apache/cassandra/concurrent/Stage.java
index e00da7b..a34c3d3 100644
--- a/src/java/org/apache/cassandra/concurrent/Stage.java
+++ b/src/java/org/apache/cassandra/concurrent/Stage.java
@@ -25,7 +25,6 @@ import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.RejectedExecutionHandler;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -45,7 +44,9 @@ import org.apache.cassandra.utils.ExecutorUtils;
 
 import org.apache.cassandra.utils.FBUtilities;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static java.util.stream.Collectors.toMap;
+import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
 public enum Stage
 {
@@ -182,8 +183,8 @@ public enum Stage
     {
         return new JMXEnabledThreadPoolExecutor(numThreads,
                                                 KEEP_ALIVE_SECONDS,
-                                                TimeUnit.SECONDS,
-                                                new LinkedBlockingQueue<>(),
+                                                SECONDS,
+                                                newBlockingQueue(),
                                                 new NamedThreadFactory(jmxName),
                                                 jmxType);
     }
diff --git a/src/java/org/apache/cassandra/cql3/CQL3Type.java b/src/java/org/apache/cassandra/cql3/CQL3Type.java
index 5059104..f23ff14 100644
--- a/src/java/org/apache/cassandra/cql3/CQL3Type.java
+++ b/src/java/org/apache/cassandra/cql3/CQL3Type.java
@@ -24,7 +24,6 @@ import java.util.List;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.db.marshal.CollectionType.Kind;
diff --git a/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java b/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
index 940f0a4..aab2046 100644
--- a/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
@@ -27,7 +27,6 @@ import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.CQL3Type.Tuple;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.CqlBuilder;
-import org.apache.cassandra.cql3.CqlBuilder.Appender;
 import org.apache.cassandra.db.marshal.AbstractType;
 
 import org.apache.commons.lang3.text.StrBuilder;
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFExecutorService.java b/src/java/org/apache/cassandra/cql3/functions/UDFExecutorService.java
index 5e08ad8..a6e3a92 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFExecutorService.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFExecutorService.java
@@ -17,12 +17,12 @@
  */
 package org.apache.cassandra.cql3.functions;
 
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.utils.FBUtilities;
+
+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
@@ -35,10 +35,10 @@ final class UDFExecutorService extends JMXEnabledThreadPoolExecutor
 
     UDFExecutorService(NamedThreadFactory threadFactory, String jmxPath)
     {
-        super(FBUtilities.getAvailableProcessors(),
+        super(getAvailableProcessors(),
               KEEPALIVE,
-              TimeUnit.MILLISECONDS,
-              new LinkedBlockingQueue<>(),
+              MILLISECONDS,
+              newBlockingQueue(),
               threadFactory,
               jmxPath);
     }
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
index 550905f..2b3c7aa 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@ -58,6 +58,7 @@ import org.apache.cassandra.service.ClientWarn;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 import static com.google.common.collect.Iterables.any;
 import static com.google.common.collect.Iterables.transform;
@@ -512,7 +513,7 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
         catch (InterruptedException e)
         {
             Thread.currentThread().interrupt();
-            throw new RuntimeException(e);
+            throw new UncheckedInterruptedException(e);
         }
         catch (ExecutionException e)
         {
@@ -538,7 +539,7 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
             catch (InterruptedException e1)
             {
                 Thread.currentThread().interrupt();
-                throw new RuntimeException(e);
+                throw new UncheckedInterruptedException(e1);
             }
             catch (ExecutionException e1)
             {
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/TypeCodec.java b/src/java/org/apache/cassandra/cql3/functions/types/TypeCodec.java
index a728a1c..2c15a25 100644
--- a/src/java/org/apache/cassandra/cql3/functions/types/TypeCodec.java
+++ b/src/java/org/apache/cassandra/cql3/functions/types/TypeCodec.java
@@ -36,8 +36,6 @@ import com.google.common.io.ByteStreams;
 import com.google.common.reflect.TypeToken;
 
 import org.apache.cassandra.transport.ProtocolVersion;
-import org.apache.cassandra.cql3.functions.types.DataType.CollectionType;
-import org.apache.cassandra.cql3.functions.types.DataType.Name;
 import org.apache.cassandra.cql3.functions.types.exceptions.InvalidTypeException;
 import org.apache.cassandra.cql3.functions.types.utils.Bytes;
 import org.apache.cassandra.utils.vint.VIntCoding;
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index e7b1577..9378d0a 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -21,9 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 import com.google.common.collect.HashMultiset;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Maps;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateFunctionStatement.java
index 18d8479..9c2bba9 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateFunctionStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateFunctionStatement.java
@@ -26,8 +26,6 @@ import com.google.common.collect.Lists;
 
 import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.FunctionResource;
-import org.apache.cassandra.auth.IResource;
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.CQLStatement;
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/DropIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/DropIndexStatement.java
index 2186470..24b372d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/DropIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/DropIndexStatement.java
@@ -22,7 +22,6 @@ import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QualifiedName;
-import org.apache.cassandra.schema.Diff;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.schema.KeyspaceMetadata.KeyspaceDiff;
 import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
diff --git a/src/java/org/apache/cassandra/db/BufferClustering.java b/src/java/org/apache/cassandra/db/BufferClustering.java
index a6dcd1b..e3592e1 100644
--- a/src/java/org/apache/cassandra/db/BufferClustering.java
+++ b/src/java/org/apache/cassandra/db/BufferClustering.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
 
 /**
diff --git a/src/java/org/apache/cassandra/db/BufferClusteringBound.java b/src/java/org/apache/cassandra/db/BufferClusteringBound.java
index 9b511fa..f95f3a9 100644
--- a/src/java/org/apache/cassandra/db/BufferClusteringBound.java
+++ b/src/java/org/apache/cassandra/db/BufferClusteringBound.java
@@ -22,7 +22,6 @@ import java.nio.ByteBuffer;
 
 import com.google.common.base.Preconditions;
 
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 
diff --git a/src/java/org/apache/cassandra/db/BufferClusteringBoundary.java b/src/java/org/apache/cassandra/db/BufferClusteringBoundary.java
index cad4d87..63ee9d4 100644
--- a/src/java/org/apache/cassandra/db/BufferClusteringBoundary.java
+++ b/src/java/org/apache/cassandra/db/BufferClusteringBoundary.java
@@ -22,7 +22,6 @@ import java.nio.ByteBuffer;
 
 import com.google.common.base.Preconditions;
 
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 497466a..224236b 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -92,14 +92,18 @@ import org.apache.cassandra.streaming.TableStreamManager;
 import org.apache.cassandra.utils.*;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.Refs;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 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.db.commitlog.CommitLogPosition.NONE;
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
 import static org.apache.cassandra.utils.Throwables.maybeFail;
 import static org.apache.cassandra.utils.Throwables.merge;
+import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownLatch;
 
 public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 {
@@ -953,7 +957,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      */
     private final class PostFlush implements Callable<CommitLogPosition>
     {
-        final CountDownLatch latch = new CountDownLatch(1);
+        final org.apache.cassandra.utils.concurrent.CountDownLatch latch = newCountDownLatch(1);
         final List<Memtable> memtables;
         volatile Throwable flushFailure = null;
 
@@ -972,10 +976,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             }
             catch (InterruptedException e)
             {
-                throw new IllegalStateException();
+                throw new UncheckedInterruptedException(e);
             }
 
-            CommitLogPosition commitLogUpperBound = CommitLogPosition.NONE;
+            CommitLogPosition commitLogUpperBound = NONE;
             // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
             if (flushFailure == null && !memtables.isEmpty())
             {
@@ -987,7 +991,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             metric.pendingFlushes.dec();
 
             if (flushFailure != null)
-                throw Throwables.propagate(flushFailure);
+                throw propagate(flushFailure);
 
             return commitLogUpperBound;
         }
@@ -1091,7 +1095,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 logger.trace("Flush task {}@{} signaling post flush task", hashCode(), name);
 
             // signal the post-flush we've done our work
-            postFlush.latch.countDown();
+            postFlush.latch.decrement();
 
             if (logger.isTraceEnabled())
                 logger.trace("Flush task task {}@{} finished", hashCode(), name);
diff --git a/src/java/org/apache/cassandra/db/DiskBoundaryManager.java b/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
index cc617da..48a40dd 100644
--- a/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
+++ b/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
@@ -21,7 +21,6 @@ package org.apache.cassandra.db;
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.List;
-import java.util.stream.Collectors;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 08acf6b..39cb746 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -72,6 +72,7 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 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 static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
@@ -613,7 +614,7 @@ public class Keyspace
                             }
                             catch (InterruptedException e)
                             {
-                                // Just continue
+                                throw new UncheckedInterruptedException(e);
                             }
                             continue;
                         }
diff --git a/src/java/org/apache/cassandra/db/MutableDeletionInfo.java b/src/java/org/apache/cassandra/db/MutableDeletionInfo.java
index bfe4d4c..a7879d2 100644
--- a/src/java/org/apache/cassandra/db/MutableDeletionInfo.java
+++ b/src/java/org/apache/cassandra/db/MutableDeletionInfo.java
@@ -24,7 +24,6 @@ import com.google.common.base.Objects;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.rows.*;
-import org.apache.cassandra.db.rows.EncodingStats;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 
diff --git a/src/java/org/apache/cassandra/db/Slice.java b/src/java/org/apache/cassandra/db/Slice.java
index 8956bd1..e2c787d 100644
--- a/src/java/org/apache/cassandra/db/Slice.java
+++ b/src/java/org/apache/cassandra/db/Slice.java
@@ -22,7 +22,6 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.ByteArrayAccessor;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
diff --git a/src/java/org/apache/cassandra/db/SnapshotCommand.java b/src/java/org/apache/cassandra/db/SnapshotCommand.java
index 484db2f..e909e50 100644
--- a/src/java/org/apache/cassandra/db/SnapshotCommand.java
+++ b/src/java/org/apache/cassandra/db/SnapshotCommand.java
@@ -22,8 +22,6 @@ import java.io.IOException;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.Message;
-import org.apache.cassandra.net.Verb;
 
 public class SnapshotCommand
 {
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index a418c6d..f7946b9 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -59,7 +59,6 @@ import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.RestorableMeter;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.*;
-import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.service.paxos.Commit;
 import org.apache.cassandra.service.paxos.PaxosState;
diff --git a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
index 8430541..856b27c 100644
--- a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
+++ b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db;
 
 import java.io.IOException;
 
-import org.apache.cassandra.db.marshal.ByteArrayAccessor;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.util.DataInputPlus;
diff --git a/src/java/org/apache/cassandra/db/aggregation/GroupingState.java b/src/java/org/apache/cassandra/db/aggregation/GroupingState.java
index 2e522c4..b77802c 100644
--- a/src/java/org/apache/cassandra/db/aggregation/GroupingState.java
+++ b/src/java/org/apache/cassandra/db/aggregation/GroupingState.java
@@ -23,7 +23,6 @@ import java.nio.ByteBuffer;
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.ClusteringComparator;
 import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.db.marshal.ByteArrayAccessor;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.ByteBufferUtil;
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
index cdf96cd..edba125 100755
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
@@ -24,6 +24,7 @@ import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.BooleanSupplier;
 
+import com.codahale.metrics.Timer.Context;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.*;
 import org.slf4j.Logger;
@@ -40,9 +41,11 @@ 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 static org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation;
+import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue;
 
 /**
  * Performs eager-creation of commit log segments in a background thread. All the
@@ -62,7 +65,7 @@ public abstract class AbstractCommitLogSegmentManager
      */
     private volatile CommitLogSegment availableSegment = null;
 
-    private final WaitQueue segmentPrepared = new WaitQueue();
+    private final WaitQueue segmentPrepared = newWaitQueue();
 
     /** Active segments, containing unflushed data. The tail of this queue is the one we allocate writes to */
     private final ConcurrentLinkedQueue<CommitLogSegment> activeSegments = new ConcurrentLinkedQueue<>();
@@ -89,7 +92,7 @@ public abstract class AbstractCommitLogSegmentManager
     protected final CommitLog commitLog;
     private volatile boolean shutdown;
     private final BooleanSupplier managerThreadWaitCondition = () -> (availableSegment == null && !atSegmentBufferLimit()) || shutdown;
-    private final WaitQueue managerThreadWaitQueue = new WaitQueue();
+    private final WaitQueue managerThreadWaitQueue = newWaitQueue();
 
     private volatile SimpleCachedBufferPool bufferPool;
 
@@ -265,7 +268,7 @@ public abstract class AbstractCommitLogSegmentManager
     {
         do
         {
-            WaitQueue.Signal prepared = segmentPrepared.register(commitLog.metrics.waitingOnSegmentAllocation.time());
+            WaitQueue.Signal prepared = segmentPrepared.register(commitLog.metrics.waitingOnSegmentAllocation.time(), Context::stop);
             if (availableSegment == null && allocatingFrom == currentAllocatingFrom)
                 prepared.awaitUninterruptibly();
             else
@@ -430,7 +433,7 @@ public abstract class AbstractCommitLogSegmentManager
         }
         catch (InterruptedException e)
         {
-            throw new RuntimeException(e);
+            throw new UncheckedInterruptedException(e);
         }
 
         for (CommitLogSegment segment : activeSegments)
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
index 24838a9..07c505b 100644
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
@@ -25,7 +25,6 @@ import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.codahale.metrics.Timer.Context;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation;
@@ -33,8 +32,10 @@ import org.apache.cassandra.utils.MonotonicClock;
 import org.apache.cassandra.utils.NoSpamLogger;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
+import static com.codahale.metrics.Timer.*;
 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.WaitQueue.newWaitQueue;
 
 public abstract class AbstractCommitLogService
 {
@@ -55,7 +56,7 @@ public abstract class AbstractCommitLogService
     protected final AtomicLong pending = new AtomicLong(0);
 
     // signal that writers can wait on to be notified of a completed sync
-    protected final WaitQueue syncComplete = new WaitQueue();
+    protected final WaitQueue syncComplete = newWaitQueue();
 
     final CommitLog commitLog;
     private final String name;
@@ -304,7 +305,7 @@ public abstract class AbstractCommitLogService
     {
         do
         {
-            WaitQueue.Signal signal = context != null ? syncComplete.register(context) : syncComplete.register();
+            WaitQueue.Signal signal = context != null ? syncComplete.register(context, Context::stop) : syncComplete.register();
             if (lastSyncedAt < syncTime)
                 signal.awaitUninterruptibly();
             else
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index 7670c5e..5245c0a 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -46,6 +46,7 @@ import org.apache.cassandra.security.EncryptionContext;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.MBeanWrapper;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 import static org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation;
 import static org.apache.cassandra.db.commitlog.CommitLogSegment.CommitLogSegmentFileComparator;
@@ -469,7 +470,7 @@ public class CommitLog implements CommitLogMBean
         }
         catch (InterruptedException e)
         {
-            throw new RuntimeException(e);
+            throw new UncheckedInterruptedException(e);
         }
         segmentManager.stopUnsafe(deleteSegments);
         CommitLogSegment.resetReplayLimit();
@@ -549,7 +550,7 @@ public class CommitLog implements CommitLogMBean
          */
         public boolean useEncryption()
         {
-            return encryptionContext.isEnabled();
+            return encryptionContext != null && encryptionContext.isEnabled();
         }
 
         /**
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
index b58a316..e1d0874 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
@@ -38,6 +38,8 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.CompressionParams;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.WrappedRunnable;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -193,7 +195,7 @@ public class CommitLogArchiver
         }
         catch (InterruptedException e)
         {
-            throw new AssertionError(e);
+            throw new UncheckedInterruptedException(e);
         }
         catch (ExecutionException e)
         {
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index 55461a3..4a68e8b 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -48,8 +48,10 @@ import org.apache.cassandra.utils.IntegerInterval;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
+import static com.codahale.metrics.Timer.*;
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
 import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt;
+import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue;
 
 /*
  * A single commit log file on disk. Manages creation of the file and writing mutations to disk,
@@ -111,7 +113,7 @@ public abstract class CommitLogSegment
     private int endOfBuffer;
 
     // a signal for writers to wait on to confirm the log message they provided has been written to disk
-    private final WaitQueue syncComplete = new WaitQueue();
+    private final WaitQueue syncComplete = newWaitQueue();
 
     // a map of Cf->dirty interval in this segment; if interval is not covered by the clean set, the log contains unflushed data
     private final NonBlockingHashMap<TableId, IntegerInterval> tableDirty = new NonBlockingHashMap<>(1024);
@@ -511,7 +513,6 @@ public abstract class CommitLogSegment
         while (lastSyncedOffset < position)
         {
             WaitQueue.Signal signal = syncComplete.register();
-            
             if (lastSyncedOffset < position)
                 signal.awaitUninterruptibly();
             else
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java
index b9bd744..0e051cf 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java
@@ -18,8 +18,6 @@
 
 package org.apache.cassandra.db.commitlog;
 
-import java.io.File;
-
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.io.util.FileUtils;
 
diff --git a/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java b/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java
index a13f615..f503658 100644
--- a/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java
@@ -27,7 +27,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSWriteError;
-import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.io.compress.ICompressor;
 import org.apache.cassandra.security.EncryptionUtils;
 import org.apache.cassandra.security.EncryptionContext;
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java b/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java
index 95fc7b8..4efc75a 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java
@@ -31,7 +31,6 @@ import com.google.common.base.Preconditions;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
-import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.index.Index;
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 2b62c4b..ace83d1 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -80,7 +80,6 @@ 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.Throwables;
 import org.apache.cassandra.utils.concurrent.Refs;
 
 import static java.util.Collections.singleton;
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java
index 129ee79..72c0f3a 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java
@@ -29,7 +29,6 @@ import com.google.common.collect.Iterables;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
-import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.index.Index;
diff --git a/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java b/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java
index 03d4111..e3d4c3d 100644
--- a/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java
+++ b/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java
@@ -30,7 +30,6 @@ import com.google.common.collect.Iterables;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
-import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.index.Index;
diff --git a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
index 1746d7c..1bb1701 100644
--- a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
+++ b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db.compaction;
 
 import java.util.*;
 import java.util.function.LongPredicate;
-import java.util.function.Predicate;
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
diff --git a/src/java/org/apache/cassandra/db/compaction/Upgrader.java b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
index e1406aa..0cb4500 100644
--- a/src/java/org/apache/cassandra/db/compaction/Upgrader.java
+++ b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.db.compaction;
 import java.io.File;
 import java.util.*;
 import java.util.function.LongPredicate;
-import java.util.function.Predicate;
 
 import com.google.common.base.Throwables;
 import com.google.common.collect.Sets;
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
index d363dcf..2251f6a 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.db.compaction.writers;
 
-import java.io.File;
 import java.util.Collection;
 import java.util.List;
 import java.util.Set;
diff --git a/src/java/org/apache/cassandra/db/lifecycle/ILifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/ILifecycleTransaction.java
index 3de0a35..c014e38 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/ILifecycleTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/ILifecycleTransaction.java
@@ -21,7 +21,6 @@ package org.apache.cassandra.db.lifecycle;
 import java.util.Collection;
 import java.util.Set;
 
-import org.apache.cassandra.io.sstable.SSTable;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.utils.concurrent.Transactional;
 
diff --git a/src/java/org/apache/cassandra/db/marshal/DurationType.java b/src/java/org/apache/cassandra/db/marshal/DurationType.java
index 134a6f8..2afbfc1 100644
--- a/src/java/org/apache/cassandra/db/marshal/DurationType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DurationType.java
@@ -26,7 +26,6 @@ import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.DurationSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.TypeSerializer;
-import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
diff --git a/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java b/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java
index f883ccd..8f1d677 100644
--- a/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java
+++ b/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java
@@ -23,7 +23,6 @@ import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.Constants;
 import org.apache.cassandra.cql3.Duration;
 import org.apache.cassandra.cql3.Term;
-import org.apache.cassandra.cql3.statements.RequestValidations;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.SimpleDateSerializer;
 import org.apache.cassandra.serializers.TypeSerializer;
diff --git a/src/java/org/apache/cassandra/db/marshal/TimeType.java b/src/java/org/apache/cassandra/db/marshal/TimeType.java
index be20ba7..fd8fac4 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimeType.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
 import java.time.LocalTime;
-import java.time.ZoneId;
 import java.time.ZoneOffset;
 
 import org.apache.cassandra.cql3.Constants;
diff --git a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
index 6cf1375..5b2443b 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
@@ -21,7 +21,6 @@ import java.nio.ByteBuffer;
 import java.util.UUID;
 
 import org.apache.cassandra.cql3.CQL3Type;
-import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.Constants;
 import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
diff --git a/src/java/org/apache/cassandra/db/marshal/TimestampType.java b/src/java/org/apache/cassandra/db/marshal/TimestampType.java
index 0dac6b0..ccf1da3 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimestampType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimestampType.java
@@ -23,7 +23,6 @@ import java.util.Date;
 import org.apache.cassandra.cql3.Constants;
 import org.apache.cassandra.cql3.Duration;
 import org.apache.cassandra.cql3.Term;
-import org.apache.cassandra.cql3.statements.RequestValidations;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java b/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
index 0f8555f..d08c62f 100644
--- a/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
+++ b/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
@@ -25,7 +25,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
@@ -41,6 +40,7 @@ import org.apache.cassandra.utils.NoSpamLogger;
 import static java.lang.System.getProperty;
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
 /**
  * A task for monitoring in progress operations, currently only read queries, and aborting them if they time out.
@@ -207,7 +207,7 @@ class MonitoringTask
         OperationsQueue(int maxOperations)
         {
             this.maxOperations = maxOperations;
-            this.queue = maxOperations > 0 ? new ArrayBlockingQueue<>(maxOperations) : new LinkedBlockingQueue<>();
+            this.queue = maxOperations > 0 ? newBlockingQueue(maxOperations) : newBlockingQueue();
             this.numDroppedOperations = new AtomicLong();
         }
 
diff --git a/src/java/org/apache/cassandra/db/partitions/Partition.java b/src/java/org/apache/cassandra/db/partitions/Partition.java
index b6297a1..a9a9653 100644
--- a/src/java/org/apache/cassandra/db/partitions/Partition.java
+++ b/src/java/org/apache/cassandra/db/partitions/Partition.java
@@ -22,7 +22,6 @@ import java.util.NavigableSet;
 import javax.annotation.Nullable;
 
 import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.db.Slices;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
diff --git a/src/java/org/apache/cassandra/db/partitions/PurgeFunction.java b/src/java/org/apache/cassandra/db/partitions/PurgeFunction.java
index d9e9036..09f3ae3 100644
--- a/src/java/org/apache/cassandra/db/partitions/PurgeFunction.java
+++ b/src/java/org/apache/cassandra/db/partitions/PurgeFunction.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.db.partitions;
 
 import java.util.function.LongPredicate;
-import java.util.function.Predicate;
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java b/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java
index be32847..a2c3a4d 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java
@@ -17,8 +17,8 @@
  */
 package org.apache.cassandra.db.rows;
 
-import org.apache.cassandra.db.ClusteringBoundOrBoundary;
 import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.db.ClusteringBoundOrBoundary;
 
 public abstract class AbstractRangeTombstoneMarker<B extends ClusteringBoundOrBoundary<?>> implements RangeTombstoneMarker
 {
diff --git a/src/java/org/apache/cassandra/db/rows/ThrottledUnfilteredIterator.java b/src/java/org/apache/cassandra/db/rows/ThrottledUnfilteredIterator.java
index 40be716..cbbac64 100644
--- a/src/java/org/apache/cassandra/db/rows/ThrottledUnfilteredIterator.java
+++ b/src/java/org/apache/cassandra/db/rows/ThrottledUnfilteredIterator.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.rows;
 
-import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Iterator;
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java
index ff9e6f7..f6af23c 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java
@@ -24,7 +24,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataInputPlus;
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java
index b2b2ce5..3113778 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java
@@ -28,7 +28,7 @@ import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
 import org.apache.cassandra.io.sstable.SSTable;
-import org.apache.cassandra.streaming.StreamReceiveTask;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/db/view/ViewBuilderTask.java b/src/java/org/apache/cassandra/db/view/ViewBuilderTask.java
index 3c9c037..70f7bb7 100644
--- a/src/java/org/apache/cassandra/db/view/ViewBuilderTask.java
+++ b/src/java/org/apache/cassandra/db/view/ViewBuilderTask.java
@@ -29,7 +29,6 @@ import java.util.concurrent.atomic.AtomicLong;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.PeekingIterator;
 import com.google.common.util.concurrent.Futures;
diff --git a/src/java/org/apache/cassandra/db/view/ViewUtils.java b/src/java/org/apache/cassandra/db/view/ViewUtils.java
index b5aa063..c248ddc 100644
--- a/src/java/org/apache/cassandra/db/view/ViewUtils.java
+++ b/src/java/org/apache/cassandra/db/view/ViewUtils.java
@@ -29,7 +29,6 @@ import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.locator.EndpointsForToken;
 import org.apache.cassandra.locator.NetworkTopologyStrategy;
 import org.apache.cassandra.locator.Replica;
-import org.apache.cassandra.utils.FBUtilities;
 
 public final class ViewUtils
 {
diff --git a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
index a6314dc..3a5db52 100644
--- a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
@@ -37,7 +37,6 @@ import java.math.BigInteger;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
diff --git a/src/java/org/apache/cassandra/dht/Datacenters.java b/src/java/org/apache/cassandra/dht/Datacenters.java
index 9695a09..b1d96eb 100644
--- a/src/java/org/apache/cassandra/dht/Datacenters.java
+++ b/src/java/org/apache/cassandra/dht/Datacenters.java
@@ -25,7 +25,6 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.FBUtilities;
 
 public class Datacenters
 {
diff --git a/src/java/org/apache/cassandra/dht/StreamStateStore.java b/src/java/org/apache/cassandra/dht/StreamStateStore.java
index e62bc04..6d79e9a 100644
--- a/src/java/org/apache/cassandra/dht/StreamStateStore.java
+++ b/src/java/org/apache/cassandra/dht/StreamStateStore.java
@@ -30,7 +30,6 @@ import org.apache.cassandra.streaming.StreamEvent;
 import org.apache.cassandra.streaming.StreamEventHandler;
 import org.apache.cassandra.streaming.StreamRequest;
 import org.apache.cassandra.streaming.StreamState;
-import org.apache.cassandra.utils.Pair;
 
 /**
  * Store and update available ranges (data already received) to system keyspace.
diff --git a/src/java/org/apache/cassandra/diag/DiagnosticEventService.java b/src/java/org/apache/cassandra/diag/DiagnosticEventService.java
index 34a4de7..cab9596 100644
--- a/src/java/org/apache/cassandra/diag/DiagnosticEventService.java
+++ b/src/java/org/apache/cassandra/diag/DiagnosticEventService.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.diag;
 
 import java.io.Serializable;
-import java.lang.management.ManagementFactory;
 import java.util.Collection;
 import java.util.Map;
 import java.util.Objects;
@@ -27,8 +26,6 @@ import java.util.SortedMap;
 import java.util.function.Consumer;
 
 import javax.annotation.Nullable;
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
 
 import com.google.common.collect.ImmutableCollection;
 import com.google.common.collect.ImmutableMap;
diff --git a/src/java/org/apache/cassandra/fql/FullQueryLogger.java b/src/java/org/apache/cassandra/fql/FullQueryLogger.java
index 9725cea..49e326e 100644
--- a/src/java/org/apache/cassandra/fql/FullQueryLogger.java
+++ b/src/java/org/apache/cassandra/fql/FullQueryLogger.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.fql;
 import java.io.File;
 import java.nio.ByteBuffer;
 import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
@@ -49,6 +48,7 @@ import org.apache.cassandra.transport.Message;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.binlog.BinLog;
 import org.apache.cassandra.utils.binlog.BinLogOptions;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 import org.apache.cassandra.utils.concurrent.WeightedQueue;
 import org.github.jamm.MemoryLayoutSpecification;
 
@@ -161,7 +161,7 @@ public class FullQueryLogger implements QueryEvents.Listener
         }
         catch (InterruptedException e)
         {
-            throw new RuntimeException(e);
+            throw new UncheckedInterruptedException(e);
         }
         finally
         {
diff --git a/src/java/org/apache/cassandra/gms/FailureDetector.java b/src/java/org/apache/cassandra/gms/FailureDetector.java
index 7cd14a3..3ba96af 100644
--- a/src/java/org/apache/cassandra/gms/FailureDetector.java
+++ b/src/java/org/apache/cassandra/gms/FailureDetector.java
@@ -27,7 +27,6 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
-import javax.management.openmbean.CompositeData;
 import javax.management.openmbean.*;
 import org.apache.cassandra.locator.Replica;
 import org.slf4j.Logger;
diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java
index dc1e3a1..7dd73c7 100644
--- a/src/java/org/apache/cassandra/gms/Gossiper.java
+++ b/src/java/org/apache/cassandra/gms/Gossiper.java
@@ -71,6 +71,7 @@ import org.apache.cassandra.utils.MBeanWrapper;
 import org.apache.cassandra.utils.NoSpamLogger;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.RecomputingSupplier;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 import static org.apache.cassandra.config.CassandraRelevantProperties.GOSSIPER_QUARANTINE_DELAY;
 import static org.apache.cassandra.net.NoPayload.noPayload;
@@ -531,7 +532,11 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         {
             task.get();
         }
-        catch (InterruptedException | ExecutionException e)
+        catch (InterruptedException e)
+        {
+            throw new UncheckedInterruptedException(e);
+        }
+        catch (ExecutionException e)
         {
             throw new AssertionError(e);
         }
@@ -1850,9 +1855,9 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
                 }
             }
         }
-        catch (InterruptedException wtf)
+        catch (InterruptedException e)
         {
-            throw new RuntimeException(wtf);
+            throw new UncheckedInterruptedException(e);
         }
 
         return ImmutableMap.copyOf(endpointShadowStateMap);
diff --git a/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java b/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
index e6e8b38..a717027 100644
--- a/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
+++ b/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
@@ -26,7 +26,6 @@ import com.google.common.base.Preconditions;
 
 import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.io.util.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Throwables;
 import org.apache.cassandra.utils.NativeLibrary;
 
diff --git a/src/java/org/apache/cassandra/hints/HintsBufferPool.java b/src/java/org/apache/cassandra/hints/HintsBufferPool.java
index 7f66efd..8d1db8d 100644
--- a/src/java/org/apache/cassandra/hints/HintsBufferPool.java
+++ b/src/java/org/apache/cassandra/hints/HintsBufferPool.java
@@ -20,10 +20,12 @@ package org.apache.cassandra.hints;
 import java.io.Closeable;
 import java.util.UUID;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
 
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
+
+import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
 /**
  * A primitive pool of {@link HintsBuffer} buffers. Under normal conditions should only hold two buffers - the currently
@@ -45,7 +47,7 @@ final class HintsBufferPool implements Closeable
 
     HintsBufferPool(int bufferSize, FlushCallback flushCallback)
     {
-        reserveBuffers = new LinkedBlockingQueue<>();
+        reserveBuffers = newBlockingQueue();
         this.bufferSize = bufferSize;
         this.flushCallback = flushCallback;
     }
@@ -117,7 +119,7 @@ final class HintsBufferPool implements Closeable
             }
             catch (InterruptedException e)
             {
-                throw new RuntimeException(e);
+                throw new UncheckedInterruptedException(e);
             }
         }
         currentBuffer = buffer == null ? createBuffer() : buffer;
diff --git a/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java b/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
index b5eb0b1..2a5239c 100644
--- a/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
+++ b/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
@@ -37,6 +37,11 @@ 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 static java.lang.Thread.MIN_PRIORITY;
+import static java.util.concurrent.TimeUnit.MINUTES;
+import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
 /**
  * A multi-threaded (by default) executor for dispatching hints.
@@ -60,9 +65,9 @@ final class HintsDispatchExecutor
         this.isAlive = isAlive;
 
         scheduledDispatches = new ConcurrentHashMap<>();
-        executor = new JMXEnabledThreadPoolExecutor(maxThreads, 1, TimeUnit.MINUTES,
-                                                    new LinkedBlockingQueue<>(),
-                                                    new NamedThreadFactory("HintsDispatcher", Thread.MIN_PRIORITY),
+        executor = new JMXEnabledThreadPoolExecutor(maxThreads, 1, MINUTES,
+                                                    newBlockingQueue(),
+                                                    new NamedThreadFactory("HintsDispatcher", MIN_PRIORITY),
                                                     "internal");
     }
 
@@ -79,7 +84,7 @@ final class HintsDispatchExecutor
         }
         catch (InterruptedException e)
         {
-            throw new AssertionError(e);
+            throw new UncheckedInterruptedException(e);
         }
     }
 
@@ -120,7 +125,11 @@ final class HintsDispatchExecutor
             if (future != null)
                 future.get();
         }
-        catch (ExecutionException | InterruptedException e)
+        catch (InterruptedException e)
+        {
+            throw new UncheckedInterruptedException(e);
+        }
+        catch (ExecutionException e)
         {
             throw new RuntimeException(e);
         }
@@ -167,7 +176,7 @@ final class HintsDispatchExecutor
             }
             catch (InterruptedException e)
             {
-                throw new RuntimeException(e);
+                throw new UncheckedInterruptedException(e);
             }
 
             hostId = hostIdSupplier.get();
diff --git a/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java b/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java
index ca38c0c..0dfc6e1 100644
--- a/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java
+++ b/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java
@@ -19,12 +19,9 @@ package org.apache.cassandra.hints;
 
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.schema.Schema;
 
-import static org.apache.cassandra.utils.FBUtilities.getBroadcastAddressAndPort;
-
 /**
  * A simple dispatch trigger that's being run every 10 seconds.
  *
diff --git a/src/java/org/apache/cassandra/hints/HintsDispatcher.java b/src/java/org/apache/cassandra/hints/HintsDispatcher.java
index 743b275..4fe1ae3 100644
--- a/src/java/org/apache/cassandra/hints/HintsDispatcher.java
+++ b/src/java/org/apache/cassandra/hints/HintsDispatcher.java
@@ -24,6 +24,7 @@ import java.util.function.BooleanSupplier;
 import java.util.function.Function;
 
 import com.google.common.util.concurrent.RateLimiter;
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -33,10 +34,13 @@ import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.HintsServiceMetrics;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
 
+
+import static org.apache.cassandra.hints.HintsDispatcher.Callback.Outcome.*;
+import static org.apache.cassandra.metrics.HintsServiceMetrics.updateDelayMetrics;
 import static org.apache.cassandra.net.Verb.HINT_REQ;
 import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
 
 /**
  * Dispatches a single hints file to a specified node in a batched manner.
@@ -205,12 +209,12 @@ final class HintsDispatcher implements AutoCloseable
         return callback;
     }
 
-    private static final class Callback implements RequestCallback
+    static final class Callback implements RequestCallback
     {
         enum Outcome { SUCCESS, TIMEOUT, FAILURE, INTERRUPTED }
 
         private final long start = approxTime.now();
-        private final SimpleCondition condition = new SimpleCondition();
+        private final Condition condition = newOneTimeCondition();
         private volatile Outcome outcome;
         private final long hintCreationNanoTime;
 
@@ -229,10 +233,10 @@ final class HintsDispatcher implements AutoCloseable
             catch (InterruptedException e)
             {
                 logger.warn("Hint dispatch was interrupted", e);
-                return Outcome.INTERRUPTED;
+                return INTERRUPTED;
             }
 
-            return timedOut ? Outcome.TIMEOUT : outcome;
+            return timedOut ? TIMEOUT : outcome;
         }
 
         @Override
@@ -244,15 +248,15 @@ final class HintsDispatcher implements AutoCloseable
         @Override
         public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason)
         {
-            outcome = Outcome.FAILURE;
+            outcome = FAILURE;
             condition.signalAll();
         }
 
         @Override
         public void onResponse(Message msg)
         {
-            HintsServiceMetrics.updateDelayMetrics(msg.from(), approxTime.now() - this.hintCreationNanoTime);
-            outcome = Outcome.SUCCESS;
+            updateDelayMetrics(msg.from(), approxTime.now() - this.hintCreationNanoTime);
+            outcome = SUCCESS;
             condition.signalAll();
         }
     }
diff --git a/src/java/org/apache/cassandra/hints/HintsService.java b/src/java/org/apache/cassandra/hints/HintsService.java
index 75960de..a399a2f 100644
--- a/src/java/org/apache/cassandra/hints/HintsService.java
+++ b/src/java/org/apache/cassandra/hints/HintsService.java
@@ -48,6 +48,7 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.MBeanWrapper;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 import static com.google.common.collect.Iterables.transform;
 
@@ -339,7 +340,11 @@ public final class HintsService implements HintsServiceMBean
             flushFuture.get();
             closeFuture.get();
         }
-        catch (InterruptedException | ExecutionException e)
+        catch (InterruptedException e)
+        {
+            throw new UncheckedInterruptedException(e);
+        }
+        catch (ExecutionException e)
         {
             throw new RuntimeException(e);
         }
@@ -376,7 +381,11 @@ public final class HintsService implements HintsServiceMBean
             flushFuture.get();
             closeFuture.get();
         }
-        catch (InterruptedException | ExecutionException e)
+        catch (InterruptedException e)
+        {
+            throw new UncheckedInterruptedException(e);
+        }
+        catch (ExecutionException e)
         {
             throw new RuntimeException(e);
         }
diff --git a/src/java/org/apache/cassandra/hints/HintsWriteExecutor.java b/src/java/org/apache/cassandra/hints/HintsWriteExecutor.java
index 51a5362..26e3103 100644
--- a/src/java/org/apache/cassandra/hints/HintsWriteExecutor.java
+++ b/src/java/org/apache/cassandra/hints/HintsWriteExecutor.java
@@ -30,6 +30,7 @@ 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;
 
 /**
  * A single threaded executor that exclusively writes all the hints and otherwise manipulate the writers.
@@ -102,7 +103,11 @@ final class HintsWriteExecutor
         {
             executor.submit(new FsyncWritersTask(stores)).get();
         }
-        catch (InterruptedException | ExecutionException e)
+        catch (InterruptedException e)
+        {
+            throw new UncheckedInterruptedException(e);
+        }
+        catch (ExecutionException e)
         {
             throw new RuntimeException(e);
         }
diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index 79d4e5e..a3fd881 100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@ -48,7 +48,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.*;
@@ -79,8 +78,11 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.concurrent.Refs;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.concurrent.Stage.KEEP_ALIVE_SECONDS;
 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,9 +164,9 @@ public class SecondaryIndexManager implements IndexRegistry, INotificationConsum
     // 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,
-                                     Stage.KEEP_ALIVE_SECONDS,
-                                     TimeUnit.SECONDS,
-                                     new LinkedBlockingQueue<>(),
+                                     KEEP_ALIVE_SECONDS,
+                                     SECONDS,
+                                     newBlockingQueue(),
                                      new NamedThreadFactory("SecondaryIndexManagement"),
                                      "internal"));
 
diff --git a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
index 815b881..457dee1 100644
--- a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
@@ -24,7 +24,6 @@ import java.util.List;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ClusteringIndexNamesFilter;
-import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter;
 import org.apache.cassandra.db.filter.DataLimits;
 import org.apache.cassandra.db.filter.RowFilter;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
@@ -34,7 +33,6 @@ import org.apache.cassandra.index.internal.CassandraIndex;
 import org.apache.cassandra.index.internal.CassandraIndexSearcher;
 import org.apache.cassandra.index.internal.IndexEntry;
 import org.apache.cassandra.utils.btree.BTreeSet;
-import org.apache.cassandra.utils.concurrent.OpOrder;
 
 
 public class CompositesSearcher extends CassandraIndexSearcher
diff --git a/src/java/org/apache/cassandra/index/sasi/TermIterator.java b/src/java/org/apache/cassandra/index/sasi/TermIterator.java
index 85f81b0..3473234 100644
--- a/src/java/org/apache/cassandra/index/sasi/TermIterator.java
+++ b/src/java/org/apache/cassandra/index/sasi/TermIterator.java
@@ -26,7 +26,6 @@ import java.util.concurrent.atomic.AtomicLong;
 import io.netty.util.concurrent.FastThreadLocal;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
 import org.apache.cassandra.index.sasi.disk.Token;
 import org.apache.cassandra.index.sasi.plan.Expression;
 import org.apache.cassandra.index.sasi.utils.RangeUnionIterator;
@@ -34,11 +33,16 @@ import org.apache.cassandra.index.sasi.utils.RangeIterator;
 import org.apache.cassandra.io.util.FileUtils;
 
 import com.google.common.util.concurrent.MoreExecutors;
-import com.google.common.util.concurrent.Uninterruptibles;
 
+import org.apache.cassandra.utils.concurrent.CountDownLatch;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static java.lang.String.format;
+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;
+
 public class TermIterator extends RangeIterator<Long, Token>
 {
     private static final Logger logger = LoggerFactory.getLogger(TermIterator.class);
@@ -99,14 +103,14 @@ public class TermIterator extends RangeIterator<Long, Token>
 
         try
         {
-            final CountDownLatch latch = new CountDownLatch(perSSTableIndexes.size());
+            final CountDownLatch latch = newCountDownLatch(perSSTableIndexes.size());
             final ExecutorService searchExecutor = SEARCH_EXECUTOR.get();
 
             for (final SSTableIndex index : perSSTableIndexes)
             {
-                if (e.getOp() == Expression.Op.PREFIX &&
-                    index.mode() == OnDiskIndexBuilder.Mode.CONTAINS && !index.hasMarkedPartials())
-                    throw new UnsupportedOperationException(String.format("The index %s has not yet been upgraded " +
+                if (e.getOp() == PREFIX &&
+                    index.mode() == CONTAINS && !index.hasMarkedPartials())
+                    throw new UnsupportedOperationException(format("The index %s has not yet been upgraded " +
                                                                           "to support prefix queries in CONTAINS mode. " +
                                                                           "Wait for compaction or rebuild the index.",
                                                                           index.getPath()));
@@ -114,7 +118,7 @@ public class TermIterator extends RangeIterator<Long, Token>
 
                 if (!index.reference())
                 {
-                    latch.countDown();
+                    latch.decrement();
                     continue;
                 }
 
@@ -142,16 +146,16 @@ public class TermIterator extends RangeIterator<Long, Token>
                         releaseIndex(referencedIndexes, index);
 
                         if (logger.isDebugEnabled())
-                            logger.debug(String.format("Failed search an index %s, skipping.", index.getPath()), e1);
+                            logger.debug(format("Failed search an index %s, skipping.", index.getPath()), e1);
                     }
                     finally
                     {
-                        latch.countDown();
+                        latch.decrement();
                     }
                 });
             }
 
-            Uninterruptibles.awaitUninterruptibly(latch);
+            latch.awaitUninterruptibly();
 
             // checkpoint right away after all indexes complete search because we might have crossed the quota
             e.checkpoint();
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 1f01a9c..e4ac3f3 100644
--- a/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
+++ b/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.index.sasi.disk;
 
 import java.io.File;
 import java.nio.ByteBuffer;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
@@ -43,16 +42,19 @@ import org.apache.cassandra.io.sstable.format.SSTableFlushObserver;
 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 com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.Uninterruptibles;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 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
 {
@@ -64,14 +66,14 @@ public class PerSSTableIndexWriter implements SSTableFlushObserver
 
     static
     {
-        INDEX_FLUSHER_GENERAL = new JMXEnabledThreadPoolExecutor(POOL_SIZE, POOL_SIZE, 1, TimeUnit.MINUTES,
-                                                                 new LinkedBlockingQueue<>(),
+        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, TimeUnit.MINUTES,
-                                                                  new LinkedBlockingQueue<>(),
+        INDEX_FLUSHER_MEMTABLE = new JMXEnabledThreadPoolExecutor(POOL_SIZE, POOL_SIZE, 1, MINUTES,
+                                                                  newBlockingQueue(),
                                                                   new NamedThreadFactory("SASI-Memtable"),
                                                                   "internal");
         INDEX_FLUSHER_MEMTABLE.allowCoreThreadTimeOut(true);
@@ -141,11 +143,11 @@ public class PerSSTableIndexWriter implements SSTableFlushObserver
 
         try
         {
-            CountDownLatch latch = new CountDownLatch(indexes.size());
+            CountDownLatch latch = newCountDownLatch(indexes.size());
             for (Index index : indexes.values())
                 index.complete(latch);
 
-            Uninterruptibles.awaitUninterruptibly(latch);
+            latch.awaitUninterruptibly();
         }
         finally
         {
@@ -339,7 +341,7 @@ public class PerSSTableIndexWriter implements SSTableFlushObserver
                         FileUtils.delete(outputFile + "_" + segment);
                     }
 
-                    latch.countDown();
+                    latch.decrement();
                 }
             });
         }
diff --git a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
index 45cf7af..e472ca8 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
@@ -26,12 +26,10 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.EOFException;
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.Collection;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.SortedSet;
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index 7ac2ebc..aef054e 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -36,6 +36,8 @@ 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;
+
 
 /**
  * A SSTable writer that doesn't assume rows are in sorted order.
@@ -165,7 +167,7 @@ class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
             }
             catch (InterruptedException e)
             {
-                throw new RuntimeException(e);
+                throw new UncheckedInterruptedException(e);
             }
         }
     }
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 45297ef..a946e4f 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -69,12 +69,10 @@ import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.*;
-import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.utils.concurrent.Ref;
-import org.apache.cassandra.utils.concurrent.SelfRefCounted;
-import org.apache.cassandra.utils.BloomFilterSerializer;
+import org.apache.cassandra.utils.concurrent.*;
 
 import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR;
+import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
 /**
  * An SSTableReader can be constructed in a number of places, but typically is either
@@ -525,7 +523,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
     public static Collection<SSTableReader> openAll(Set<Map.Entry<Descriptor, Set<Component>>> entries,
                                                     final TableMetadataRef metadata)
     {
-        final Collection<SSTableReader> sstables = new LinkedBlockingQueue<>();
+        final Collection<SSTableReader> sstables = newBlockingQueue();
 
         ExecutorService executor = DebuggableThreadPoolExecutor.createWithFixedPoolSize("SSTableBatchOpen", FBUtilities.getAvailableProcessors());
         for (final Map.Entry<Descriptor, Set<Component>> entry : entries)
@@ -564,7 +562,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         }
         catch (InterruptedException e)
         {
-            throw new AssertionError(e);
+            throw new UncheckedInterruptedException(e);
         }
 
         return sstables;
diff --git a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java
index 7d1e91d..516508e 100644
--- a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java
+++ b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java
@@ -29,7 +29,6 @@ import com.google.common.base.Preconditions;
 import net.nicoulaj.compilecommand.annotations.DontInline;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.utils.FastByteOperations;
-import org.apache.cassandra.utils.memory.MemoryUtil;
 
 /**
  * An implementation of the DataOutputStreamPlus interface using a ByteBuffer to stage writes
diff --git a/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java
index 3d83212..a90477f 100644
--- a/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java
+++ b/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java
@@ -25,8 +25,6 @@ import java.nio.channels.WritableByteChannel;
 
 import org.apache.cassandra.utils.memory.MemoryUtil;
 
-import com.google.common.base.Function;
-
 /**
  * Base class for DataOutput implementations that does not have an optimized implementations of Plus methods
  * and does no buffering.
diff --git a/src/java/org/apache/cassandra/locator/Endpoints.java b/src/java/org/apache/cassandra/locator/Endpoints.java
index c1a9282..32e972a 100644
--- a/src/java/org/apache/cassandra/locator/Endpoints.java
+++ b/src/java/org/apache/cassandra/locator/Endpoints.java
@@ -28,8 +28,6 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 
-import com.google.common.collect.Lists;
-
 /**
  * A collection of Endpoints for a given ring position.  This will typically reside in a ReplicaLayout,
  * representing some subset of the endpoints for the Token or Range
diff --git a/src/java/org/apache/cassandra/locator/EndpointsByReplica.java b/src/java/org/apache/cassandra/locator/EndpointsByReplica.java
index 72d8751..9590842 100644
--- a/src/java/org/apache/cassandra/locator/EndpointsByReplica.java
+++ b/src/java/org/apache/cassandra/locator/EndpointsByReplica.java
@@ -23,8 +23,6 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict;
 
-import java.util.Collections;
-import java.util.HashMap;
 import java.util.Map;
 
 public class EndpointsByReplica extends ReplicaMultimap<Replica, EndpointsForRange>
diff --git a/src/java/org/apache/cassandra/locator/InOurDcTester.java b/src/java/org/apache/cassandra/locator/InOurDcTester.java
index 514c7ef..181cace 100644
--- a/src/java/org/apache/cassandra/locator/InOurDcTester.java
+++ b/src/java/org/apache/cassandra/locator/InOurDcTester.java
@@ -19,7 +19,7 @@
 package org.apache.cassandra.locator;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.utils.FBUtilities;
+
 import java.util.function.Predicate;
 
 public class InOurDcTester
diff --git a/src/java/org/apache/cassandra/locator/RangesByEndpoint.java b/src/java/org/apache/cassandra/locator/RangesByEndpoint.java
index 1a71141..cbf5a63 100644
--- a/src/java/org/apache/cassandra/locator/RangesByEndpoint.java
+++ b/src/java/org/apache/cassandra/locator/RangesByEndpoint.java
@@ -22,8 +22,6 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 
-import java.util.Collections;
-import java.util.HashMap;
 import java.util.Map;
 
 public class RangesByEndpoint extends ReplicaMultimap<InetAddressAndPort, RangesAtEndpoint>
diff --git a/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
index dea8c76..b950ec3 100644
--- a/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
+++ b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
@@ -30,8 +30,6 @@ import org.apache.cassandra.net.OutboundConnectionSettings;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.cassandra.net.ConnectionType.SMALL_MESSAGES;
-
 /**
  * Sidekick helper for snitches that want to reconnect from one IP addr for a node to another.
  * Typically, this is for situations like EC2 where a node will have a public address and a private address,
diff --git a/src/java/org/apache/cassandra/locator/ReplicaPlans.java b/src/java/org/apache/cassandra/locator/ReplicaPlans.java
index 67b89e5..85979d3 100644
--- a/src/java/org/apache/cassandra/locator/ReplicaPlans.java
+++ b/src/java/org/apache/cassandra/locator/ReplicaPlans.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.locator;
 
 import com.carrotsearch.hppc.ObjectIntHashMap;
-import com.carrotsearch.hppc.cursors.ObjectIntCursor;
 import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ArrayListMultimap;
diff --git a/src/java/org/apache/cassandra/locator/TokenMetadata.java b/src/java/org/apache/cassandra/locator/TokenMetadata.java
index f811a4f..fb9d43b 100644
--- a/src/java/org/apache/cassandra/locator/TokenMetadata.java
+++ b/src/java/org/apache/cassandra/locator/TokenMetadata.java
@@ -30,6 +30,7 @@ import javax.annotation.concurrent.GuardedBy;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.*;
+
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
index 46e5940..7d0b969 100644
--- a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
@@ -26,10 +26,8 @@ import com.codahale.metrics.Meter;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.compaction.ActiveCompactions;
 import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableMetadata;
 
diff --git a/src/java/org/apache/cassandra/net/AsyncChannelPromise.java b/src/java/org/apache/cassandra/net/AsyncChannelPromise.java
index d2c9d0b..c652b8c 100644
--- a/src/java/org/apache/cassandra/net/AsyncChannelPromise.java
+++ b/src/java/org/apache/cassandra/net/AsyncChannelPromise.java
@@ -24,6 +24,7 @@ import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelPromise;
 import io.netty.util.concurrent.Future;
 import io.netty.util.concurrent.GenericFutureListener;
+import org.apache.cassandra.utils.concurrent.AsyncPromise;
 
 /**
  * See {@link AsyncPromise} and {@link io.netty.channel.ChannelPromise}
@@ -92,7 +93,7 @@ public class AsyncChannelPromise extends AsyncPromise<Void> implements ChannelPr
         return setSuccess(null);
     }
 
-    public ChannelPromise setSuccess(Void v)
+    public AsyncChannelPromise setSuccess(Void v)
     {
         super.setSuccess(v);
         return this;
@@ -103,58 +104,56 @@ public class AsyncChannelPromise extends AsyncPromise<Void> implements ChannelPr
         return trySuccess(null);
     }
 
-    public ChannelPromise setFailure(Throwable throwable)
+    public AsyncChannelPromise setFailure(Throwable throwable)
     {
         super.setFailure(throwable);
         return this;
     }
 
-    public ChannelPromise sync() throws InterruptedException
+    public AsyncChannelPromise sync() throws InterruptedException
     {
         super.sync();
         return this;
     }
 
-    public ChannelPromise syncUninterruptibly()
+    public AsyncChannelPromise syncUninterruptibly()
     {
         super.syncUninterruptibly();
         return this;
     }
 
-    public ChannelPromise await() throws InterruptedException
+    public AsyncChannelPromise await() throws InterruptedException
     {
         super.await();
         return this;
     }
 
-    public ChannelPromise awaitUninterruptibly()
+    public AsyncChannelPromise awaitUninterruptibly()
     {
         super.awaitUninterruptibly();
         return this;
     }
 
-    public ChannelPromise addListener(GenericFutureListener<? extends Future<? super Void>> listener)
+    public AsyncChannelPromise addListener(GenericFutureListener<? extends Future<? super Void>> listener)
     {
         super.addListener(listener);
         return this;
     }
 
-    public ChannelPromise addListeners(GenericFutureListener<? extends Future<? super Void>>... listeners)
+    public AsyncChannelPromise addListeners(GenericFutureListener<? extends Future<? super Void>>... listeners)
     {
         super.addListeners(listeners);
         return this;
     }
 
-    public ChannelPromise removeListener(GenericFutureListener<? extends Future<? super Void>> listener)
+    public AsyncChannelPromise removeListener(GenericFutureListener<? extends Future<? super Void>> listener)
     {
-        super.removeListener(listener);
-        return this;
+        throw new UnsupportedOperationException();
     }
 
-    public ChannelPromise removeListeners(GenericFutureListener<? extends Future<? super Void>>... listeners)
+    public AsyncChannelPromise removeListeners(GenericFutureListener<? extends Future<? super Void>>... listeners)
     {
-        super.removeListeners(listeners);
-        return this;
+        throw new UnsupportedOperationException();
     }
 
     public ChannelPromise unvoid()
diff --git a/src/java/org/apache/cassandra/net/AsyncOneResponse.java b/src/java/org/apache/cassandra/net/AsyncOneResponse.java
index ba83c84..f3c42cd 100644
--- a/src/java/org/apache/cassandra/net/AsyncOneResponse.java
+++ b/src/java/org/apache/cassandra/net/AsyncOneResponse.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.net;
 
 import com.google.common.annotations.VisibleForTesting;
 
-import io.netty.util.concurrent.ImmediateEventExecutor;
+import org.apache.cassandra.utils.concurrent.AsyncPromise;
 
 /**
  * A callback specialized for returning a value from a single target; that is, this is for messages
@@ -27,11 +27,6 @@ import io.netty.util.concurrent.ImmediateEventExecutor;
  */
 public class AsyncOneResponse<T> extends AsyncPromise<T> implements RequestCallback<T>
 {
-    public AsyncOneResponse()
-    {
-        super(ImmediateEventExecutor.INSTANCE);
-    }
-
     public void onResponse(Message<T> response)
     {
         setSuccess(response.payload);
diff --git a/src/java/org/apache/cassandra/net/AsyncPromise.java b/src/java/org/apache/cassandra/net/AsyncPromise.java
deleted file mode 100644
index d8bb24f..0000000
--- a/src/java/org/apache/cassandra/net/AsyncPromise.java
+++ /dev/null
@@ -1,489 +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.concurrent.CancellationException;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import io.netty.util.concurrent.EventExecutor;
-import io.netty.util.concurrent.Future;
-import io.netty.util.concurrent.GenericFutureListener;
-import io.netty.util.concurrent.Promise;
-import io.netty.util.internal.PlatformDependent;
-import io.netty.util.internal.ThrowableUtil;
-import org.apache.cassandra.utils.concurrent.WaitQueue;
-
-import static java.util.concurrent.atomic.AtomicReferenceFieldUpdater.*;
-import static org.apache.cassandra.utils.Clock.Global.nanoTime;
-
-/**
- * Netty's DefaultPromise uses a mutex to coordinate notifiers AND waiters between the eventLoop and the other threads.
- * Since we register cross-thread listeners, this has the potential to block internode messaging for an unknown
- * number of threads for an unknown period of time, if we are unlucky with the scheduler (which will certainly
- * happen, just with some unknown but low periodicity)
- *
- * At the same time, we manage some other efficiencies:
- *  - We save some space when registering listeners, especially if there is only one listener, as we perform no
- *    extra allocations in this case.
- *  - We permit efficient initial state declaration, avoiding unnecessary CAS or lock acquisitions when mutating
- *    a Promise we are ourselves constructing (and can easily add more; only those we use have been added)
- *
- * We can also make some guarantees about our behaviour here, although we primarily mirror Netty.
- * Specifically, we can guarantee that notifiers are always invoked in the order they are added (which may be true
- * for netty, but was unclear and is not declared).  This is useful for ensuring the correctness of some of our
- * behaviours in OutboundConnection without having to jump through extra hoops.
- *
- * The implementation loosely follows that of Netty's DefaultPromise, with some slight changes; notably that we have
- * no synchronisation on our listeners, instead using a CoW list that is cleared each time we notify listeners.
- *
- * We handle special values slightly differently.  We do not use a special value for null, instead using
- * a special value to indicate the result has not been set yet.  This means that once isSuccess() holds,
- * the result must be a correctly typed object (modulo generics pitfalls).
- * All special values are also instances of FailureHolder, which simplifies a number of the logical conditions.
- *
- * @param <V>
- */
-public class AsyncPromise<V> implements Promise<V>
-{
-    private static final Logger logger = LoggerFactory.getLogger(AsyncPromise.class);
-
-    private final EventExecutor executor;
-    private volatile Object result;
-    private volatile GenericFutureListener<? extends Future<? super V>> listeners;
-    private volatile WaitQueue waiting;
-    private static final AtomicReferenceFieldUpdater<AsyncPromise, Object> resultUpdater = newUpdater(AsyncPromise.class, Object.class, "result");
-    private static final AtomicReferenceFieldUpdater<AsyncPromise, GenericFutureListener> listenersUpdater = newUpdater(AsyncPromise.class, GenericFutureListener.class, "listeners");
-    private static final AtomicReferenceFieldUpdater<AsyncPromise, WaitQueue> waitingUpdater = newUpdater(AsyncPromise.class, WaitQueue.class, "waiting");
-
-    private static final FailureHolder UNSET = new FailureHolder(null);
-    private static final FailureHolder UNCANCELLABLE = new FailureHolder(null);
-    private static final FailureHolder CANCELLED = new FailureHolder(ThrowableUtil.unknownStackTrace(new CancellationException(), AsyncPromise.class, "cancel(...)"));
-
-    private static final DeferredGenericFutureListener NOTIFYING = future -> {};
-    private static interface DeferredGenericFutureListener<F extends Future<?>> extends GenericFutureListener<F> {}
-
-    private static final class FailureHolder
-    {
-        final Throwable cause;
-        private FailureHolder(Throwable cause)
-        {
-            this.cause = cause;
-        }
-    }
-
-    public AsyncPromise(EventExecutor executor)
-    {
-        this(executor, UNSET);
-    }
-
-    private AsyncPromise(EventExecutor executor, FailureHolder initialState)
-    {
-        this.executor = executor;
-        this.result = initialState;
-    }
-
-    public AsyncPromise(EventExecutor executor, GenericFutureListener<? extends Future<? super V>> listener)
-    {
-        this(executor);
-        this.listeners = listener;
-    }
-
-    AsyncPromise(EventExecutor executor, FailureHolder initialState, GenericFutureListener<? extends Future<? super V>> listener)
-    {
-        this(executor, initialState);
-        this.listeners = listener;
-    }
-
-    public static <V> AsyncPromise<V> uncancellable(EventExecutor executor)
-    {
-        return new AsyncPromise<>(executor, UNCANCELLABLE);
-    }
-
-    public static <V> AsyncPromise<V> uncancellable(EventExecutor executor, GenericFutureListener<? extends Future<? super V>> listener)
-    {
-        return new AsyncPromise<>(executor, UNCANCELLABLE);
-    }
-
-    public Promise<V> setSuccess(V v)
-    {
-        if (!trySuccess(v))
-            throw new IllegalStateException("complete already: " + this);
-        return this;
-    }
-
-    public Promise<V> setFailure(Throwable throwable)
-    {
-        if (!tryFailure(throwable))
-            throw new IllegalStateException("complete already: " + this);
-        return this;
-    }
-
-    public boolean trySuccess(V v)
-    {
-        return trySet(v);
-    }
-
-    public boolean tryFailure(Throwable throwable)
-    {
-        return trySet(new FailureHolder(throwable));
-    }
-
-    public boolean setUncancellable()
-    {
-        if (trySet(UNCANCELLABLE))
-            return true;
-        return result == UNCANCELLABLE;
-    }
-
-    public boolean cancel(boolean b)
-    {
-        return trySet(CANCELLED);
-    }
-
-    /**
-     * Shared implementation of various promise completion methods.
-     * Updates the result if it is possible to do so, returning success/failure.
-     *
-     * If the promise is UNSET the new value will succeed;
-     *          if it is UNCANCELLABLE it will succeed only if the new value is not CANCELLED
-     *          otherwise it will fail, as isDone() is implied
-     *
-     * If the update succeeds, and the new state implies isDone(), any listeners and waiters will be notified
-     */
-    private boolean trySet(Object v)
-    {
-        while (true)
-        {
-            Object current = result;
-            if (isDone(current) || (current == UNCANCELLABLE && v == CANCELLED))
-                return false;
-            if (resultUpdater.compareAndSet(this, current, v))
-            {
-                if (v != UNCANCELLABLE)
-                {
-                    notifyListeners();
-                    notifyWaiters();
-                }
-                return true;
-            }
-        }
-    }
-
-    public boolean isSuccess()
-    {
-        return isSuccess(result);
-    }
-
-    private static boolean isSuccess(Object result)
-    {
-        return !(result instanceof FailureHolder);
-    }
-
-    public boolean isCancelled()
-    {
-        return isCancelled(result);
-    }
-
-    private static boolean isCancelled(Object result)
-    {
-        return result == CANCELLED;
-    }
-
-    public boolean isDone()
-    {
-        return isDone(result);
-    }
-
-    private static boolean isDone(Object result)
-    {
-        return result != UNSET && result != UNCANCELLABLE;
-    }
-
-    public boolean isCancellable()
-    {
-        Object result = this.result;
-        return result == UNSET;
-    }
-
-    public Throwable cause()
-    {
-        Object result = this.result;
-        if (result instanceof FailureHolder)
-            return ((FailureHolder) result).cause;
-        return null;
-    }
-
-    /**
-     * if isSuccess(), returns the value, otherwise returns null
-     */
-    @SuppressWarnings("unchecked")
-    public V getNow()
-    {
-        Object result = this.result;
-        if (isSuccess(result))
-            return (V) result;
-        return null;
-    }
-
-    public V get() throws InterruptedException, ExecutionException
-    {
-        await();
-        return getWhenDone();
-    }
-
-    public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-    {
-        if (!await(timeout, unit))
-            throw new TimeoutException();
-        return getWhenDone();
-    }
-
-    /**
-     * Shared implementation of get() after suitable await(); assumes isDone(), and returns
-     * either the success result or throws the suitable exception under failure
-     */
-    @SuppressWarnings("unchecked")
-    private V getWhenDone() throws ExecutionException
-    {
-        Object result = this.result;
-        if (isSuccess(result))
-            return (V) result;
-        if (result == CANCELLED)
-            throw new CancellationException();
-        throw new ExecutionException(((FailureHolder) result).cause);
-    }
-
-    /**
-     * waits for completion; in case of failure rethrows the original exception without a new wrapping exception
-     * so may cause problems for reporting stack traces
-     */
-    public Promise<V> sync() throws InterruptedException
-    {
-        await();
-        rethrowIfFailed();
-        return this;
-    }
-
-    /**
-     * waits for completion; in case of failure rethrows the original exception without a new wrapping exception
-     * so may cause problems for reporting stack traces
-     */
-    public Promise<V> syncUninterruptibly()
-    {
-        awaitUninterruptibly();
-        rethrowIfFailed();
-        return this;
-    }
-
-    private void rethrowIfFailed()
-    {
-        Throwable cause = this.cause();
-        if (cause != null)
-        {
-            PlatformDependent.throwException(cause);
-        }
-    }
-
-    public Promise<V> addListener(GenericFutureListener<? extends Future<? super V>> listener)
-    {
-        listenersUpdater.accumulateAndGet(this, listener, AsyncPromise::appendListener);
-        if (isDone())
-            notifyListeners();
-        return this;
-    }
-
-    public Promise<V> addListeners(GenericFutureListener<? extends Future<? super V>> ... listeners)
-    {
-        // this could be more efficient if we cared, but we do not
-        return addListener(future -> {
-            for (GenericFutureListener<? extends Future<? super V>> listener : listeners)
-                AsyncPromise.invokeListener((GenericFutureListener<Future<? super V>>)listener, future);
-        });
-    }
-
-    public Promise<V> removeListener(GenericFutureListener<? extends Future<? super V>> listener)
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public Promise<V> removeListeners(GenericFutureListener<? extends Future<? super V>> ... listeners)
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    @SuppressWarnings("unchecked")
-    private void notifyListeners()
-    {
-        if (!executor.inEventLoop())
-        {
-            // submit this method, to guarantee we invoke in the submitted order
-            executor.execute(this::notifyListeners);
-            return;
-        }
-
-        if (listeners == null || listeners instanceof DeferredGenericFutureListener<?>)
-            return; // either no listeners, or we are already notifying listeners, so we'll get to the new one when ready
-
-        // first run our notifiers
-        while (true)
-        {
-            GenericFutureListener listeners = listenersUpdater.getAndSet(this, NOTIFYING);
-            if (listeners != null)
-                invokeListener(listeners, this);
-
-            if (listenersUpdater.compareAndSet(this, NOTIFYING, null))
-                return;
-        }
-    }
-
-    private static <F extends Future<?>> void invokeListener(GenericFutureListener<F> listener, F future)
-    {
-        try
-        {
-            listener.operationComplete(future);
-        }
-        catch (Throwable t)
-        {
-            logger.error("Failed to invoke listener {} to {}", listener, future, t);
-        }
-    }
-
-    private static <F extends Future<?>> GenericFutureListener<F> appendListener(GenericFutureListener<F> prevListener, GenericFutureListener<F> newListener)
-    {
-        GenericFutureListener<F> result = newListener;
-
-        if (prevListener != null && prevListener != NOTIFYING)
-        {
-            result = future -> {
-                invokeListener(prevListener, future);
-                // we will wrap the outer invocation with invokeListener, so no need to do it here too
-                newListener.operationComplete(future);
-            };
-        }
-
-        if (prevListener instanceof DeferredGenericFutureListener<?>)
-        {
-            GenericFutureListener<F> wrap = result;
-            result = (DeferredGenericFutureListener<F>) wrap::operationComplete;
-        }
-
-        return result;
-    }
-
-    public Promise<V> await() throws InterruptedException
-    {
-        await(0L, (signal, nanos) -> { signal.await(); return true; } );
-        return this;
-    }
-
-    public Promise<V> awaitUninterruptibly()
-    {
-        await(0L, (signal, nanos) -> { signal.awaitUninterruptibly(); return true; } );
-        return this;
-    }
-
-    public boolean await(long timeout, TimeUnit unit) throws InterruptedException
-    {
-        return await(unit.toNanos(timeout),
-                     (signal, nanos) -> signal.awaitUntil(nanos + nanoTime()));
-    }
-
-    public boolean await(long timeoutMillis) throws InterruptedException
-    {
-        return await(timeoutMillis, TimeUnit.MILLISECONDS);
-    }
-
-    public boolean awaitUninterruptibly(long timeout, TimeUnit unit)
-    {
-        return await(unit.toNanos(timeout),
-                     (signal, nanos) -> signal.awaitUntilUninterruptibly(nanos + nanoTime()));
-    }
-
-    public boolean awaitUninterruptibly(long timeoutMillis)
-    {
-        return awaitUninterruptibly(timeoutMillis, TimeUnit.MILLISECONDS);
-    }
-
-    interface Awaiter<T extends Throwable>
-    {
-        boolean await(WaitQueue.Signal value, long nanos) throws T;
-    }
-
-    /**
-     * A clean way to implement each variant of await using lambdas; we permit a nanos parameter
-     * so that we can implement this without any unnecessary lambda allocations, although not
-     * all implementations need the nanos parameter (i.e. those that wait indefinitely)
-     */
-    private <T extends Throwable> boolean await(long nanos, Awaiter<T> awaiter) throws T
-    {
-        if (isDone())
-            return true;
-
-        WaitQueue.Signal await = registerToWait();
-        if (null != await)
-            return awaiter.await(await, nanos);
-
-        return true;
-    }
-
-    /**
-     * Register a signal that will be notified when the promise is completed;
-     * if the promise becomes completed before this signal is registered, null is returned
-     */
-    private WaitQueue.Signal registerToWait()
-    {
-        WaitQueue waiting = this.waiting;
-        if (waiting == null && !waitingUpdater.compareAndSet(this, null, waiting = new WaitQueue()))
-            waiting = this.waiting;
-        assert waiting != null;
-
-        WaitQueue.Signal signal = waiting.register();
-        if (!isDone())
-            return signal;
-        signal.cancel();
-        return null;
-    }
-
-    private void notifyWaiters()
-    {
-        WaitQueue waiting = this.waiting;
-        if (waiting != null)
-            waiting.signalAll();
-    }
-
-    public String toString()
-    {
-        Object result = this.result;
-        if (isSuccess(result))
-            return "(success: " + result + ')';
-        if (result == UNCANCELLABLE)
-            return "(uncancellable)";
-        if (result == CANCELLED)
-            return "(cancelled)";
-        if (isDone(result))
-            return "(failure: " + ((FailureHolder) result).cause + ')';
-        return "(incomplete)";
-    }
-}
diff --git a/src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.java b/src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.java
index 84fb8ac..7e73355 100644
--- a/src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.java
+++ b/src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.java
@@ -21,7 +21,6 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -33,6 +32,8 @@ import io.netty.buffer.Unpooled;
 import io.netty.channel.Channel;
 import org.apache.cassandra.io.util.RebufferingInputStream;
 
+import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
+
 // TODO: rewrite
 public class AsyncStreamingInputPlus extends RebufferingInputStream
 {
@@ -65,7 +66,7 @@ public class AsyncStreamingInputPlus extends RebufferingInputStream
         super(Unpooled.EMPTY_BUFFER.nioBuffer());
         currentBuf = Unpooled.EMPTY_BUFFER;
 
-        queue = new LinkedBlockingQueue<>();
+        queue = newBlockingQueue();
         rebufferTimeoutNanos = rebufferTimeoutUnit.toNanos(rebufferTimeout);
 
         this.channel = channel;
diff --git a/src/java/org/apache/cassandra/net/FrameDecoderCrc.java b/src/java/org/apache/cassandra/net/FrameDecoderCrc.java
index 238a890..2a54f5f 100644
--- a/src/java/org/apache/cassandra/net/FrameDecoderCrc.java
+++ b/src/java/org/apache/cassandra/net/FrameDecoderCrc.java
@@ -25,7 +25,6 @@ import java.util.zip.CRC32;
 import io.netty.channel.ChannelPipeline;
 
 import static org.apache.cassandra.net.Crc.*;
-import static org.apache.cassandra.net.Crc.updateCrc32;
 
 /**
  * Framing format that protects integrity of data in movement with CRCs (of both header and payload).
diff --git a/src/java/org/apache/cassandra/net/FrameEncoderCrc.java b/src/java/org/apache/cassandra/net/FrameEncoderCrc.java
index 1d16868..3646248 100644
--- a/src/java/org/apache/cassandra/net/FrameEncoderCrc.java
+++ b/src/java/org/apache/cassandra/net/FrameEncoderCrc.java
@@ -23,7 +23,6 @@ import java.util.zip.CRC32;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.ChannelHandler;
-import org.apache.cassandra.utils.memory.BufferPool;
 
 import static org.apache.cassandra.net.Crc.*;
 
diff --git a/src/java/org/apache/cassandra/net/FrameEncoderLegacyLZ4.java b/src/java/org/apache/cassandra/net/FrameEncoderLegacyLZ4.java
index 000fab7..fd8b36b 100644
--- a/src/java/org/apache/cassandra/net/FrameEncoderLegacyLZ4.java
+++ b/src/java/org/apache/cassandra/net/FrameEncoderLegacyLZ4.java
@@ -27,7 +27,6 @@ import net.jpountz.xxhash.XXHash32;
 import net.jpountz.xxhash.XXHashFactory;
 import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.memory.BufferPool;
 
 import static java.lang.Integer.reverseBytes;
 import static java.lang.Math.min;
diff --git a/src/java/org/apache/cassandra/net/FrameEncoderUnprotected.java b/src/java/org/apache/cassandra/net/FrameEncoderUnprotected.java
index 3d10acf..8ea36c6 100644
--- a/src/java/org/apache/cassandra/net/FrameEncoderUnprotected.java
+++ b/src/java/org/apache/cassandra/net/FrameEncoderUnprotected.java
@@ -21,7 +21,6 @@ import java.nio.ByteBuffer;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.ChannelHandler;
-import org.apache.cassandra.utils.memory.BufferPool;
 
 import static org.apache.cassandra.net.FrameEncoderCrc.HEADER_LENGTH;
 import static org.apache.cassandra.net.FrameEncoderCrc.writeHeader;
diff --git a/src/java/org/apache/cassandra/net/FutureCombiner.java b/src/java/org/apache/cassandra/net/FutureCombiner.java
index dd094bd..865ca46 100644
--- a/src/java/org/apache/cassandra/net/FutureCombiner.java
+++ b/src/java/org/apache/cassandra/net/FutureCombiner.java
@@ -25,6 +25,7 @@ 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.
diff --git a/src/java/org/apache/cassandra/net/HandshakeProtocol.java b/src/java/org/apache/cassandra/net/HandshakeProtocol.java
index bfdcc2c..a82c115 100644
--- a/src/java/org/apache/cassandra/net/HandshakeProtocol.java
+++ b/src/java/org/apache/cassandra/net/HandshakeProtocol.java
@@ -40,7 +40,6 @@ import static org.apache.cassandra.net.MessagingService.VERSION_30;
 import static org.apache.cassandra.net.MessagingService.VERSION_40;
 import static org.apache.cassandra.net.Message.validateLegacyProtocolMagic;
 import static org.apache.cassandra.net.Crc.*;
-import static org.apache.cassandra.net.Crc.computeCrc32;
 import static org.apache.cassandra.net.OutboundConnectionSettings.*;
 
 /**
@@ -53,7 +52,8 @@ import static org.apache.cassandra.net.OutboundConnectionSettings.*;
  * it will simply disconnect and reconnect with a more appropriate version. But if the version is acceptable, the connection
  * initiator sends the third message of the protocol, after which it considers the connection ready.
  */
-class HandshakeProtocol
+@VisibleForTesting
+public class HandshakeProtocol
 {
     static final long TIMEOUT_MILLIS = 3 * DatabaseDescriptor.getRpcTimeout(MILLISECONDS);
 
diff --git a/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java b/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
index 3663029..752dc63 100644
--- a/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
+++ b/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
@@ -58,9 +58,6 @@ import org.apache.cassandra.utils.memory.BufferPools;
 import static java.lang.Math.*;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.cassandra.net.MessagingService.*;
-import static org.apache.cassandra.net.MessagingService.VERSION_40;
-import static org.apache.cassandra.net.MessagingService.current_version;
-import static org.apache.cassandra.net.MessagingService.minimum_version;
 import static org.apache.cassandra.net.SocketFactory.WIRETRACE;
 import static org.apache.cassandra.net.SocketFactory.newSslHandler;
 
diff --git a/src/java/org/apache/cassandra/net/InboundSockets.java b/src/java/org/apache/cassandra/net/InboundSockets.java
index fc57224..d1a4f7b 100644
--- a/src/java/org/apache/cassandra/net/InboundSockets.java
+++ b/src/java/org/apache/cassandra/net/InboundSockets.java
@@ -38,6 +38,7 @@ import io.netty.util.concurrent.SucceededFuture;
 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;
 
 class InboundSockets
 {
diff --git a/src/java/org/apache/cassandra/net/MessagingServiceMBeanImpl.java b/src/java/org/apache/cassandra/net/MessagingServiceMBeanImpl.java
index bea2b8c..a633e53 100644
--- a/src/java/org/apache/cassandra/net/MessagingServiceMBeanImpl.java
+++ b/src/java/org/apache/cassandra/net/MessagingServiceMBeanImpl.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.net;
 
 import java.io.IOException;
 import java.net.UnknownHostException;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
diff --git a/src/java/org/apache/cassandra/net/OutboundConnection.java b/src/java/org/apache/cassandra/net/OutboundConnection.java
index 82eb6ce..0179166 100644
--- a/src/java/org/apache/cassandra/net/OutboundConnection.java
+++ b/src/java/org/apache/cassandra/net/OutboundConnection.java
@@ -23,7 +23,6 @@ import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.nio.channels.ClosedChannelException;
 import java.util.Objects;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -34,7 +33,9 @@ import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import javax.annotation.Nullable;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.Uninterruptibles;
+
+import org.apache.cassandra.utils.concurrent.AsyncPromise;
+import org.apache.cassandra.utils.concurrent.CountDownLatch;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -55,6 +56,7 @@ import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.NoSpamLogger;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 import static java.lang.Math.max;
 import static java.lang.Math.min;
@@ -68,6 +70,7 @@ import static org.apache.cassandra.net.SocketFactory.*;
 import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemory;
 import static org.apache.cassandra.utils.MonotonicClock.approxTime;
 import static org.apache.cassandra.utils.Throwables.isCausedBy;
+import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownLatch;
 
 /**
  * Represents a connection type to a peer, and handles the state transistions on the connection and the netty {@link Channel}.
@@ -1031,7 +1034,7 @@ public class OutboundConnection
                 }
                 catch (InterruptedException e)
                 {
-                    throw new RuntimeException(e);
+                    throw new UncheckedInterruptedException(e);
                 }
             });
         }
@@ -1514,13 +1517,12 @@ public class OutboundConnection
 
         Runnable clearQueue = () ->
         {
-            CountDownLatch done = new CountDownLatch(1);
+            CountDownLatch done = newCountDownLatch(1);
             queue.runEventually(withLock -> {
                 withLock.consume(this::onClosed);
-                done.countDown();
+                done.decrement();
             });
-            //noinspection UnstableApiUsage
-            Uninterruptibles.awaitUninterruptibly(done);
+            done.awaitUninterruptibly();
         };
 
         if (flushQueue)
diff --git a/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java b/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java
index 15f83c2..71eb132 100644
--- a/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java
+++ b/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java
@@ -24,6 +24,7 @@ import java.nio.channels.ClosedChannelException;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.cassandra.utils.concurrent.AsyncPromise;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -50,7 +51,6 @@ 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.HandshakeProtocol.Initiate;
 import org.apache.cassandra.net.OutboundConnectionInitiator.Result.MessagingSuccess;
 import org.apache.cassandra.net.OutboundConnectionInitiator.Result.StreamingSuccess;
 import org.apache.cassandra.security.ISslContextFactory;
diff --git a/src/java/org/apache/cassandra/net/OutboundConnections.java b/src/java/org/apache/cassandra/net/OutboundConnections.java
index 3f607d1..68157c3 100644
--- a/src/java/org/apache/cassandra/net/OutboundConnections.java
+++ b/src/java/org/apache/cassandra/net/OutboundConnections.java
@@ -27,22 +27,30 @@ import java.util.function.Function;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.carrotsearch.hppc.ObjectObjectHashMap;
 import io.netty.util.concurrent.Future;
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.InternodeOutboundMetrics;
 import org.apache.cassandra.utils.NoSpamLogger;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
-
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
+
+import static java.lang.Integer.getInteger;
+import static java.lang.Math.max;
+import static org.apache.cassandra.config.Config.PROPERTY_PREFIX;
+import static org.apache.cassandra.gms.Gossiper.instance;
+import static org.apache.cassandra.net.FrameEncoderCrc.HEADER_AND_TRAILER_LENGTH;
+import static org.apache.cassandra.net.LegacyLZ4Constants.HEADER_LENGTH;
 import static org.apache.cassandra.net.MessagingService.current_version;
 import static org.apache.cassandra.net.ConnectionType.URGENT_MESSAGES;
 import static org.apache.cassandra.net.ConnectionType.LARGE_MESSAGES;
 import static org.apache.cassandra.net.ConnectionType.SMALL_MESSAGES;
+import static org.apache.cassandra.net.ResourceLimits.*;
+import static org.apache.cassandra.net.Verb.Priority.P0;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
 
 /**
  * Groups a set of outbound connections to a given peer, and routes outgoing messages to the appropriate connection
@@ -54,12 +62,12 @@ public class OutboundConnections
     private static final Logger logger = LoggerFactory.getLogger(OutboundConnections.class);
 
     @VisibleForTesting
-    public static final int LARGE_MESSAGE_THRESHOLD = Integer.getInteger(Config.PROPERTY_PREFIX + "otcp_large_message_threshold", 1024 * 64)
-    - Math.max(Math.max(LegacyLZ4Constants.HEADER_LENGTH, FrameEncoderCrc.HEADER_AND_TRAILER_LENGTH), FrameEncoderLZ4.HEADER_AND_TRAILER_LENGTH);
+    public static final int LARGE_MESSAGE_THRESHOLD = getInteger(PROPERTY_PREFIX + "otcp_large_message_threshold", 1024 * 64)
+    - max(max(HEADER_LENGTH, HEADER_AND_TRAILER_LENGTH), FrameEncoderLZ4.HEADER_AND_TRAILER_LENGTH);
 
-    private final SimpleCondition metricsReady = new SimpleCondition();
+    private final Condition metricsReady = newOneTimeCondition();
     private volatile InternodeOutboundMetrics metrics;
-    private final ResourceLimits.Limit reserveCapacity;
+    private final Limit reserveCapacity;
 
     private OutboundConnectionSettings template;
     public final OutboundConnection small;
@@ -69,8 +77,8 @@ public class OutboundConnections
     private OutboundConnections(OutboundConnectionSettings template)
     {
         this.template = template = template.withDefaultReserveLimits();
-        reserveCapacity = new ResourceLimits.Concurrent(template.applicationSendQueueReserveEndpointCapacityInBytes);
-        ResourceLimits.EndpointAndGlobal reserveCapacityInBytes = new ResourceLimits.EndpointAndGlobal(reserveCapacity, template.applicationSendQueueReserveGlobalCapacityInBytes);
+        reserveCapacity = new Concurrent(template.applicationSendQueueReserveEndpointCapacityInBytes);
+        EndpointAndGlobal reserveCapacityInBytes = new EndpointAndGlobal(reserveCapacity, template.applicationSendQueueReserveGlobalCapacityInBytes);
         this.small = new OutboundConnection(SMALL_MESSAGES, template, reserveCapacityInBytes);
         this.large = new OutboundConnection(LARGE_MESSAGES, template, reserveCapacityInBytes);
         this.urgent = new OutboundConnection(URGENT_MESSAGES, template, reserveCapacityInBytes);
@@ -158,7 +166,7 @@ public class OutboundConnections
         }
         catch (InterruptedException e)
         {
-            throw new RuntimeException(e);
+            throw new UncheckedInterruptedException(e);
         }
 
         if (metrics != null)
@@ -299,7 +307,7 @@ public class OutboundConnections
                     continue;
 
                 if (cur.small == prev.small && cur.large == prev.large && cur.urgent == prev.urgent
-                    && !Gossiper.instance.isKnownEndpoint(connections.template.to))
+                    && !instance.isKnownEndpoint(connections.template.to))
                 {
                     logger.info("Closing outbound connections to {}, as inactive and not known by Gossiper",
                                 connections.template.to);
diff --git a/src/java/org/apache/cassandra/net/OutboundMessageQueue.java b/src/java/org/apache/cassandra/net/OutboundMessageQueue.java
index d7360a0..860890a 100644
--- a/src/java/org/apache/cassandra/net/OutboundMessageQueue.java
+++ b/src/java/org/apache/cassandra/net/OutboundMessageQueue.java
@@ -17,10 +17,8 @@
  */
 package org.apache.cassandra.net;
 
-import java.util.Collections;
 import java.util.IdentityHashMap;
 import java.util.Set;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLongFieldUpdater;
 import java.util.concurrent.atomic.AtomicReference;
@@ -28,13 +26,17 @@ import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import java.util.function.Consumer;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.Uninterruptibles;
+
+import org.apache.cassandra.utils.concurrent.CountDownLatch;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.utils.MonotonicClock;
 
+import static java.lang.Long.MAX_VALUE;
 import static java.lang.Math.min;
+import static java.util.Collections.newSetFromMap;
+import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownLatch;
 
 /**
  * A composite queue holding messages to be delivered by an {@link OutboundConnection}.
@@ -437,8 +439,8 @@ class OutboundMessageQueue
 
     private class RemoveRunner extends AtomicReference<Remove> implements Runnable
     {
-        final CountDownLatch done = new CountDownLatch(1);
-        final Set<Message<?>> removed = Collections.newSetFromMap(new IdentityHashMap<>());
+        final CountDownLatch done = newCountDownLatch(1);
+        final Set<Message<?>> removed = newSetFromMap(new IdentityHashMap<>());
 
         RemoveRunner() { super(new Remove(null, null)); }
 
@@ -449,7 +451,7 @@ class OutboundMessageQueue
 
         public void run()
         {
-            Set<Message<?>> remove = Collections.newSetFromMap(new IdentityHashMap<>());
+            Set<Message<?>> remove = newSetFromMap(new IdentityHashMap<>());
             removeRunner = null;
             Remove undo = getAndSet(null);
             while (undo.message != null)
@@ -460,7 +462,7 @@ class OutboundMessageQueue
 
             class Remover implements PrunableArrayQueue.Pruner<Message<?>>
             {
-                private long earliestExpiresAt = Long.MAX_VALUE;
+                private long earliestExpiresAt = MAX_VALUE;
 
                 @Override
                 public boolean shouldPrune(Message<?> message)
@@ -488,7 +490,7 @@ class OutboundMessageQueue
             long nowNanos = clock.now();
             maybeUpdateNextExpirationDeadline(nowNanos, maybeUpdateEarliestExpiresAt(nowNanos, remover.earliestExpiresAt));
 
-            done.countDown();
+            done.decrement();
         }
     }
 
@@ -518,7 +520,7 @@ class OutboundMessageQueue
         }
 
         //noinspection UnstableApiUsage
-        Uninterruptibles.awaitUninterruptibly(runner.done);
+        runner.done.awaitUninterruptibly();
         return runner.removed.contains(remove);
     }
 
diff --git a/src/java/org/apache/cassandra/net/StartupClusterConnectivityChecker.java b/src/java/org/apache/cassandra/net/StartupClusterConnectivityChecker.java
index cd607a4..93305db 100644
--- a/src/java/org/apache/cassandra/net/StartupClusterConnectivityChecker.java
+++ b/src/java/org/apache/cassandra/net/StartupClusterConnectivityChecker.java
@@ -23,7 +23,7 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
+import org.apache.cassandra.utils.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Function;
@@ -32,7 +32,6 @@ import java.util.stream.Collectors;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.SetMultimap;
-import com.google.common.util.concurrent.Uninterruptibles;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -48,6 +47,7 @@ import static org.apache.cassandra.net.Verb.PING_REQ;
 import static org.apache.cassandra.net.ConnectionType.LARGE_MESSAGES;
 import static org.apache.cassandra.net.ConnectionType.SMALL_MESSAGES;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownLatch;
 
 public class StartupClusterConnectivityChecker
 {
@@ -123,7 +123,7 @@ public class StartupClusterConnectivityChecker
         for (String datacenter: datacenterToPeers.keys())
         {
             dcToRemainingPeers.put(datacenter,
-                                   new CountDownLatch(Math.max(datacenterToPeers.get(datacenter).size() - 1, 0)));
+                                   newCountDownLatch(Math.max(datacenterToPeers.get(datacenter).size() - 1, 0)));
         }
 
         long startNanos = nanoTime();
@@ -144,7 +144,7 @@ public class StartupClusterConnectivityChecker
                 String datacenter = peerToDatacenter.get(peer);
                 // We have to check because we might only have the local DC in the map
                 if (dcToRemainingPeers.containsKey(datacenter))
-                    dcToRemainingPeers.get(datacenter).countDown();
+                    dcToRemainingPeers.get(datacenter).decrement();
             }
         }
 
@@ -153,14 +153,14 @@ public class StartupClusterConnectivityChecker
         {
             long remainingNanos = Math.max(1, timeoutNanos - (nanoTime() - startNanos));
             //noinspection UnstableApiUsage
-            succeeded &= Uninterruptibles.awaitUninterruptibly(countDownLatch, remainingNanos, TimeUnit.NANOSECONDS);
+            succeeded &= countDownLatch.awaitUninterruptibly(remainingNanos, TimeUnit.NANOSECONDS);
         }
 
         Gossiper.instance.unregister(listener);
 
-        Map<String, Long> numDown = dcToRemainingPeers.entrySet().stream()
+        Map<String, Integer> numDown = dcToRemainingPeers.entrySet().stream()
                                                       .collect(Collectors.toMap(Map.Entry::getKey,
-                                                                                e -> e.getValue().getCount()));
+                                                                                e -> e.getValue().count()));
 
         if (succeeded)
         {
@@ -189,7 +189,7 @@ public class StartupClusterConnectivityChecker
                 String datacenter = getDatacenter.apply(msg.from());
                 // We have to check because we might only have the local DC in the map
                 if (dcToRemainingPeers.containsKey(datacenter))
-                    dcToRemainingPeers.get(datacenter).countDown();
+                    dcToRemainingPeers.get(datacenter).decrement();
             }
         };
 
@@ -228,7 +228,7 @@ public class StartupClusterConnectivityChecker
             {
                 String datacenter = getDatacenter.apply(endpoint);
                 if (dcToRemainingPeers.containsKey(datacenter))
-                    dcToRemainingPeers.get(datacenter).countDown();
+                    dcToRemainingPeers.get(datacenter).decrement();
             }
         }
     }
diff --git a/src/java/org/apache/cassandra/net/Verb.java b/src/java/org/apache/cassandra/net/Verb.java
index fad2fbf..dd7fab3 100644
--- a/src/java/org/apache/cassandra/net/Verb.java
+++ b/src/java/org/apache/cassandra/net/Verb.java
@@ -87,8 +87,6 @@ import org.apache.cassandra.utils.UUIDSerializer;
 
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static org.apache.cassandra.concurrent.Stage.*;
-import static org.apache.cassandra.concurrent.Stage.INTERNAL_RESPONSE;
-import static org.apache.cassandra.concurrent.Stage.MISC;
 import static org.apache.cassandra.net.VerbTimeouts.*;
 import static org.apache.cassandra.net.Verb.Kind.*;
 import static org.apache.cassandra.net.Verb.Priority.*;
diff --git a/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java b/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java
index 233c5bc..9ab307b 100644
--- a/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java
+++ b/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java
@@ -60,11 +60,11 @@ public class AsymmetricRemoteSyncTask extends SyncTask implements CompletableRem
     {
         if (success)
         {
-            set(stat.withSummaries(summaries));
+            trySuccess(stat.withSummaries(summaries));
         }
         else
         {
-            setException(RepairException.warn(desc, previewKind, String.format("Sync failed between %s and %s", nodePair.coordinator, nodePair.peer)));
+            tryFailure(RepairException.warn(desc, previewKind, String.format("Sync failed between %s and %s", nodePair.coordinator, nodePair.peer)));
         }
     }
 
diff --git a/src/java/org/apache/cassandra/repair/LocalSyncTask.java b/src/java/org/apache/cassandra/repair/LocalSyncTask.java
index 5916401..dee5d98 100644
--- a/src/java/org/apache/cassandra/repair/LocalSyncTask.java
+++ b/src/java/org/apache/cassandra/repair/LocalSyncTask.java
@@ -40,7 +40,6 @@ import org.apache.cassandra.streaming.StreamState;
 import org.apache.cassandra.tracing.TraceState;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.MerkleTrees;
 
 /**
  * LocalSyncTask performs streaming between local(coordinator) node and remote replica.
@@ -150,13 +149,13 @@ public class LocalSyncTask extends SyncTask implements StreamEventHandler
         String message = String.format("Sync complete using session %s between %s and %s on %s", desc.sessionId, nodePair.coordinator, nodePair.peer, desc.columnFamily);
         logger.info("{} {}", previewKind.logPrefix(desc.sessionId), message);
         Tracing.traceRepair(message);
-        set(stat.withSummaries(result.createSummaries()));
+        trySuccess(stat.withSummaries(result.createSummaries()));
         finished();
     }
 
     public void onFailure(Throwable t)
     {
-        setException(t);
+        tryFailure(t);
         finished();
     }
 
diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java
index 9661064..7747e18 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.utils.concurrent.AsyncFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,7 +52,7 @@ import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
 /**
  * RepairJob runs repair on given ColumnFamily.
  */
-public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
+public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
 {
     private static final Logger logger = LoggerFactory.getLogger(RepairJob.class);
 
@@ -162,7 +163,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
                     SystemDistributedKeyspace.successfulRepairJob(session.getId(), desc.keyspace, desc.columnFamily);
                 }
                 cfs.metric.repairsCompleted.inc();
-                set(new RepairResult(desc, stats));
+                trySuccess(new RepairResult(desc, stats));
             }
 
             /**
@@ -179,7 +180,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
                     SystemDistributedKeyspace.failedRepairJob(session.getId(), desc.keyspace, desc.columnFamily, t);
                 }
                 cfs.metric.repairsCompleted.inc();
-                setException(t);
+                tryFailure(t);
             }
         }, taskExecutor);
     }
diff --git a/src/java/org/apache/cassandra/repair/RepairJobDesc.java b/src/java/org/apache/cassandra/repair/RepairJobDesc.java
index 4aaf655..97fd06a 100644
--- a/src/java/org/apache/cassandra/repair/RepairJobDesc.java
+++ b/src/java/org/apache/cassandra/repair/RepairJobDesc.java
@@ -32,7 +32,6 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.utils.UUIDSerializer;
 
diff --git a/src/java/org/apache/cassandra/repair/RepairRunnable.java b/src/java/org/apache/cassandra/repair/RepairRunnable.java
index f536da6..3a9de96 100644
--- a/src/java/org/apache/cassandra/repair/RepairRunnable.java
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@ -28,7 +28,6 @@ import java.util.List;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -99,6 +98,8 @@ 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;
 
 public class RepairRunnable implements Runnable, ProgressEventNotifier
 {
@@ -685,8 +686,8 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
     {
         return MoreExecutors.listeningDecorator(new JMXEnabledThreadPoolExecutor(options.getJobThreads(),
                                                                                  Integer.MAX_VALUE,
-                                                                                 TimeUnit.SECONDS,
-                                                                                 new LinkedBlockingQueue<>(),
+                                                                                 SECONDS,
+                                                                                 newBlockingQueue(),
                                                                                  new NamedThreadFactory("Repair#" + cmd),
                                                                                  "internal"));
     }
diff --git a/src/java/org/apache/cassandra/repair/RepairSession.java b/src/java/org/apache/cassandra/repair/RepairSession.java
index 4e5daf5..f44ab35 100644
--- a/src/java/org/apache/cassandra/repair/RepairSession.java
+++ b/src/java/org/apache/cassandra/repair/RepairSession.java
@@ -33,6 +33,7 @@ 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.utils.concurrent.AsyncFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -92,7 +93,7 @@ import org.apache.cassandra.utils.Throwables;
  * Similarly, if a job is sequential, it will handle one SymmetricSyncTask at a time, but will handle
  * all of them in parallel otherwise.
  */
-public class RepairSession extends AbstractFuture<RepairSessionResult> implements IEndpointStateChangeSubscriber,
+public class RepairSession extends AsyncFuture<RepairSessionResult> implements IEndpointStateChangeSubscriber,
                                                                                   IFailureDetectionEventListener,
                                                                                   LocalSessions.Listener
 {
@@ -279,7 +280,7 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
         {
             logger.info("{} {}", previewKind.logPrefix(getId()), message = String.format("No neighbors to repair with on range %s: session completed", commonRange));
             Tracing.traceRepair(message);
-            set(new RepairSessionResult(id, keyspace, commonRange.ranges, Lists.<RepairResult>newArrayList(), commonRange.hasSkippedReplicas));
+            trySuccess(new RepairSessionResult(id, keyspace, commonRange.ranges, Lists.<RepairResult>newArrayList(), commonRange.hasSkippedReplicas));
             if (!previewKind.isPreview())
             {
                 SystemDistributedKeyspace.failRepairs(getId(), keyspace, cfnames, new RuntimeException(message));
@@ -295,7 +296,7 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
                 message = String.format("Cannot proceed on repair because a neighbor (%s) is dead: session failed", endpoint);
                 logger.error("{} {}", previewKind.logPrefix(getId()), message);
                 Exception e = new IOException(message);
-                setException(e);
+                tryFailure(e);
                 if (!previewKind.isPreview())
                 {
                     SystemDistributedKeyspace.failRepairs(getId(), keyspace, cfnames, e);
@@ -321,7 +322,7 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
                 // this repair session is completed
                 logger.info("{} {}", previewKind.logPrefix(getId()), "Session completed successfully");
                 Tracing.traceRepair("Completed sync of range {}", commonRange);
-                set(new RepairSessionResult(id, keyspace, commonRange.ranges, results, commonRange.hasSkippedReplicas));
+                trySuccess(new RepairSessionResult(id, keyspace, commonRange.ranges, results, commonRange.hasSkippedReplicas));
 
                 taskExecutor.shutdown();
                 // mark this session as terminated
@@ -355,7 +356,7 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
      */
     public void forceShutdown(Throwable reason)
     {
-        setException(reason);
+        tryFailure(reason);
         taskExecutor.shutdownNow();
         terminate();
     }
diff --git a/src/java/org/apache/cassandra/repair/SnapshotTask.java b/src/java/org/apache/cassandra/repair/SnapshotTask.java
index 40e4b3d..b75d889 100644
--- a/src/java/org/apache/cassandra/repair/SnapshotTask.java
+++ b/src/java/org/apache/cassandra/repair/SnapshotTask.java
@@ -19,21 +19,20 @@ package org.apache.cassandra.repair;
 
 import java.util.concurrent.RunnableFuture;
 
-import com.google.common.util.concurrent.AbstractFuture;
-
 import org.apache.cassandra.exceptions.RequestFailureReason;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.RequestCallback;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.messages.SnapshotMessage;
+import org.apache.cassandra.utils.concurrent.AsyncFuture;
 
 import static org.apache.cassandra.net.Verb.SNAPSHOT_MSG;
 
 /**
  * SnapshotTask is a task that sends snapshot request.
  */
-public class SnapshotTask extends AbstractFuture<InetAddressAndPort> implements RunnableFuture<InetAddressAndPort>
+public class SnapshotTask extends AsyncFuture<InetAddressAndPort> implements RunnableFuture<InetAddressAndPort>
 {
     private final RepairJobDesc desc;
     private final InetAddressAndPort endpoint;
@@ -71,7 +70,7 @@ public class SnapshotTask extends AbstractFuture<InetAddressAndPort> implements
         @Override
         public void onResponse(Message msg)
         {
-            task.set(task.endpoint);
+            task.trySuccess(task.endpoint);
         }
 
         @Override
@@ -84,7 +83,7 @@ public class SnapshotTask extends AbstractFuture<InetAddressAndPort> implements
         public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason)
         {
             //listener.failedSnapshot();
-            task.setException(new RuntimeException("Could not create snapshot at " + from));
+            task.tryFailure(new RuntimeException("Could not create snapshot at " + from));
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java b/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java
index d2ae0f3..ed4306d 100644
--- a/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java
+++ b/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java
@@ -74,11 +74,11 @@ public class SymmetricRemoteSyncTask extends SyncTask implements CompletableRemo
     {
         if (success)
         {
-            set(stat.withSummaries(summaries));
+            trySuccess(stat.withSummaries(summaries));
         }
         else
         {
-            setException(RepairException.warn(desc, previewKind, String.format("Sync failed between %s and %s", nodePair.coordinator, nodePair.peer)));
+            tryFailure(RepairException.warn(desc, previewKind, String.format("Sync failed between %s and %s", nodePair.coordinator, nodePair.peer)));
         }
         finished();
     }
diff --git a/src/java/org/apache/cassandra/repair/SyncTask.java b/src/java/org/apache/cassandra/repair/SyncTask.java
index 5b23397..733203a 100644
--- a/src/java/org/apache/cassandra/repair/SyncTask.java
+++ b/src/java/org/apache/cassandra/repair/SyncTask.java
@@ -23,8 +23,8 @@ import java.util.concurrent.TimeUnit;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import com.google.common.util.concurrent.AbstractFuture;
 
+import org.apache.cassandra.utils.concurrent.AsyncFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,7 +37,7 @@ import org.apache.cassandra.tracing.Tracing;
 
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
 
-public abstract class SyncTask extends AbstractFuture<SyncStat> implements Runnable
+public abstract class SyncTask extends AsyncFuture<SyncStat> implements Runnable
 {
     private static Logger logger = LoggerFactory.getLogger(SyncTask.class);
 
@@ -81,7 +81,7 @@ public abstract class SyncTask extends AbstractFuture<SyncStat> implements Runna
         {
             logger.info(String.format(format, "are consistent"));
             Tracing.traceRepair("Endpoint {} is consistent with {} for {}", nodePair.coordinator, nodePair.peer, desc.columnFamily);
-            set(stat);
+            trySuccess(stat);
             return;
         }
 
diff --git a/src/java/org/apache/cassandra/repair/ValidationPartitionIterator.java b/src/java/org/apache/cassandra/repair/ValidationPartitionIterator.java
index ccfae41..ca337fa 100644
--- a/src/java/org/apache/cassandra/repair/ValidationPartitionIterator.java
+++ b/src/java/org/apache/cassandra/repair/ValidationPartitionIterator.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair;
 
-import java.io.IOException;
 import java.util.Map;
 
 import org.apache.cassandra.db.partitions.AbstractUnfilteredPartitionIterator;
diff --git a/src/java/org/apache/cassandra/repair/ValidationTask.java b/src/java/org/apache/cassandra/repair/ValidationTask.java
index 5a72903..616a2d8 100644
--- a/src/java/org/apache/cassandra/repair/ValidationTask.java
+++ b/src/java/org/apache/cassandra/repair/ValidationTask.java
@@ -19,8 +19,6 @@ package org.apache.cassandra.repair;
 
 import java.util.concurrent.ExecutionException;
 
-import com.google.common.util.concurrent.AbstractFuture;
-
 import org.apache.cassandra.exceptions.RepairException;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.Message;
@@ -28,6 +26,7 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.messages.ValidationRequest;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.utils.MerkleTrees;
+import org.apache.cassandra.utils.concurrent.AsyncFuture;
 
 import static org.apache.cassandra.net.Verb.VALIDATION_REQ;
 
@@ -35,7 +34,7 @@ import static org.apache.cassandra.net.Verb.VALIDATION_REQ;
  * ValidationTask sends {@link ValidationRequest} to a replica.
  * When a replica sends back message, task completes.
  */
-public class ValidationTask extends AbstractFuture<TreeResponse> implements Runnable
+public class ValidationTask extends AsyncFuture<TreeResponse> implements Runnable
 {
     private final RepairJobDesc desc;
     private final InetAddressAndPort endpoint;
@@ -71,17 +70,17 @@ public class ValidationTask extends AbstractFuture<TreeResponse> implements Runn
         if (trees == null)
         {
             active = false;
-            setException(RepairException.warn(desc, previewKind, "Validation failed in " + endpoint));
+            tryFailure(RepairException.warn(desc, previewKind, "Validation failed in " + endpoint));
         }
         else if (active)
         {
-            set(new TreeResponse(endpoint, trees));
+            trySuccess(new TreeResponse(endpoint, trees));
         }
         else
         {
             // If the task has already been aborted, just release the possibly off-heap trees and move along.
             trees.release();
-            set(null);
+            trySuccess(null);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/repair/consistent/ConsistentSession.java b/src/java/org/apache/cassandra/repair/consistent/ConsistentSession.java
index e4d8ff0..4384bfa 100644
--- a/src/java/org/apache/cassandra/repair/consistent/ConsistentSession.java
+++ b/src/java/org/apache/cassandra/repair/consistent/ConsistentSession.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair.consistent;
 
-import java.net.InetAddress;
 import java.util.Collection;
 import java.util.EnumMap;
 import java.util.List;
diff --git a/src/java/org/apache/cassandra/repair/consistent/RepairedState.java b/src/java/org/apache/cassandra/repair/consistent/RepairedState.java
index ac0e7cb..2aaec41 100644
--- a/src/java/org/apache/cassandra/repair/consistent/RepairedState.java
+++ b/src/java/org/apache/cassandra/repair/consistent/RepairedState.java
@@ -23,22 +23,16 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.PeekingIterator;
 import com.google.common.collect.Sets;
 
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.service.ActiveRepairService;
-import org.apache.cassandra.utils.UUIDGen;
 
 import static org.apache.cassandra.service.ActiveRepairService.UNREPAIRED_SSTABLE;
 
diff --git a/src/java/org/apache/cassandra/repair/consistent/admin/PendingStat.java b/src/java/org/apache/cassandra/repair/consistent/admin/PendingStat.java
index 0c4424e..f10f96d 100644
--- a/src/java/org/apache/cassandra/repair/consistent/admin/PendingStat.java
+++ b/src/java/org/apache/cassandra/repair/consistent/admin/PendingStat.java
@@ -32,10 +32,8 @@ import javax.management.openmbean.OpenDataException;
 import javax.management.openmbean.OpenType;
 import javax.management.openmbean.SimpleType;
 
-import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
-import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.FileUtils;
diff --git a/src/java/org/apache/cassandra/schema/CompressionParams.java b/src/java/org/apache/cassandra/schema/CompressionParams.java
index 53760a9..c68bdf1d 100644
--- a/src/java/org/apache/cassandra/schema/CompressionParams.java
+++ b/src/java/org/apache/cassandra/schema/CompressionParams.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.schema;
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
diff --git a/src/java/org/apache/cassandra/schema/Indexes.java b/src/java/org/apache/cassandra/schema/Indexes.java
index a83be4b..2e95779 100644
--- a/src/java/org/apache/cassandra/schema/Indexes.java
+++ b/src/java/org/apache/cassandra/schema/Indexes.java
@@ -21,7 +21,6 @@ import java.util.*;
 import java.util.stream.Stream;
 
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Sets;
 
 import static java.lang.String.format;
 
diff --git a/src/java/org/apache/cassandra/schema/MigrationCoordinator.java b/src/java/org/apache/cassandra/schema/MigrationCoordinator.java
index 9f12fe0..60fe1b6 100644
--- a/src/java/org/apache/cassandra/schema/MigrationCoordinator.java
+++ b/src/java/org/apache/cassandra/schema/MigrationCoordinator.java
@@ -62,6 +62,7 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue;
 
 public class MigrationCoordinator
 {
@@ -133,7 +134,7 @@ public class MigrationCoordinator
         final Set<InetAddressAndPort> outstandingRequests = Sets.newConcurrentHashSet();
         final Deque<InetAddressAndPort> requestQueue      = new ArrayDeque<>();
 
-        private final WaitQueue waitQueue = new WaitQueue();
+        private final WaitQueue waitQueue = newWaitQueue();
 
         volatile boolean receivedSchema;
 
@@ -561,12 +562,12 @@ public class MigrationCoordinator
             logger.debug("Nothing in versionInfo - so no schemas to wait for");
         }
 
-        WaitQueue.Signal signal = null;
+        List<WaitQueue.Signal> signalList = null;
         try
         {
             synchronized (this)
             {
-                List<WaitQueue.Signal> signalList = new ArrayList<>(versionInfo.size());
+                signalList = new ArrayList<>(versionInfo.size());
                 for (VersionInfo version : versionInfo.values())
                 {
                     if (version.wasReceived())
@@ -577,22 +578,15 @@ public class MigrationCoordinator
 
                 if (signalList.isEmpty())
                     return true;
-
-                WaitQueue.Signal[] signals = new WaitQueue.Signal[signalList.size()];
-                signalList.toArray(signals);
-                signal = WaitQueue.all(signals);
             }
 
-            return signal.awaitUntil(nanoTime() + TimeUnit.MILLISECONDS.toNanos(waitMillis));
-        }
-        catch (InterruptedException e)
-        {
-            throw new RuntimeException(e);
+            long deadline = nanoTime() + TimeUnit.MILLISECONDS.toNanos(waitMillis);
+            return signalList.stream().allMatch(signal -> signal.awaitUntilUninterruptibly(deadline));
         }
         finally
         {
-            if (signal != null)
-                signal.cancel();
+            if (signalList != null)
+                signalList.forEach(WaitQueue.Signal::cancel);
         }
     }
 }
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/schema/SchemaEvent.java b/src/java/org/apache/cassandra/schema/SchemaEvent.java
index d163a11..773decc 100644
--- a/src/java/org/apache/cassandra/schema/SchemaEvent.java
+++ b/src/java/org/apache/cassandra/schema/SchemaEvent.java
@@ -29,7 +29,6 @@ import java.util.UUID;
 import java.util.stream.Collectors;
 import javax.annotation.Nullable;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import com.google.common.collect.MapDifference;
 
diff --git a/src/java/org/apache/cassandra/schema/SchemaMigrationEvent.java b/src/java/org/apache/cassandra/schema/SchemaMigrationEvent.java
index a984804..e6f4e71 100644
--- a/src/java/org/apache/cassandra/schema/SchemaMigrationEvent.java
+++ b/src/java/org/apache/cassandra/schema/SchemaMigrationEvent.java
@@ -21,9 +21,7 @@ package org.apache.cassandra.schema;
 import java.io.Serializable;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Queue;
 import java.util.UUID;
-import java.util.concurrent.CountDownLatch;
 
 import javax.annotation.Nullable;
 
diff --git a/src/java/org/apache/cassandra/serializers/MapSerializer.java b/src/java/org/apache/cassandra/serializers/MapSerializer.java
index 9eae598..13468fc 100644
--- a/src/java/org/apache/cassandra/serializers/MapSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/MapSerializer.java
@@ -24,7 +24,6 @@ import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.db.marshal.ValueComparators;
diff --git a/src/java/org/apache/cassandra/serializers/SetSerializer.java b/src/java/org/apache/cassandra/serializers/SetSerializer.java
index 0b7a2a5..6be919f 100644
--- a/src/java/org/apache/cassandra/serializers/SetSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/SetSerializer.java
@@ -24,7 +24,6 @@ import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.db.marshal.ValueComparators;
 import org.apache.cassandra.db.marshal.ValueAccessor;
diff --git a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
index cd43338..b3b6728 100644
--- a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
@@ -22,16 +22,14 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
-import java.util.stream.Collectors;
 
 import org.apache.cassandra.db.ConsistencyLevel;
 
 import org.apache.cassandra.locator.EndpointsForToken;
-import org.apache.cassandra.locator.ReplicaPlan;
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.IMutation;
 import org.apache.cassandra.db.WriteType;
@@ -41,12 +39,20 @@ import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.RequestCallback;
 import org.apache.cassandra.net.Message;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
+import static java.lang.Long.MAX_VALUE;
+import static java.lang.Math.min;
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
-
+import static java.util.stream.Collectors.toList;
+import static org.apache.cassandra.config.DatabaseDescriptor.getCounterWriteRpcTimeout;
+import static org.apache.cassandra.config.DatabaseDescriptor.getWriteRpcTimeout;
+import static org.apache.cassandra.db.WriteType.COUNTER;
+import static org.apache.cassandra.locator.ReplicaPlan.ForTokenWrite;
+import static org.apache.cassandra.schema.Schema.instance;
+import static org.apache.cassandra.service.StorageProxy.WritePerformer;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
 
 public abstract class AbstractWriteResponseHandler<T> implements RequestCallback<T>
 {
@@ -54,13 +60,13 @@ public abstract class AbstractWriteResponseHandler<T> implements RequestCallback
 
     //Count down until all responses and expirations have occured before deciding whether the ideal CL was reached.
     private AtomicInteger responsesAndExpirations;
-    private final SimpleCondition condition = new SimpleCondition();
-    protected final ReplicaPlan.ForTokenWrite replicaPlan;
+    private final Condition condition = newOneTimeCondition();
+    protected final ForTokenWrite replicaPlan;
 
     protected final Runnable callback;
     protected final WriteType writeType;
-    private static final AtomicIntegerFieldUpdater<AbstractWriteResponseHandler> failuresUpdater
-    = AtomicIntegerFieldUpdater.newUpdater(AbstractWriteResponseHandler.class, "failures");
+    private static final AtomicIntegerFieldUpdater<AbstractWriteResponseHandler> failuresUpdater =
+        AtomicIntegerFieldUpdater.newUpdater(AbstractWriteResponseHandler.class, "failures");
     private volatile int failures = 0;
     private final Map<InetAddressAndPort, RequestFailureReason> failureReasonByEndpoint;
     private final long queryStartNanoTime;
@@ -82,7 +88,7 @@ public abstract class AbstractWriteResponseHandler<T> implements RequestCallback
      * @param callback           A callback to be called when the write is successful.
      * @param queryStartNanoTime
      */
-    protected AbstractWriteResponseHandler(ReplicaPlan.ForTokenWrite replicaPlan,
+    protected AbstractWriteResponseHandler(ForTokenWrite replicaPlan,
                                            Runnable callback,
                                            WriteType writeType,
                                            long queryStartNanoTime)
@@ -103,9 +109,9 @@ public abstract class AbstractWriteResponseHandler<T> implements RequestCallback
         {
             success = condition.await(timeoutNanos, NANOSECONDS);
         }
-        catch (InterruptedException ex)
+        catch (InterruptedException e)
         {
-            throw new AssertionError(ex);
+            throw new UncheckedInterruptedException(e);
         }
 
         if (!success)
@@ -128,9 +134,9 @@ public abstract class AbstractWriteResponseHandler<T> implements RequestCallback
 
     public final long currentTimeoutNanos()
     {
-        long requestTimeout = writeType == WriteType.COUNTER
-                              ? DatabaseDescriptor.getCounterWriteRpcTimeout(NANOSECONDS)
-                              : DatabaseDescriptor.getWriteRpcTimeout(NANOSECONDS);
+        long requestTimeout = writeType == COUNTER
+                              ? getCounterWriteRpcTimeout(NANOSECONDS)
+                              : getWriteRpcTimeout(NANOSECONDS);
         return requestTimeout - (nanoTime() - queryStartNanoTime);
     }
 
@@ -282,7 +288,7 @@ public abstract class AbstractWriteResponseHandler<T> implements RequestCallback
         {
             // The condition being signaled is a valid proxy for the CL being achieved
             // Only mark it as failed if the requested CL was achieved.
-            if (!condition.isSignaled() && requestedCLAchieved)
+            if (!condition.isSignalled() && requestedCLAchieved)
             {
                 replicaPlan.keyspace().metric.writeFailedIdealCL.inc();
             }
@@ -296,18 +302,18 @@ public abstract class AbstractWriteResponseHandler<T> implements RequestCallback
     /**
      * Cheap Quorum backup.  If we failed to reach quorum with our initial (full) nodes, reach out to other nodes.
      */
-    public void maybeTryAdditionalReplicas(IMutation mutation, StorageProxy.WritePerformer writePerformer, String localDC)
+    public void maybeTryAdditionalReplicas(IMutation mutation, WritePerformer writePerformer, String localDC)
     {
         EndpointsForToken uncontacted = replicaPlan.liveUncontacted();
         if (uncontacted.isEmpty())
             return;
 
-        long timeout = Long.MAX_VALUE;
+        long timeout = MAX_VALUE;
         List<ColumnFamilyStore> cfs = mutation.getTableIds().stream()
-                                              .map(Schema.instance::getColumnFamilyStoreInstance)
-                                              .collect(Collectors.toList());
+                                              .map(instance::getColumnFamilyStoreInstance)
+                                              .collect(toList());
         for (ColumnFamilyStore cf : cfs)
-            timeout = Math.min(timeout, cf.additionalWriteLatencyNanos);
+            timeout = min(timeout, cf.additionalWriteLatencyNanos);
 
         // no latency information, or we're overloaded
         if (timeout > mutation.getTimeout(NANOSECONDS))
@@ -325,9 +331,9 @@ public abstract class AbstractWriteResponseHandler<T> implements RequestCallback
                                      localDC);
             }
         }
-        catch (InterruptedException ex)
+        catch (InterruptedException e)
         {
-            throw new AssertionError(ex);
+            throw new UncheckedInterruptedException(e);
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index e172ed3..9da9111 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -32,13 +32,14 @@ import com.google.common.cache.CacheBuilder;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
-import com.google.common.util.concurrent.AbstractFuture;
+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.db.compaction.CompactionManager;
 import org.apache.cassandra.locator.EndpointsByRange;
 import org.apache.cassandra.locator.EndpointsForRange;
+import org.apache.cassandra.utils.concurrent.CountDownLatch;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -95,8 +96,14 @@ import org.apache.cassandra.utils.UUIDGen;
 
 import static com.google.common.collect.Iterables.concat;
 import static com.google.common.collect.Iterables.transform;
+import static java.util.Collections.synchronizedSet;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.cassandra.config.DatabaseDescriptor.getRpcTimeout;
+import static org.apache.cassandra.net.Message.out;
 import static org.apache.cassandra.net.Verb.PREPARE_MSG;
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
+import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownLatch;
+import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
 /**
  * ActiveRepairService is the starting point for manual "active" repairs.
@@ -174,7 +181,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
             // queue will _never_ be full. Idle core threads will eventually time
             // out and may be re-created if/when subsequent tasks are submitted.
             corePoolSize = maxPoolSize;
-            queue = new LinkedBlockingQueue<>();
+            queue = newBlockingQueue();
         }
 
         ThreadPoolExecutor executor = new JMXEnabledThreadPoolExecutor(corePoolSize,
@@ -379,7 +386,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
         DatabaseDescriptor.useOffheapMerkleTrees(value);
     }
 
-    private <T extends AbstractFuture &
+    private <T extends ListenableFuture &
                IEndpointStateChangeSubscriber &
                IFailureDetectionEventListener> void registerOnFdAndGossip(final T task)
     {
@@ -541,15 +548,15 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
 
         long repairedAt = getRepairedAt(options, isForcedRepair);
         registerParentRepairSession(parentRepairSession, coordinator, columnFamilyStores, options.getRanges(), options.isIncremental(), repairedAt, options.isGlobal(), options.getPreviewKind());
-        final CountDownLatch prepareLatch = new CountDownLatch(endpoints.size());
+        final CountDownLatch prepareLatch = newCountDownLatch(endpoints.size());
         final AtomicBoolean status = new AtomicBoolean(true);
-        final Set<String> failedNodes = Collections.synchronizedSet(new HashSet<String>());
+        final Set<String> failedNodes = synchronizedSet(new HashSet<String>());
         RequestCallback callback = new RequestCallback()
         {
             @Override
             public void onResponse(Message msg)
             {
-                prepareLatch.countDown();
+                prepareLatch.decrement();
             }
 
             @Override
@@ -557,7 +564,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
             {
                 status.set(false);
                 failedNodes.add(from.toString());
-                prepareLatch.countDown();
+                prepareLatch.decrement();
             }
 
             @Override
@@ -576,7 +583,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
             if (FailureDetector.instance.isAlive(neighbour))
             {
                 PrepareMessage message = new PrepareMessage(parentRepairSession, tableIds, options.getRanges(), options.isIncremental(), repairedAt, options.isGlobal(), options.getPreviewKind());
-                Message<RepairMessage> msg = Message.out(PREPARE_MSG, message);
+                Message<RepairMessage> msg = out(PREPARE_MSG, message);
                 MessagingService.instance().sendWithCallback(msg, neighbour, callback);
             }
             else
@@ -585,7 +592,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
                 // remaining ones go down, we still want to fail so we don't create repair sessions that can't complete
                 if (isForcedRepair && !options.isIncremental())
                 {
-                    prepareLatch.countDown();
+                    prepareLatch.decrement();
                 }
                 else
                 {
@@ -597,7 +604,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
         }
         try
         {
-            if (!prepareLatch.await(DatabaseDescriptor.getRpcTimeout(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS))
+            if (!prepareLatch.await(getRpcTimeout(MILLISECONDS), MILLISECONDS))
                 failRepair(parentRepairSession, "Did not get replies from all endpoints.");
         }
         catch (InterruptedException e)
diff --git a/src/java/org/apache/cassandra/service/EchoVerbHandler.java b/src/java/org/apache/cassandra/service/EchoVerbHandler.java
index 77fe4ab..228808d 100644
--- a/src/java/org/apache/cassandra/service/EchoVerbHandler.java
+++ b/src/java/org/apache/cassandra/service/EchoVerbHandler.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.service;
  * under the License.
  *
  */
-import org.apache.cassandra.exceptions.RequestFailureReason;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
diff --git a/src/java/org/apache/cassandra/service/GCInspector.java b/src/java/org/apache/cassandra/service/GCInspector.java
index 55f1a2e..ad4c2bb 100644
--- a/src/java/org/apache/cassandra/service/GCInspector.java
+++ b/src/java/org/apache/cassandra/service/GCInspector.java
@@ -28,11 +28,8 @@ import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
-import javax.management.InstanceAlreadyExistsException;
-import javax.management.MBeanRegistrationException;
 import javax.management.MBeanServer;
 import javax.management.MalformedObjectNameException;
-import javax.management.NotCompliantMBeanException;
 import javax.management.Notification;
 import javax.management.NotificationListener;
 import javax.management.ObjectName;
diff --git a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java
index 549dc38..a75e75e 100644
--- a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java
+++ b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java
@@ -34,7 +34,9 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
+import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
 public class PendingRangeCalculatorService
 {
@@ -44,8 +46,8 @@ public class PendingRangeCalculatorService
 
     // the executor will only run a single range calculation at a time while keeping at most one task queued in order
     // to trigger an update only after the most recent state change and not for each update individually
-    private final JMXEnabledThreadPoolExecutor executor = new JMXEnabledThreadPoolExecutor(1, Integer.MAX_VALUE, TimeUnit.SECONDS,
-            new LinkedBlockingQueue<>(1), new NamedThreadFactory("PendingRangeCalculator"), "internal");
+    private final JMXEnabledThreadPoolExecutor executor = new JMXEnabledThreadPoolExecutor(1, Integer.MAX_VALUE, SECONDS,
+                                                                                           newBlockingQueue(1), new NamedThreadFactory("PendingRangeCalculator"), "internal");
 
     private AtomicInteger updateJobs = new AtomicInteger(0);
 
diff --git a/src/java/org/apache/cassandra/service/QueryState.java b/src/java/org/apache/cassandra/service/QueryState.java
index adb13b5..b2275f2 100644
--- a/src/java/org/apache/cassandra/service/QueryState.java
+++ b/src/java/org/apache/cassandra/service/QueryState.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.service;
 
 import java.net.InetAddress;
 
-import org.apache.cassandra.transport.ClientStat;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
diff --git a/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java b/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java
index 1309d6e..850c982 100644
--- a/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java
+++ b/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java
@@ -17,16 +17,11 @@
  */
 package org.apache.cassandra.service;
 
-import java.util.concurrent.Executor;
-import java.util.concurrent.Executors;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.SnapshotCommand;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index 0f29875..c02a77d 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -31,7 +31,6 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Future;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
@@ -45,7 +44,8 @@ import com.google.common.cache.CacheLoader;
 import com.google.common.collect.Iterables;
 import com.google.common.primitives.Ints;
 import com.google.common.util.concurrent.Uninterruptibles;
-import org.apache.commons.lang3.StringUtils;
+import org.apache.cassandra.utils.concurrent.CountDownLatch;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -134,21 +134,21 @@ import org.apache.cassandra.utils.MBeanWrapper;
 import org.apache.cassandra.utils.MonotonicClock;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.UUIDGen;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
+import static com.google.common.collect.Iterables.concat;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.cassandra.net.Message.out;
 import static org.apache.cassandra.net.NoPayload.noPayload;
-import static org.apache.cassandra.net.Verb.BATCH_STORE_REQ;
-import static org.apache.cassandra.net.Verb.MUTATION_REQ;
-import static org.apache.cassandra.net.Verb.PAXOS_COMMIT_REQ;
-import static org.apache.cassandra.net.Verb.PAXOS_PREPARE_REQ;
-import static org.apache.cassandra.net.Verb.PAXOS_PROPOSE_REQ;
-import static org.apache.cassandra.net.Verb.TRUNCATE_REQ;
+import static org.apache.cassandra.net.Verb.*;
 import static org.apache.cassandra.service.BatchlogResponseHandler.BatchlogCleanup;
 import static org.apache.cassandra.service.paxos.PrepareVerbHandler.doPrepare;
 import static org.apache.cassandra.service.paxos.ProposeVerbHandler.doPropose;
 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.CountDownLatch.newCountDownLatch;
+import static org.apache.commons.lang3.StringUtils.join;
 
 public class StorageProxy implements StorageProxyMBean
 {
@@ -2093,16 +2093,16 @@ public class StorageProxy implements StorageProxyMBean
         final String myVersion = Schema.instance.getVersion().toString();
         final Map<InetAddressAndPort, UUID> versions = new ConcurrentHashMap<>();
         final Set<InetAddressAndPort> liveHosts = Gossiper.instance.getLiveMembers();
-        final CountDownLatch latch = new CountDownLatch(liveHosts.size());
+        final CountDownLatch latch = newCountDownLatch(liveHosts.size());
 
         RequestCallback<UUID> cb = message ->
         {
             // record the response from the remote node.
             versions.put(message.from(), message.payload);
-            latch.countDown();
+            latch.decrement();
         };
         // an empty message acts as a request to the SchemaVersionVerbHandler.
-        Message message = Message.out(Verb.SCHEMA_VERSION_REQ, noPayload);
+        Message message = out(SCHEMA_VERSION_REQ, noPayload);
         for (InetAddressAndPort endpoint : liveHosts)
             MessagingService.instance().sendWithCallback(message, endpoint, cb);
 
@@ -2111,14 +2111,14 @@ public class StorageProxy implements StorageProxyMBean
             // wait for as long as possible. timeout-1s if possible.
             latch.await(DatabaseDescriptor.getRpcTimeout(NANOSECONDS), NANOSECONDS);
         }
-        catch (InterruptedException ex)
+        catch (InterruptedException e)
         {
-            throw new AssertionError("This latch shouldn't have been interrupted.");
+            throw new UncheckedInterruptedException(e);
         }
 
         // maps versions to hosts that are on that version.
         Map<String, List<String>> results = new HashMap<String, List<String>>();
-        Iterable<InetAddressAndPort> allHosts = Iterables.concat(Gossiper.instance.getLiveMembers(), Gossiper.instance.getUnreachableMembers());
+        Iterable<InetAddressAndPort> allHosts = concat(Gossiper.instance.getLiveMembers(), Gossiper.instance.getUnreachableMembers());
         for (InetAddressAndPort host : allHosts)
         {
             UUID version = versions.get(host);
@@ -2134,7 +2134,7 @@ public class StorageProxy implements StorageProxyMBean
 
         // we're done: the results map is ready to return to the client.  the rest is just debug logging:
         if (results.get(UNREACHABLE) != null)
-            logger.debug("Hosts not in agreement. Didn't get a response from everybody: {}", StringUtils.join(results.get(UNREACHABLE), ","));
+            logger.debug("Hosts not in agreement. Didn't get a response from everybody: {}", join(results.get(UNREACHABLE), ","));
         for (Map.Entry<String, List<String>> entry : results.entrySet())
         {
             // check for version disagreement. log the hosts that don't agree.
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index cbbf9df..54e5f18 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -110,6 +110,7 @@ import org.apache.cassandra.tracing.TraceKeyspace;
 import org.apache.cassandra.transport.ClientResourceLimits;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 import org.apache.cassandra.utils.logging.LoggingSupportFactory;
 import org.apache.cassandra.utils.progress.ProgressEvent;
 import org.apache.cassandra.utils.progress.ProgressEventType;
@@ -1360,7 +1361,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
         catch (InterruptedException e)
         {
-            throw new RuntimeException("Interrupted while waiting on rebuild streaming");
+            throw new UncheckedInterruptedException(e);
         }
         catch (ExecutionException e)
         {
@@ -1699,7 +1700,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 }
                 catch (InterruptedException e)
                 {
-                    throw new AssertionError(e);
+                    throw new UncheckedInterruptedException(e);
                 }
 
                 // check for operator errors...
@@ -1727,7 +1728,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 }
                 catch (InterruptedException e)
                 {
-                    throw new AssertionError(e);
+                    throw new UncheckedInterruptedException(e);
                 }
 
             }
@@ -4564,7 +4565,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
         catch (InterruptedException e)
         {
-            throw new RuntimeException("Node interrupted while decommissioning");
+            throw new UncheckedInterruptedException(e);
         }
         catch (ExecutionException e)
         {
@@ -4728,7 +4729,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             {
                 relocator.stream().get();
             }
-            catch (ExecutionException | InterruptedException e)
+            catch (InterruptedException e)
+            {
+                throw new UncheckedInterruptedException(e);
+            }
+            catch (ExecutionException e)
             {
                 throw new RuntimeException("Interrupted while waiting for stream/fetch ranges to finish: " + e.getMessage());
             }
diff --git a/src/java/org/apache/cassandra/service/TruncateResponseHandler.java b/src/java/org/apache/cassandra/service/TruncateResponseHandler.java
index f6c1506..9976376 100644
--- a/src/java/org/apache/cassandra/service/TruncateResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/TruncateResponseHandler.java
@@ -21,26 +21,28 @@ import java.net.InetAddress;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.TruncateResponse;
 import org.apache.cassandra.exceptions.RequestFailureReason;
 import org.apache.cassandra.exceptions.TruncateException;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.RequestCallback;
 import org.apache.cassandra.net.Message;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
+
 
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static org.apache.cassandra.config.DatabaseDescriptor.getTruncateRpcTimeout;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
 
 public class TruncateResponseHandler implements RequestCallback<TruncateResponse>
 {
     protected static final Logger logger = LoggerFactory.getLogger(TruncateResponseHandler.class);
-    protected final SimpleCondition condition = new SimpleCondition();
+    protected final Condition condition = newOneTimeCondition();
     private final int responseCount;
     protected final AtomicInteger responses = new AtomicInteger(0);
     private final long start;
@@ -64,9 +66,9 @@ public class TruncateResponseHandler implements RequestCallback<TruncateResponse
         {
             completedInTime = condition.await(timeoutNanos, NANOSECONDS); // TODO truncate needs a much longer timeout
         }
-        catch (InterruptedException ex)
+        catch (InterruptedException e)
         {
-            throw new AssertionError(ex);
+            throw new UncheckedInterruptedException(e);
         }
 
         if (!completedInTime)
diff --git a/src/java/org/apache/cassandra/service/pager/PagingState.java b/src/java/org/apache/cassandra/service/pager/PagingState.java
index 4dde936..528d8a8 100644
--- a/src/java/org/apache/cassandra/service/pager/PagingState.java
+++ b/src/java/org/apache/cassandra/service/pager/PagingState.java
@@ -26,7 +26,6 @@ import com.google.common.primitives.Ints;
 
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.ByteArrayAccessor;
 import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.CompositeType;
diff --git a/src/java/org/apache/cassandra/service/paxos/AbstractPaxosCallback.java b/src/java/org/apache/cassandra/service/paxos/AbstractPaxosCallback.java
index 5624df4..54cdb1f 100644
--- a/src/java/org/apache/cassandra/service/paxos/AbstractPaxosCallback.java
+++ b/src/java/org/apache/cassandra/service/paxos/AbstractPaxosCallback.java
@@ -20,16 +20,18 @@
  */
 package org.apache.cassandra.service.paxos;
 
-import java.util.concurrent.CountDownLatch;
+import org.apache.cassandra.utils.concurrent.CountDownLatch;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.WriteType;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.net.RequestCallback;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownLatch;
 
 public abstract class AbstractPaxosCallback<T> implements RequestCallback<T>
 {
@@ -42,13 +44,13 @@ public abstract class AbstractPaxosCallback<T> implements RequestCallback<T>
     {
         this.targets = targets;
         this.consistency = consistency;
-        latch = new CountDownLatch(targets);
+        latch = newCountDownLatch(targets);
         this.queryStartNanoTime = queryStartNanoTime;
     }
 
     public int getResponseCount()
     {
-        return (int) (targets - latch.getCount());
+        return (int) (targets - latch.count());
     }
 
     public void await() throws WriteTimeoutException
@@ -59,9 +61,9 @@ public abstract class AbstractPaxosCallback<T> implements RequestCallback<T>
             if (!latch.await(timeout, NANOSECONDS))
                 throw new WriteTimeoutException(WriteType.CAS, consistency, getResponseCount(), targets);
         }
-        catch (InterruptedException ex)
+        catch (InterruptedException e)
         {
-            throw new AssertionError("This latch shouldn't have been interrupted.");
+            throw new UncheckedInterruptedException(e);
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java b/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
index 93941e9..72a93a7 100644
--- a/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
+++ b/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
@@ -70,8 +70,8 @@ public class PrepareCallback extends AbstractPaxosCallback<PrepareResponse>
         if (!response.promised)
         {
             promised = false;
-            while (latch.getCount() > 0)
-                latch.countDown();
+            while (latch.count() > 0)
+                latch.decrement();
             return;
         }
 
@@ -79,7 +79,7 @@ public class PrepareCallback extends AbstractPaxosCallback<PrepareResponse>
         if (response.mostRecentCommit.isAfter(mostRecentCommit))
             mostRecentCommit = response.mostRecentCommit;
 
-        latch.countDown();
+        latch.decrement();
     }
 
     public Iterable<InetAddressAndPort> replicasMissingMostRecentCommit(TableMetadata metadata, int nowInSec)
diff --git a/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java b/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java
index 7e755a0..dc2f9a7 100644
--- a/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java
+++ b/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java
@@ -64,12 +64,12 @@ public class ProposeCallback extends AbstractPaxosCallback<Boolean>
         if (msg.payload)
             accepts.incrementAndGet();
 
-        latch.countDown();
+        latch.decrement();
 
-        if (isSuccessful() || (failFast && (latch.getCount() + accepts.get() < requiredAccepts)))
+        if (isSuccessful() || (failFast && (latch.count() + accepts.get() < requiredAccepts)))
         {
-            while (latch.getCount() > 0)
-                latch.countDown();
+            while (latch.count() > 0)
+                latch.decrement();
         }
     }
 
@@ -88,6 +88,6 @@ public class ProposeCallback extends AbstractPaxosCallback<Boolean>
     {
         // We need to check the latch first to avoid racing with a late arrival
         // between the latch check and the accepts one
-        return latch.getCount() == 0 && accepts.get() == 0;
+        return latch.count() == 0 && accepts.get() == 0;
     }
 }
diff --git a/src/java/org/apache/cassandra/service/reads/HybridSpeculativeRetryPolicy.java b/src/java/org/apache/cassandra/service/reads/HybridSpeculativeRetryPolicy.java
index 8228c45..edf2b91 100644
--- a/src/java/org/apache/cassandra/service/reads/HybridSpeculativeRetryPolicy.java
+++ b/src/java/org/apache/cassandra/service/reads/HybridSpeculativeRetryPolicy.java
@@ -23,7 +23,6 @@ import java.util.regex.Pattern;
 import com.google.common.base.Objects;
 
 import com.codahale.metrics.Snapshot;
-import com.codahale.metrics.Timer;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.TableParams;
 
diff --git a/src/java/org/apache/cassandra/service/reads/ReadCallback.java b/src/java/org/apache/cassandra/service/reads/ReadCallback.java
index 4a83677..8026162 100644
--- a/src/java/org/apache/cassandra/service/reads/ReadCallback.java
+++ b/src/java/org/apache/cassandra/service/reads/ReadCallback.java
@@ -23,11 +23,13 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.MessageParams;
+import org.apache.cassandra.locator.ReplicaPlan;
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.MessageParams;
 import org.apache.cassandra.db.PartitionRangeReadCommand;
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.db.ReadResponse;
@@ -36,7 +38,6 @@ import org.apache.cassandra.exceptions.ReadTimeoutException;
 import org.apache.cassandra.exceptions.RequestFailureReason;
 import org.apache.cassandra.locator.Endpoints;
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.locator.ReplicaPlan;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.ParamType;
 import org.apache.cassandra.net.RequestCallback;
@@ -45,17 +46,21 @@ import org.apache.cassandra.service.reads.trackwarnings.CoordinatorWarnings;
 import org.apache.cassandra.service.reads.trackwarnings.WarningContext;
 import org.apache.cassandra.service.reads.trackwarnings.WarningsSnapshot;
 import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static java.util.concurrent.atomic.AtomicIntegerFieldUpdater.newUpdater;
+import static org.apache.cassandra.tracing.Tracing.isTracing;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
 
 public class ReadCallback<E extends Endpoints<E>, P extends ReplicaPlan.ForRead<E>> implements RequestCallback<ReadResponse>
 {
-    protected static final Logger logger = LoggerFactory.getLogger( ReadCallback.class );
+    protected static final Logger logger = LoggerFactory.getLogger(ReadCallback.class);
 
     public final ResponseResolver<E, P> resolver;
-    final SimpleCondition condition = new SimpleCondition();
+    final Condition condition = newOneTimeCondition();
     private final long queryStartNanoTime;
     final int blockFor; // TODO: move to replica plan as well?
     // this uses a plain reference, but is initialised before handoff to any other threads; the later updates
@@ -63,7 +68,7 @@ public class ReadCallback<E extends Endpoints<E>, P extends ReplicaPlan.ForRead<
     final ReplicaPlan.Shared<E, P> replicaPlan;
     private final ReadCommand command;
     private static final AtomicIntegerFieldUpdater<ReadCallback> failuresUpdater
-            = AtomicIntegerFieldUpdater.newUpdater(ReadCallback.class, "failures");
+            = newUpdater(ReadCallback.class, "failures");
     private volatile int failures = 0;
     private final Map<InetAddressAndPort, RequestFailureReason> failureReasonByEndpoint;
     private volatile WarningContext warningContext;
@@ -95,11 +100,11 @@ public class ReadCallback<E extends Endpoints<E>, P extends ReplicaPlan.ForRead<
         long time = unit.toNanos(timePastStart) - (nanoTime() - queryStartNanoTime);
         try
         {
-            return condition.await(time, TimeUnit.NANOSECONDS);
+            return condition.await(time, NANOSECONDS);
         }
-        catch (InterruptedException ex)
+        catch (InterruptedException e)
         {
-            throw new AssertionError(ex);
+            throw new UncheckedInterruptedException(e);
         }
     }
 
@@ -130,7 +135,7 @@ public class ReadCallback<E extends Endpoints<E>, P extends ReplicaPlan.ForRead<
         if (signaled && !failed)
             return;
 
-        if (Tracing.isTracing())
+        if (isTracing())
         {
             String gotData = received > 0 ? (resolver.isDataPresent() ? " (including data)" : " (only digests)") : "";
             Tracing.trace("{}; received {} of {} responses{}", failed ? "Failed" : "Timed out", received, blockFor, gotData);
@@ -204,7 +209,6 @@ public class ReadCallback<E extends Endpoints<E>, P extends ReplicaPlan.ForRead<
         onResponse(message);
     }
 
-
     @Override
     public boolean trackLatencyForSnitch()
     {
diff --git a/src/java/org/apache/cassandra/service/reads/ShortReadProtection.java b/src/java/org/apache/cassandra/service/reads/ShortReadProtection.java
index a1bdc0e..8bf877f 100644
--- a/src/java/org/apache/cassandra/service/reads/ShortReadProtection.java
+++ b/src/java/org/apache/cassandra/service/reads/ShortReadProtection.java
@@ -18,15 +18,12 @@
 
 package org.apache.cassandra.service.reads;
 
-import java.net.InetAddress;
-
 
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.db.filter.DataLimits;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 import org.apache.cassandra.db.transform.MorePartitions;
 import org.apache.cassandra.db.transform.Transformation;
-import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.Replica;
 
 /**
diff --git a/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java b/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java
index 194ad5b..df42f58 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java
@@ -21,7 +21,9 @@ package org.apache.cassandra.service.reads.repair;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
+
+import org.apache.cassandra.utils.concurrent.AsyncFuture;
+import org.apache.cassandra.utils.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
 
@@ -30,7 +32,6 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Predicates;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.AbstractFuture;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.ConsistencyLevel;
@@ -38,7 +39,6 @@ import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
-import org.apache.cassandra.locator.Endpoints;
 import org.apache.cassandra.locator.EndpointsForToken;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.Replica;
@@ -51,12 +51,14 @@ import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 import static org.apache.cassandra.net.Verb.*;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownLatch;
 
 public class BlockingPartitionRepair
-        extends AbstractFuture<Object> implements RequestCallback<Object>
+        extends AsyncFuture<Object> implements RequestCallback<Object>
 {
     private final DecoratedKey key;
     private final ReplicaPlan.ForTokenWrite writePlan;
@@ -94,7 +96,7 @@ public class BlockingPartitionRepair
         // empty mutations. If we'd also speculated on either of the read stages, the number
         // of empty mutations would be greater than blockFor, causing the latch ctor to throw
         // an illegal argument exception due to a negative start value. So here we clamp it 0
-        latch = new CountDownLatch(Math.max(blockFor, 0));
+        latch = newCountDownLatch(Math.max(blockFor, 0));
     }
 
     int blockFor()
@@ -105,7 +107,7 @@ public class BlockingPartitionRepair
     @VisibleForTesting
     int waitingOn()
     {
-        return (int) latch.getCount();
+        return (int) latch.count();
     }
 
     @VisibleForTesting
@@ -114,7 +116,7 @@ public class BlockingPartitionRepair
         if (shouldBlockOn.test(from))
         {
             pendingRepairs.remove(writePlan.lookup(from));
-            latch.countDown();
+            latch.decrement();
         }
     }
 
@@ -185,7 +187,7 @@ public class BlockingPartitionRepair
         }
         catch (InterruptedException e)
         {
-            throw new AssertionError(e);
+            throw new UncheckedInterruptedException(e);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/service/reads/repair/PartitionIteratorMergeListener.java b/src/java/org/apache/cassandra/service/reads/repair/PartitionIteratorMergeListener.java
index 7247704..9992bd5 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/PartitionIteratorMergeListener.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/PartitionIteratorMergeListener.java
@@ -21,7 +21,6 @@ package org.apache.cassandra.service.reads.repair;
 import java.util.List;
 
 import org.apache.cassandra.db.Columns;
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.db.RegularAndStaticColumns;
diff --git a/src/java/org/apache/cassandra/service/reads/repair/ReadRepair.java b/src/java/org/apache/cassandra/service/reads/repair/ReadRepair.java
index 4747651..9dcb969 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/ReadRepair.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/ReadRepair.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.service.reads.repair;
 
 import java.util.Map;
 import java.util.function.Consumer;
-import java.util.function.Supplier;
 
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.locator.Endpoints;
diff --git a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairDiagnostics.java b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairDiagnostics.java
index b9167bd..b883a88 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairDiagnostics.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairDiagnostics.java
@@ -25,7 +25,6 @@ import com.google.common.collect.Lists;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.diag.DiagnosticEventService;
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.locator.ReplicaPlan;
 import org.apache.cassandra.service.reads.DigestResolver;
 import org.apache.cassandra.service.reads.repair.PartitionRepairEvent.PartitionRepairEventType;
diff --git a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairEvent.java b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairEvent.java
index 5cec802..65c0215 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairEvent.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairEvent.java
@@ -22,7 +22,6 @@ import java.io.Serializable;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
diff --git a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairStrategy.java b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairStrategy.java
index 7a4b795..8e0c1b5 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairStrategy.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairStrategy.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.service.reads.repair;
 
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.locator.Endpoints;
-import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.locator.ReplicaPlan;
 
 public enum ReadRepairStrategy implements ReadRepair.Factory
diff --git a/src/java/org/apache/cassandra/service/reads/repair/RepairedDataVerifier.java b/src/java/org/apache/cassandra/service/reads/repair/RepairedDataVerifier.java
index d1cff11..c5a0f95 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/RepairedDataVerifier.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/RepairedDataVerifier.java
@@ -18,11 +18,7 @@
 
 package org.apache.cassandra.service.reads.repair;
 
-import java.time.LocalDate;
-import java.time.format.DateTimeFormatter;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -30,14 +26,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.ReadCommand;
-import org.apache.cassandra.db.SnapshotCommand;
-import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.TableMetrics;
-import org.apache.cassandra.net.Message;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.net.Verb;
-import org.apache.cassandra.schema.TableId;
-import org.apache.cassandra.service.SnapshotVerbHandler;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.DiagnosticSnapshotService;
 import org.apache.cassandra.utils.NoSpamLogger;
diff --git a/src/java/org/apache/cassandra/streaming/SessionSummary.java b/src/java/org/apache/cassandra/streaming/SessionSummary.java
index 5b168a0..65ecd32 100644
--- a/src/java/org/apache/cassandra/streaming/SessionSummary.java
+++ b/src/java/org/apache/cassandra/streaming/SessionSummary.java
@@ -28,7 +28,6 @@ import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.locator.InetAddressAndPort.Serializer;
 
 import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer;
 
diff --git a/src/java/org/apache/cassandra/streaming/StreamManager.java b/src/java/org/apache/cassandra/streaming/StreamManager.java
index c8c091d..7d8f143 100644
--- a/src/java/org/apache/cassandra/streaming/StreamManager.java
+++ b/src/java/org/apache/cassandra/streaming/StreamManager.java
@@ -30,7 +30,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.RateLimiter;
 
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
@@ -146,13 +145,7 @@ public class StreamManager implements StreamManagerMBean
     {
         result.addEventListener(notifier);
         // Make sure we remove the stream on completion (whether successful or not)
-        result.addListener(new Runnable()
-        {
-            public void run()
-            {
-                initiatorStreams.remove(result.planId);
-            }
-        }, MoreExecutors.directExecutor());
+        result.addListener(() -> initiatorStreams.remove(result.planId));
 
         initiatorStreams.put(result.planId, result);
     }
@@ -161,13 +154,7 @@ public class StreamManager implements StreamManagerMBean
     {
         result.addEventListener(notifier);
         // Make sure we remove the stream on completion (whether successful or not)
-        result.addListener(new Runnable()
-        {
-            public void run()
-            {
-                followerStreams.remove(result.planId);
-            }
-        }, MoreExecutors.directExecutor());
+        result.addListener(() -> followerStreams.remove(result.planId));
 
         StreamResultFuture previous = followerStreams.putIfAbsent(result.planId, result);
         return previous ==  null ? result : previous;
diff --git a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
index 89a6cf1..2f9ca8b 100644
--- a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
+++ b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
@@ -21,9 +21,9 @@ import java.util.Collection;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentLinkedQueue;
 
-import com.google.common.util.concurrent.AbstractFuture;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.cassandra.utils.concurrent.AsyncFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,7 +44,7 @@ import org.apache.cassandra.utils.FBUtilities;
  * You can attach {@link StreamEventHandler} to this object to listen on {@link StreamEvent}s to
  * track progress of the streaming.
  */
-public final class StreamResultFuture extends AbstractFuture<StreamState>
+public final class StreamResultFuture extends AsyncFuture<StreamState>
 {
     private static final Logger logger = LoggerFactory.getLogger(StreamResultFuture.class);
 
@@ -69,7 +69,7 @@ public final class StreamResultFuture extends AbstractFuture<StreamState>
 
         // if there is no session to listen to, we immediately set result for returning
         if (!coordinator.isFollower() && !coordinator.hasActiveSessions())
-            set(getCurrentState());
+            trySuccess(getCurrentState());
     }
 
     private StreamResultFuture(UUID planId, StreamOperation streamOperation, UUID pendingRepair, PreviewKind previewKind)
@@ -217,12 +217,12 @@ public final class StreamResultFuture extends AbstractFuture<StreamState>
             if (finalState.hasFailedSession())
             {
                 logger.warn("[Stream #{}] Stream failed", planId);
-                setException(new StreamException(finalState, "Stream failed"));
+                tryFailure(new StreamException(finalState, "Stream failed"));
             }
             else
             {
                 logger.info("[Stream #{}] All sessions completed", planId);
-                set(finalState);
+                trySuccess(finalState);
             }
         }
     }
diff --git a/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java b/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java
index 96e7626..9562981 100644
--- a/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java
+++ b/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java
@@ -20,9 +20,6 @@ package org.apache.cassandra.streaming;
 
 import java.io.IOException;
 
-import com.google.common.annotations.VisibleForTesting;
-
-import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.messages.StreamMessage;
 
 public interface StreamingMessageSender
diff --git a/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java b/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java
index a63db22..f80c84c 100644
--- a/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java
+++ b/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java
@@ -25,14 +25,12 @@ import java.nio.channels.ClosedByInterruptException;
 import java.util.Collection;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.Semaphore;
 import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Throwables;
+
+import org.apache.cassandra.utils.concurrent.Semaphore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,11 +43,8 @@ import io.netty.util.concurrent.Future;
 import io.netty.util.concurrent.GenericFutureListener;
 import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.util.DataOutputBufferFixed;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.net.AsyncChannelPromise;
 import org.apache.cassandra.net.OutboundConnectionSettings;
 import org.apache.cassandra.net.AsyncStreamingOutputPlus;
 import org.apache.cassandra.streaming.StreamConnectionFactory;
@@ -60,8 +55,25 @@ import org.apache.cassandra.streaming.messages.KeepAliveMessage;
 import org.apache.cassandra.streaming.messages.OutgoingStreamMessage;
 import org.apache.cassandra.streaming.messages.StreamInitMessage;
 import org.apache.cassandra.streaming.messages.StreamMessage;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static com.google.common.base.Throwables.getRootCause;
+import static io.netty.util.AttributeKey.valueOf;
+import static java.lang.Boolean.FALSE;
+import static java.lang.Integer.parseInt;
+import static java.lang.String.format;
+import static java.lang.System.getProperty;
+import static java.lang.Thread.currentThread;
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.cassandra.config.Config.PROPERTY_PREFIX;
+import static org.apache.cassandra.config.DatabaseDescriptor.getStreamingKeepAlivePeriod;
+import static org.apache.cassandra.net.AsyncChannelPromise.writeAndFlush;
+import static org.apache.cassandra.streaming.messages.StreamMessage.serialize;
+import static org.apache.cassandra.streaming.messages.StreamMessage.serializedSize;
+import static org.apache.cassandra.utils.FBUtilities.getAvailableProcessors;
+import static org.apache.cassandra.utils.FBUtilities.getBroadcastAddressAndPort;
+import static org.apache.cassandra.utils.JVMStabilityInspector.inspectThrowable;
+import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
+import static org.apache.cassandra.utils.concurrent.Semaphore.newFairSemaphore;
 
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
 
@@ -85,13 +97,13 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
 {
     private static final Logger logger = LoggerFactory.getLogger(NettyStreamingMessageSender.class);
 
-    private static final int DEFAULT_MAX_PARALLEL_TRANSFERS = FBUtilities.getAvailableProcessors();
-    private static final int MAX_PARALLEL_TRANSFERS = Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "streaming.session.parallelTransfers", Integer.toString(DEFAULT_MAX_PARALLEL_TRANSFERS)));
+    private static final int DEFAULT_MAX_PARALLEL_TRANSFERS = getAvailableProcessors();
+    private static final int MAX_PARALLEL_TRANSFERS = parseInt(getProperty(PROPERTY_PREFIX + "streaming.session.parallelTransfers", Integer.toString(DEFAULT_MAX_PARALLEL_TRANSFERS)));
 
-    private static final long DEFAULT_CLOSE_WAIT_IN_MILLIS = TimeUnit.MINUTES.toMillis(5);
+    private static final long DEFAULT_CLOSE_WAIT_IN_MILLIS = MINUTES.toMillis(5);
 
-    // a simple mechansim for allowing a degree of fairnes across multiple sessions
-    private static final Semaphore fileTransferSemaphore = new Semaphore(DEFAULT_MAX_PARALLEL_TRANSFERS, true);
+    // a simple mechansim for allowing a degree of fairness across multiple sessions
+    private static final Semaphore fileTransferSemaphore = newFairSemaphore(DEFAULT_MAX_PARALLEL_TRANSFERS);
 
     private final StreamSession session;
     private final boolean isPreview;
@@ -108,7 +120,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
     private volatile Channel controlMessageChannel;
 
     // note: this really doesn't need to be a LBQ, just something that's thread safe
-    private final Collection<ScheduledFuture<?>> channelKeepAlives = new LinkedBlockingQueue<>();
+    private final Collection<ScheduledFuture<?>> channelKeepAlives = newBlockingQueue();
 
     private final ThreadPoolExecutor fileTransferExecutor;
 
@@ -123,7 +135,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
      * (application level) keep-alive in the middle of a stream would be bad news.
      */
     @VisibleForTesting
-    static final AttributeKey<Boolean> TRANSFERRING_FILE_ATTR = AttributeKey.valueOf("transferringFile");
+    static final AttributeKey<Boolean> TRANSFERRING_FILE_ATTR = valueOf("transferringFile");
 
     public NettyStreamingMessageSender(StreamSession session, OutboundConnectionSettings template, StreamConnectionFactory factory, int streamingVersion, boolean isPreview)
     {
@@ -134,7 +146,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
         this.isPreview = isPreview;
 
         String name = session.peer.toString().replace(':', '.');
-        fileTransferExecutor = new DebuggableThreadPoolExecutor(1, MAX_PARALLEL_TRANSFERS, 1L, TimeUnit.SECONDS, new LinkedBlockingQueue<>(),
+        fileTransferExecutor = new DebuggableThreadPoolExecutor(1, MAX_PARALLEL_TRANSFERS, 1L, SECONDS, newBlockingQueue(),
                                                                 new NamedThreadFactory("NettyStreaming-Outbound-" + name));
         fileTransferExecutor.allowCoreThreadTimeOut(true);
     }
@@ -142,7 +154,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
     @Override
     public void initialize()
     {
-        StreamInitMessage message = new StreamInitMessage(FBUtilities.getBroadcastAddressAndPort(),
+        StreamInitMessage message = new StreamInitMessage(getBroadcastAddressAndPort(),
                                                           session.sessionIndex(),
                                                           session.planId(),
                                                           session.streamOperation(),
@@ -162,7 +174,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
     public void injectControlMessageChannel(Channel channel)
     {
         this.controlMessageChannel = channel;
-        channel.attr(TRANSFERRING_FILE_ATTR).set(Boolean.FALSE);
+        channel.attr(TRANSFERRING_FILE_ATTR).set(FALSE);
         scheduleKeepAliveTask(channel);
     }
 
@@ -186,12 +198,12 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
 
     private void scheduleKeepAliveTask(Channel channel)
     {
-        int keepAlivePeriod = DatabaseDescriptor.getStreamingKeepAlivePeriod();
+        int keepAlivePeriod = getStreamingKeepAlivePeriod();
         if (logger.isDebugEnabled())
             logger.debug("{} Scheduling keep-alive task with {}s period.", createLogTag(session, channel), keepAlivePeriod);
 
         KeepAliveTask task = new KeepAliveTask(channel, session);
-        ScheduledFuture<?> scheduledFuture = channel.eventLoop().scheduleAtFixedRate(task, 0, keepAlivePeriod, TimeUnit.SECONDS);
+        ScheduledFuture<?> scheduledFuture = channel.eventLoop().scheduleAtFixedRate(task, 0, keepAlivePeriod, SECONDS);
         channelKeepAlives.add(scheduledFuture);
         task.future = scheduledFuture;
     }
@@ -206,7 +218,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
             ChannelPipeline pipeline = channel.pipeline();
             pipeline.addLast("stream", new StreamingInboundHandler(template.to, streamingVersion, session));
         }
-        channel.attr(TRANSFERRING_FILE_ATTR).set(Boolean.FALSE);
+        channel.attr(TRANSFERRING_FILE_ATTR).set(FALSE);
         logger.debug("Creating channel id {} local {} remote {}", channel.id(), channel.localAddress(), channel.remoteAddress());
         return channel;
     }
@@ -260,10 +272,10 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
             logger.debug("{} Sending {}", createLogTag(session, channel), message);
 
         // we anticipate that the control messages are rather small, so allocating a ByteBuf shouldn't  blow out of memory.
-        long messageSize = StreamMessage.serializedSize(message, streamingVersion);
+        long messageSize = serializedSize(message, streamingVersion);
         if (messageSize > 1 << 30)
         {
-            throw new IllegalStateException(String.format("%s something is seriously wrong with the calculated stream control message's size: %d bytes, type is %s",
+            throw new IllegalStateException(format("%s something is seriously wrong with the calculated stream control message's size: %d bytes, type is %s",
                                                           createLogTag(session, channel), messageSize, message.type));
         }
 
@@ -272,11 +284,11 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
         ByteBuffer nioBuf = buf.nioBuffer(0, (int) messageSize);
         @SuppressWarnings("resource")
         DataOutputBufferFixed out = new DataOutputBufferFixed(nioBuf);
-        StreamMessage.serialize(message, out, streamingVersion, session);
+        serialize(message, out, streamingVersion, session);
         assert nioBuf.position() == nioBuf.limit();
         buf.writerIndex(nioBuf.position());
 
-        AsyncChannelPromise.writeAndFlush(channel, buf, listener);
+        writeAndFlush(channel, buf, listener);
     }
 
     /**
@@ -346,11 +358,11 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
                 // close the DataOutputStreamPlus as we're done with it - but don't close the channel
                 try (DataOutputStreamPlus outPlus = new AsyncStreamingOutputPlus(channel))
                 {
-                    StreamMessage.serialize(msg, outPlus, streamingVersion, session);
+                    serialize(msg, outPlus, streamingVersion, session);
                 }
                 finally
                 {
-                    channel.attr(TRANSFERRING_FILE_ATTR).set(Boolean.FALSE);
+                    channel.attr(TRANSFERRING_FILE_ATTR).set(FALSE);
                 }
             }
             catch (Exception e)
@@ -359,26 +371,26 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
             }
             catch (Throwable t)
             {
-                if (closed && Throwables.getRootCause(t) instanceof ClosedByInterruptException && fileTransferExecutor.isShutdown())
+                if (closed && getRootCause(t) instanceof ClosedByInterruptException && fileTransferExecutor.isShutdown())
                 {
                     logger.debug("{} Streaming channel was closed due to the executor pool being shutdown", createLogTag(session, channel));
                 }
                 else
                 {
-                    JVMStabilityInspector.inspectThrowable(t);
+                    inspectThrowable(t);
                     if (!session.state().isFinalState())
                         session.onError(t);
                 }
             }
             finally
             {
-                fileTransferSemaphore.release();
+                fileTransferSemaphore.release(1);
             }
         }
 
         boolean acquirePermit(int logInterval)
         {
-            long logIntervalNanos = TimeUnit.MINUTES.toNanos(logInterval);
+            long logIntervalNanos = MINUTES.toNanos(logInterval);
             long timeOfLastLogging = nanoTime();
             while (true)
             {
@@ -386,7 +398,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
                     return false;
                 try
                 {
-                    if (fileTransferSemaphore.tryAcquire(1, TimeUnit.SECONDS))
+                    if (fileTransferSemaphore.tryAcquire(1, 1, SECONDS))
                         return true;
 
                     // log a helpful message to operators in case they are wondering why a given session might not be making progress.
@@ -410,7 +422,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
 
         private Channel getOrCreateChannel()
         {
-            Thread currentThread = Thread.currentThread();
+            Thread currentThread = currentThread();
             try
             {
                 Channel channel = threadToChannelMap.get(currentThread);
@@ -431,7 +443,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
         {
             try
             {
-                session.onError(t).get(DEFAULT_CLOSE_WAIT_IN_MILLIS, TimeUnit.MILLISECONDS);
+                session.onError(t).get(DEFAULT_CLOSE_WAIT_IN_MILLIS, MILLISECONDS);
             }
             catch (Exception e)
             {
@@ -444,7 +456,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
          */
         void injectChannel(Channel channel)
         {
-            Thread currentThread = Thread.currentThread();
+            Thread currentThread = currentThread();
             if (threadToChannelMap.get(currentThread) != null)
                 throw new IllegalStateException("previous channel already set");
 
@@ -456,7 +468,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
          */
         void unsetChannel()
         {
-            threadToChannelMap.remove(Thread.currentThread());
+            threadToChannelMap.remove(currentThread());
         }
     }
 
@@ -535,7 +547,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender
 
     int semaphoreAvailablePermits()
     {
-        return fileTransferSemaphore.availablePermits();
+        return fileTransferSemaphore.permits();
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
index e3f805e..8c6f71b 100644
--- a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
@@ -27,8 +27,6 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.streaming.StreamSession;
 
-import static java.lang.Math.max;
-
 /**
  * StreamMessage is an abstract base class that every messages in streaming protocol inherit.
  *
diff --git a/src/java/org/apache/cassandra/tools/BootstrapMonitor.java b/src/java/org/apache/cassandra/tools/BootstrapMonitor.java
index 67d2925..4d58638 100644
--- a/src/java/org/apache/cassandra/tools/BootstrapMonitor.java
+++ b/src/java/org/apache/cassandra/tools/BootstrapMonitor.java
@@ -20,20 +20,20 @@ package org.apache.cassandra.tools;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.text.SimpleDateFormat;
-import java.util.concurrent.locks.Condition;
 
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.apache.cassandra.utils.progress.ProgressEvent;
 import org.apache.cassandra.utils.progress.ProgressEventType;
 import org.apache.cassandra.utils.progress.jmx.JMXNotificationProgressListener;
 
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
 
 public class BootstrapMonitor extends JMXNotificationProgressListener
 {
     private final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS");
     private final PrintStream out;
-    private final Condition condition = new SimpleCondition();
+    private final Condition condition = newOneTimeCondition();
 
     public BootstrapMonitor(PrintStream out)
     {
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index 02340ba..91a008e 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -50,7 +50,6 @@ import com.google.common.base.Throwables;
 import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 import org.apache.cassandra.tools.nodetool.*;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.tools.nodetool.Sjk;
 
 import com.google.common.collect.Maps;
 
diff --git a/src/java/org/apache/cassandra/tools/RepairRunner.java b/src/java/org/apache/cassandra/tools/RepairRunner.java
index e7ac831..cd09c57 100644
--- a/src/java/org/apache/cassandra/tools/RepairRunner.java
+++ b/src/java/org/apache/cassandra/tools/RepairRunner.java
@@ -22,20 +22,22 @@ import java.io.PrintStream;
 import java.text.SimpleDateFormat;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Condition;
 
-import com.google.common.base.Throwables;
-
-import org.apache.cassandra.repair.messages.RepairOption;
-import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.service.StorageServiceMBean;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
+import org.apache.cassandra.utils.concurrent.Condition;
+
 import org.apache.cassandra.utils.progress.ProgressEvent;
 import org.apache.cassandra.utils.progress.ProgressEventType;
 import org.apache.cassandra.utils.progress.jmx.JMXNotificationProgressListener;
 
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.service.ActiveRepairService.ParentRepairStatus;
+import static org.apache.cassandra.service.ActiveRepairService.ParentRepairStatus.FAILED;
+import static org.apache.cassandra.service.ActiveRepairService.ParentRepairStatus.valueOf;
+import static org.apache.cassandra.tools.NodeProbe.JMX_NOTIFICATION_POLL_INTERVAL_SECONDS;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
+import static org.apache.cassandra.utils.progress.ProgressEventType.*;
 
 public class RepairRunner extends JMXNotificationProgressListener
 {
@@ -45,7 +47,7 @@ public class RepairRunner extends JMXNotificationProgressListener
     private final StorageServiceMBean ssProxy;
     private final String keyspace;
     private final Map<String, String> options;
-    private final SimpleCondition condition = new SimpleCondition();
+    private final Condition condition = newOneTimeCondition();
 
     private int cmd;
     private volatile Exception error;
@@ -69,10 +71,10 @@ public class RepairRunner extends JMXNotificationProgressListener
         }
         else
         {
-            while (!condition.await(NodeProbe.JMX_NOTIFICATION_POLL_INTERVAL_SECONDS, TimeUnit.SECONDS))
+            while (!condition.await(JMX_NOTIFICATION_POLL_INTERVAL_SECONDS, SECONDS))
             {
                 queryForCompletedRepair(String.format("After waiting for poll interval of %s seconds",
-                                                      NodeProbe.JMX_NOTIFICATION_POLL_INTERVAL_SECONDS));
+                                                      JMX_NOTIFICATION_POLL_INTERVAL_SECONDS));
             }
             Exception error = this.error;
             if (error == null)
@@ -125,18 +127,18 @@ public class RepairRunner extends JMXNotificationProgressListener
     {
         ProgressEventType type = event.getType();
         String message = event.getMessage();
-        if (type == ProgressEventType.PROGRESS)
+        if (type == PROGRESS)
         {
             message = message + " (progress: " + (int) event.getProgressPercentage() + "%)";
         }
         printMessage(message);
-        if (type == ProgressEventType.ERROR)
+        if (type == ERROR)
         {
             error = new RuntimeException(String.format("Repair job has failed with the error message: %s. " +
                                                        "Check the logs on the repair participants for further details",
                                                        message));
         }
-        if (type == ProgressEventType.COMPLETE)
+        if (type == COMPLETE)
         {
             condition.signalAll();
         }
@@ -155,7 +157,7 @@ public class RepairRunner extends JMXNotificationProgressListener
         }
         else
         {
-            ActiveRepairService.ParentRepairStatus parentRepairStatus = ActiveRepairService.ParentRepairStatus.valueOf(status.get(0));
+            ParentRepairStatus parentRepairStatus = valueOf(status.get(0));
             List<String> messages = status.subList(1, status.size());
             switch (parentRepairStatus)
             {
@@ -164,7 +166,7 @@ public class RepairRunner extends JMXNotificationProgressListener
                     printMessage(String.format("%s %s discovered repair %s.",
                                               triggeringCondition,
                                               queriedString, parentRepairStatus.name().toLowerCase()));
-                    if (parentRepairStatus == ActiveRepairService.ParentRepairStatus.FAILED)
+                    if (parentRepairStatus == FAILED)
                     {
                         error = new IOException(messages.get(0));
                     }
diff --git a/src/java/org/apache/cassandra/tools/nodetool/TpStats.java b/src/java/org/apache/cassandra/tools/nodetool/TpStats.java
index 5b20b13..1d16d8d 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/TpStats.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/TpStats.java
@@ -22,8 +22,6 @@ import io.airlift.airline.Command;
 import io.airlift.airline.Option;
 import org.apache.cassandra.tools.NodeProbe;
 import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
-import org.apache.cassandra.tools.nodetool.stats.TpStatsHolder;
-import org.apache.cassandra.tools.nodetool.stats.TpStatsPrinter;
 import org.apache.cassandra.tools.nodetool.stats.*;
 
 
diff --git a/src/java/org/apache/cassandra/tracing/TraceState.java b/src/java/org/apache/cassandra/tracing/TraceState.java
index 1e0813c..ae69801 100644
--- a/src/java/org/apache/cassandra/tracing/TraceState.java
+++ b/src/java/org/apache/cassandra/tracing/TraceState.java
@@ -29,6 +29,7 @@ import org.slf4j.helpers.MessageFormatter;
 
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 import org.apache.cassandra.utils.progress.ProgressEvent;
 import org.apache.cassandra.utils.progress.ProgressEventNotifier;
 import org.apache.cassandra.utils.progress.ProgressListener;
@@ -134,7 +135,7 @@ public abstract class TraceState implements ProgressEventNotifier
             }
             catch (InterruptedException e)
             {
-                throw new RuntimeException();
+                throw new UncheckedInterruptedException(e);
             }
         }
         if (status == Status.ACTIVE)
diff --git a/src/java/org/apache/cassandra/transport/Event.java b/src/java/org/apache/cassandra/transport/Event.java
index c62a73f..9515551 100644
--- a/src/java/org/apache/cassandra/transport/Event.java
+++ b/src/java/org/apache/cassandra/transport/Event.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.transport;
 
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.util.Iterator;
 import java.util.List;
diff --git a/src/java/org/apache/cassandra/transport/SimpleClient.java b/src/java/org/apache/cassandra/transport/SimpleClient.java
index ef075ab..0e98a32 100644
--- a/src/java/org/apache/cassandra/transport/SimpleClient.java
+++ b/src/java/org/apache/cassandra/transport/SimpleClient.java
@@ -50,12 +50,14 @@ import org.apache.cassandra.net.*;
 import org.apache.cassandra.security.ISslContextFactory;
 import org.apache.cassandra.security.SSLFactory;
 import org.apache.cassandra.transport.messages.*;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 import static org.apache.cassandra.transport.CQLMessageHandler.envelopeSize;
 import static org.apache.cassandra.transport.Flusher.MAX_FRAMED_PAYLOAD_SIZE;
 import static org.apache.cassandra.utils.concurrent.NonBlockingRateLimiter.NO_OP_LIMITER;
 
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
+import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
 public class SimpleClient implements Closeable
 {
@@ -299,7 +301,7 @@ public class SimpleClient implements Closeable
         }
         catch (InterruptedException e)
         {
-            throw new RuntimeException(e);
+            throw new UncheckedInterruptedException(e);
         }
     }
 
@@ -341,7 +343,7 @@ public class SimpleClient implements Closeable
         }
         catch (InterruptedException e)
         {
-            throw new RuntimeException(e);
+            throw new UncheckedInterruptedException(e);
         }
     }
 
@@ -352,7 +354,7 @@ public class SimpleClient implements Closeable
 
     public static class SimpleEventHandler implements EventHandler
     {
-        public final LinkedBlockingQueue<Event> queue = new LinkedBlockingQueue<>();
+        public final BlockingQueue<Event> queue = newBlockingQueue();
 
         public void onEvent(Event event)
         {
@@ -654,9 +656,9 @@ public class SimpleClient implements Closeable
                 else
                     responses.put(r);
             }
-            catch (InterruptedException ie)
+            catch (InterruptedException e)
             {
-                throw new RuntimeException(ie);
+                throw new UncheckedInterruptedException(e);
             }
         }
 
diff --git a/src/java/org/apache/cassandra/transport/messages/PrepareMessage.java b/src/java/org/apache/cassandra/transport/messages/PrepareMessage.java
index 5bf058c..ec29f7d 100644
--- a/src/java/org/apache/cassandra/transport/messages/PrepareMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/PrepareMessage.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.transport.messages;
 
-import java.util.function.Supplier;
-
 import com.google.common.collect.ImmutableMap;
 
 import io.netty.buffer.ByteBuf;
diff --git a/src/java/org/apache/cassandra/utils/Clock.java b/src/java/org/apache/cassandra/utils/Clock.java
index 9dd6dd4..629a585 100644
--- a/src/java/org/apache/cassandra/utils/Clock.java
+++ b/src/java/org/apache/cassandra/utils/Clock.java
@@ -17,6 +17,8 @@
  */
 package org.apache.cassandra.utils;
 
+import java.util.concurrent.TimeUnit;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -31,10 +33,10 @@ import org.slf4j.LoggerFactory;
  */
 public interface Clock
 {
+    static final Logger logger = LoggerFactory.getLogger(Clock.class);
+
     public static class Global
     {
-        private static final Logger logger = LoggerFactory.getLogger(Clock.class);
-
         /**
          * Static singleton object that will be instantiated by default with a system clock
          * implementation. Set <code>cassandra.clock</code> system property to a FQCN to use a
@@ -107,4 +109,11 @@ public interface Clock
      */
     public long currentTimeMillis();
 
+    @Intercept
+    public static void waitUntil(long deadlineNanos) throws InterruptedException
+    {
+        long waitNanos = Clock.Global.nanoTime() - deadlineNanos;
+        if (waitNanos > 0)
+            TimeUnit.NANOSECONDS.sleep(waitNanos);
+    }
 }
diff --git a/src/java/org/apache/cassandra/utils/DiagnosticSnapshotService.java b/src/java/org/apache/cassandra/utils/DiagnosticSnapshotService.java
index 0c80d06..7572ac1 100644
--- a/src/java/org/apache/cassandra/utils/DiagnosticSnapshotService.java
+++ b/src/java/org/apache/cassandra/utils/DiagnosticSnapshotService.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.utils;
 
-import java.net.InetAddress;
 import java.time.LocalDate;
 import java.time.format.DateTimeFormatter;
 import java.util.concurrent.*;
diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java
index 73aa8f0..62ab370 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -67,6 +67,7 @@ import org.apache.cassandra.io.util.DataOutputBufferFixed;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.security.ISslContextFactory;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 import static org.apache.cassandra.config.CassandraRelevantProperties.LINE_SEPARATOR;
 import static org.apache.cassandra.config.CassandraRelevantProperties.USER_HOME;
@@ -933,7 +934,7 @@ public class FBUtilities
         }
         catch (InterruptedException e)
         {
-            throw new AssertionError(e);
+            throw new UncheckedInterruptedException(e);
         }
     }
 
@@ -1089,7 +1090,7 @@ public class FBUtilities
         }
         catch (InterruptedException e)
         {
-            throw new RuntimeException(e);
+            throw new UncheckedInterruptedException(e);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java b/src/java/org/apache/cassandra/utils/Intercept.java
similarity index 65%
copy from src/java/org/apache/cassandra/streaming/StreamingMessageSender.java
copy to src/java/org/apache/cassandra/utils/Intercept.java
index 96e7626..b81947b 100644
--- a/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java
+++ b/src/java/org/apache/cassandra/utils/Intercept.java
@@ -16,22 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.cassandra.streaming;
+package org.apache.cassandra.utils;
 
-import java.io.IOException;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
 
-import com.google.common.annotations.VisibleForTesting;
-
-import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.streaming.messages.StreamMessage;
-
-public interface StreamingMessageSender
+// a marker to indicate that the method is intercepted by the Simulator
+@Retention(RetentionPolicy.SOURCE)
+@Target({ ElementType.METHOD })
+public @interface Intercept
 {
-    void initialize() throws IOException;
-
-    void sendMessage(StreamMessage message) throws IOException;
-
-    boolean connected();
-
-    void close();
 }
diff --git a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
index 4c0f972..9362cc2 100644
--- a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
+++ b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
@@ -40,6 +40,7 @@ import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 /**
  * Responsible for deciding whether to kill the JVM if it gets in an "unstable" state (think OOM).
@@ -114,6 +115,9 @@ public final class JVMStabilityInspector
             isUnstable = true;
         }
 
+        if (t instanceof InterruptedException)
+            throw new UncheckedInterruptedException((InterruptedException) t);
+
         if (DatabaseDescriptor.getDiskFailurePolicy() == Config.DiskFailurePolicy.die)
             if (t instanceof FSError || t instanceof CorruptSSTableException)
                 isUnstable = true;
diff --git a/src/java/org/apache/cassandra/utils/Mx4jTool.java b/src/java/org/apache/cassandra/utils/Mx4jTool.java
index 054fdcf..e54c0bf 100644
--- a/src/java/org/apache/cassandra/utils/Mx4jTool.java
+++ b/src/java/org/apache/cassandra/utils/Mx4jTool.java
@@ -23,8 +23,6 @@ import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CassandraRelevantProperties;
-
 import static org.apache.cassandra.config.CassandraRelevantProperties.MX4JADDRESS;
 import static org.apache.cassandra.config.CassandraRelevantProperties.MX4JPORT;
 
diff --git a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
index 004bc34..1d48c9e 100644
--- a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
+++ b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
@@ -36,10 +36,6 @@ import org.apache.cassandra.dht.Token.TokenFactory;
 import org.apache.cassandra.io.sstable.SSTableLoader;
 import org.apache.cassandra.schema.TableMetadata;
 
-import org.apache.cassandra.schema.CQLTypeParser;
-import org.apache.cassandra.schema.SchemaKeyspace;
-import org.apache.cassandra.schema.Types;
-
 public class NativeSSTableLoaderClient extends SSTableLoader.Client
 {
     protected final Map<String, TableMetadataRef> tables;
diff --git a/src/java/org/apache/cassandra/utils/SyncUtil.java b/src/java/org/apache/cassandra/utils/SyncUtil.java
index 1917e8b..b4a4bee 100644
--- a/src/java/org/apache/cassandra/utils/SyncUtil.java
+++ b/src/java/org/apache/cassandra/utils/SyncUtil.java
@@ -28,7 +28,6 @@ import java.nio.channels.FileChannel;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.cassandra.config.Config;
-import org.apache.cassandra.service.CassandraDaemon;
 
 import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
diff --git a/src/java/org/apache/cassandra/utils/Throwables.java b/src/java/org/apache/cassandra/utils/Throwables.java
index 86c0156..73e7d24 100644
--- a/src/java/org/apache/cassandra/utils/Throwables.java
+++ b/src/java/org/apache/cassandra/utils/Throwables.java
@@ -31,6 +31,7 @@ import java.util.stream.Stream;
 
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 public final class Throwables
 {
@@ -87,6 +88,9 @@ public final class Throwables
         if (fail instanceof RuntimeException)
             throw (RuntimeException) fail;
 
+        if (fail instanceof InterruptedException)
+            throw new UncheckedInterruptedException((InterruptedException) fail);
+
         if (checked != null && checked.isInstance(fail))
             throw checked.cast(fail);
 
@@ -237,7 +241,10 @@ public final class Throwables
      */
     public static RuntimeException unchecked(Throwable t)
     {
-        return t instanceof RuntimeException ? (RuntimeException)t : new RuntimeException(t);
+        return t instanceof RuntimeException ? (RuntimeException)t :
+               t instanceof InterruptedException
+               ? new UncheckedInterruptedException((InterruptedException) t)
+               : new RuntimeException(t);
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/utils/binlog/BinLog.java b/src/java/org/apache/cassandra/utils/binlog/BinLog.java
index 63f74a3..8b8588a 100644
--- a/src/java/org/apache/cassandra/utils/binlog/BinLog.java
+++ b/src/java/org/apache/cassandra/utils/binlog/BinLog.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.utils.binlog;
 
 import java.io.File;
 import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
@@ -42,12 +41,12 @@ import net.openhft.chronicle.queue.RollCycles;
 import net.openhft.chronicle.wire.WireOut;
 import net.openhft.chronicle.wire.WriteMarshallable;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.fql.FullQueryLoggerOptions;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.NoSpamLogger;
 import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 import org.apache.cassandra.utils.concurrent.WeightedQueue;
 
 /**
@@ -293,7 +292,7 @@ public class BinLog implements Runnable
                 }
                 catch (InterruptedException e)
                 {
-                    throw new RuntimeException(e);
+                    throw new UncheckedInterruptedException(e);
                 }
             }
             else
diff --git a/src/java/org/apache/cassandra/utils/binlog/ExternalArchiver.java b/src/java/org/apache/cassandra/utils/binlog/ExternalArchiver.java
index 0e7cc04..b444573 100644
--- a/src/java/org/apache/cassandra/utils/binlog/ExternalArchiver.java
+++ b/src/java/org/apache/cassandra/utils/binlog/ExternalArchiver.java
@@ -39,6 +39,7 @@ import org.slf4j.LoggerFactory;
 import net.openhft.chronicle.queue.impl.single.SingleChronicleQueue;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
@@ -137,7 +138,11 @@ public class ExternalArchiver implements BinLogArchiver
             // and try to archive all remaining files before exiting
             archiveExisting(path);
         }
-        catch (InterruptedException | ExecutionException e)
+        catch (InterruptedException e)
+        {
+            throw new UncheckedInterruptedException(e);
+        }
+        catch (ExecutionException e)
         {
             throw new RuntimeException(e);
         }
diff --git a/src/java/org/apache/cassandra/utils/btree/BTree.java b/src/java/org/apache/cassandra/utils/btree/BTree.java
index 6c546ac..4171cac 100644
--- a/src/java/org/apache/cassandra/utils/btree/BTree.java
+++ b/src/java/org/apache/cassandra/utils/btree/BTree.java
@@ -25,7 +25,6 @@ import java.util.function.Consumer;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Ordering;
 
diff --git a/src/java/org/apache/cassandra/utils/concurrent/AsyncFuture.java b/src/java/org/apache/cassandra/utils/concurrent/AsyncFuture.java
new file mode 100644
index 0000000..8930a2a
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/AsyncFuture.java
@@ -0,0 +1,444 @@
+/*
+ * 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.utils.concurrent;
+
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import javax.annotation.Nullable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.util.concurrent.EventExecutor;
+import io.netty.util.concurrent.GenericFutureListener;
+import io.netty.util.internal.ThrowableUtil;
+
+import static java.util.concurrent.atomic.AtomicReferenceFieldUpdater.newUpdater;
+
+/**
+ * Our default {@link Future} implementation, with all state being managed without locks (except those used by the JVM).
+ *
+ * Some implementation comments versus Netty's default promise:
+ *  - We permit efficient initial state declaration, avoiding unnecessary CAS or lock acquisitions when mutating
+ *    a Promise we are ourselves constructing (and can easily add more; only those we use have been added)
+ *  - We guarantee the order of invocation of listeners (and callbacks etc, and with respect to each other)
+ *  - We save some space when registering listeners, especially if there is only one listener, as we perform no
+ *    extra allocations in this case.
+ *  - We implement our invocation list as a concurrent stack, that is cleared on notification
+ *  - We handle special values slightly differently.
+ *    - We do not use a special value for null, instead using a special value to indicate the result has not been set.
+ *      This means that once isSuccess() holds, the result must be a correctly typed object (modulo generics pitfalls).
+ *    - All special values are also instances of FailureHolder, which simplifies a number of the logical conditions.
+ */
+@SuppressWarnings("rawtypes")
+public class AsyncFuture<V> extends Awaitable.AsyncAwaitable implements Future<V>
+{
+    private static final Logger logger = LoggerFactory.getLogger(AsyncFuture.class);
+
+    protected static final FailureHolder UNSET = new FailureHolder(null);
+    protected static final FailureHolder UNCANCELLABLE = new FailureHolder(null);
+    protected static final FailureHolder CANCELLED = new FailureHolder(ThrowableUtil.unknownStackTrace(new CancellationException(), AsyncFuture.class, "cancel(...)"));
+
+    static class FailureHolder
+    {
+        final Throwable cause;
+        FailureHolder(Throwable cause)
+        {
+            this.cause = cause;
+        }
+    }
+
+    protected static Throwable cause(Object result)
+    {
+        return result instanceof FailureHolder ? ((FailureHolder) result).cause : null;
+    }
+    protected static boolean isSuccess(Object result)
+    {
+        return !(result instanceof FailureHolder);
+    }
+    protected static boolean isCancelled(Object result)
+    {
+        return result == CANCELLED;
+    }
+    protected static boolean isDone(Object result)
+    {
+        return result != UNSET && result != UNCANCELLABLE;
+    }
+
+    private final @Nullable Executor notifyExecutor;
+    private volatile Object result;
+    private volatile GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>> listeners;
+    private static final AtomicReferenceFieldUpdater<AsyncFuture, Object> resultUpdater = newUpdater(AsyncFuture.class, Object.class, "result");
+    private static final AtomicReferenceFieldUpdater<AsyncFuture, GenericFutureListener> listenersUpdater = newUpdater(AsyncFuture.class, GenericFutureListener.class, "listeners");
+
+    private static final DeferredGenericFutureListener NOTIFYING = future -> {};
+
+    private static interface DeferredGenericFutureListener<F extends Future<?>> extends GenericFutureListener<F> {}
+
+    public AsyncFuture()
+    {
+        this(null, UNSET);
+    }
+
+    public AsyncFuture(Executor notifyExecutor)
+    {
+        this(notifyExecutor, UNSET);
+    }
+
+    protected AsyncFuture(Executor notifyExecutor, V immediateSuccess)
+    {
+        resultUpdater.lazySet(this, immediateSuccess);
+        this.notifyExecutor = notifyExecutor;
+    }
+
+    protected AsyncFuture(Executor notifyExecutor, Throwable immediateFailure)
+    {
+        resultUpdater.lazySet(this, new FailureHolder(immediateFailure));
+        this.notifyExecutor = notifyExecutor;
+    }
+
+    protected AsyncFuture(Executor notifyExecutor, FailureHolder initialState)
+    {
+        resultUpdater.lazySet(this, initialState);
+        this.notifyExecutor = notifyExecutor;
+    }
+
+    protected AsyncFuture(Executor notifyExecutor, GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>> listener)
+    {
+        this(notifyExecutor);
+        this.listeners = listener;
+    }
+
+    protected AsyncFuture(Executor notifyExecutor, FailureHolder initialState, GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>> listener)
+    {
+        this(notifyExecutor, initialState);
+        this.listeners = listener;
+    }
+
+    protected boolean trySuccess(V v)
+    {
+        return trySet(v);
+    }
+
+    protected boolean tryFailure(Throwable throwable)
+    {
+        return trySet(new FailureHolder(throwable));
+    }
+
+    protected boolean setUncancellable()
+    {
+        if (trySet(UNCANCELLABLE))
+            return true;
+        return isUncancellable();
+    }
+
+    protected boolean setUncancellableExclusive()
+    {
+        return trySet(UNCANCELLABLE);
+    }
+
+    protected boolean isUncancellable()
+    {
+        Object result = this.result;
+        return result == UNCANCELLABLE || (isDone(result) && !isCancelled(result));
+    }
+
+    public boolean cancel(boolean b)
+    {
+        return trySet(CANCELLED);
+    }
+
+    /**
+     * Shared implementation of various promise completion methods.
+     * Updates the result if it is possible to do so, returning success/failure.
+     *
+     * If the promise is UNSET the new value will succeed;
+     *          if it is UNCANCELLABLE it will succeed only if the new value is not CANCELLED
+     *          otherwise it will fail, as isDone() is implied
+     *
+     * If the update succeeds, and the new state implies isDone(), any listeners and waiters will be notified
+     */
+    private boolean trySet(Object v)
+    {
+        while (true)
+        {
+            Object current = result;
+            if (isDone(current) || (current == UNCANCELLABLE && v == CANCELLED))
+                return false;
+            if (resultUpdater.compareAndSet(this, current, v))
+            {
+                if (v != UNCANCELLABLE)
+                {
+                    notifyListeners();
+                    signal();
+                }
+                return true;
+            }
+        }
+    }
+
+    @Override
+    public boolean isSuccess()
+    {
+        return isSuccess(result);
+    }
+
+    @Override
+    public boolean isCancelled()
+    {
+        return isCancelled(result);
+    }
+
+    @Override
+    public boolean isDone()
+    {
+        return isDone(result);
+    }
+
+    @Override
+    public boolean isCancellable()
+    {
+        return result == UNSET;
+    }
+
+    @Override
+    public Throwable cause()
+    {
+        return cause(result);
+    }
+
+    /**
+     * if isSuccess(), returns the value, otherwise returns null
+     */
+    @Override
+    public V getNow()
+    {
+        Object result = this.result;
+        if (isSuccess(result))
+            return (V) result;
+        return null;
+    }
+
+    /**
+     * Shared implementation of get() after suitable await(); assumes isDone(), and returns
+     * either the success result or throws the suitable exception under failure
+     */
+    protected V getWhenDone() throws ExecutionException
+    {
+        Object result = this.result;
+        if (isSuccess(result))
+            return (V) result;
+        if (result == CANCELLED)
+            throw new CancellationException();
+        throw new ExecutionException(((FailureHolder) result).cause);
+    }
+
+    @Override
+    public V get() throws InterruptedException, ExecutionException
+    {
+        await();
+        return getWhenDone();
+    }
+
+    @Override
+    public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+    {
+        if (!await(timeout, unit))
+            throw new TimeoutException();
+        return getWhenDone();
+    }
+
+    @Override
+    public Future<V> addListener(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>> listener)
+    {
+        listenersUpdater.accumulateAndGet(this, listener, AsyncFuture::appendListener);
+        if (isDone())
+            notifyListeners();
+        return this;
+    }
+
+    private void notifyListeners()
+    {
+        if (notifyExecutor != null)
+        {
+            // TODO: could generify to any executor able to say if already executing within
+            if (notifyExecutor instanceof EventExecutor && ((EventExecutor) notifyExecutor).inEventLoop())
+                doNotifyListenersExclusive();
+            else if (listeners != null) // submit this method, to guarantee we invoke in the submitted order
+                notifyExecutor.execute(this::doNotifyListenersExclusive);
+        }
+        else
+        {
+            doNotifyListeners();
+        }
+    }
+
+    private void doNotifyListeners()
+    {
+        @SuppressWarnings("rawtypes") GenericFutureListener listeners;
+        while (true)
+        {
+            listeners = this.listeners;
+            if (listeners == null || listeners instanceof DeferredGenericFutureListener<?>)
+                return; // either no listeners, or we are already notifying listeners, so we'll get to the new one when ready
+
+            if (listenersUpdater.compareAndSet(this, listeners, NOTIFYING))
+            {
+                while (true)
+                {
+                    invokeListener(listeners, this);
+                    if (listenersUpdater.compareAndSet(this, NOTIFYING, null))
+                        return;
+                    listeners = listenersUpdater.getAndSet(this, NOTIFYING);
+                }
+            }
+        }
+    }
+
+    private void doNotifyListenersExclusive()
+    {
+        if (listeners == null || listeners instanceof DeferredGenericFutureListener<?>)
+            return; // either no listeners, or we are already notifying listeners, so we'll get to the new one when ready
+
+        while (true)
+        {
+            @SuppressWarnings("rawtypes") GenericFutureListener listeners = listenersUpdater.getAndSet(this, NOTIFYING);
+            if (listeners != null)
+                invokeListener(listeners, this);
+
+            if (listenersUpdater.compareAndSet(this, NOTIFYING, null))
+                return;
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public Future<V> addListeners(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>>... listeners)
+    {
+        // this could be more efficient if we cared, but we do not
+        return addListener(future -> {
+            for (GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>> listener : listeners)
+                invokeListener((GenericFutureListener<io.netty.util.concurrent.Future<? super V>>)listener, future);
+        });
+    }
+
+    @Override
+    public Future<V> removeListener(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>> listener)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public Future<V> removeListeners(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>>... listeners)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    private static <F extends Future<?>> GenericFutureListener<F> appendListener(GenericFutureListener<F> prevListener, GenericFutureListener<F> newListener)
+    {
+        GenericFutureListener<F> result = newListener;
+
+        if (prevListener != null && prevListener != NOTIFYING)
+        {
+            result = future -> {
+                invokeListener(prevListener, future);
+                // we will wrap the outer invocation with invokeListener, so no need to do it here too
+                newListener.operationComplete(future);
+            };
+        }
+
+        if (prevListener instanceof DeferredGenericFutureListener<?>)
+        {
+            GenericFutureListener<F> wrap = result;
+            result = (DeferredGenericFutureListener<F>) wrap::operationComplete;
+        }
+
+        return result;
+    }
+
+    /**
+     * Wait for this future to complete {@link Awaitable#await()}
+     */
+    @Override
+    public AsyncFuture<V> await() throws InterruptedException
+    {
+        super.await();
+        return this;
+    }
+
+    /**
+     * Wait for this future to complete {@link Awaitable#awaitUninterruptibly()}
+     */
+    @Override
+    public AsyncFuture<V> awaitUninterruptibly()
+    {
+        super.awaitUninterruptibly();
+        return this;
+    }
+
+    /**
+     * Wait for this future to complete {@link Awaitable#awaitThrowUncheckedOnInterrupt()}
+     */
+    @Override
+    public AsyncFuture<V> awaitThrowUncheckedOnInterrupt() throws UncheckedInterruptedException
+    {
+        super.awaitThrowUncheckedOnInterrupt();
+        return this;
+    }
+
+    /**
+     * {@link AsyncAwaitable#isSignalled()}
+     */
+    @Override
+    protected boolean isSignalled()
+    {
+        return isDone(result);
+    }
+
+    public String toString()
+    {
+        Object result = this.result;
+        if (isSuccess(result))
+            return "(success: " + result + ')';
+        if (result == UNCANCELLABLE)
+            return "(uncancellable)";
+        if (result == CANCELLED)
+            return "(cancelled)";
+        if (isDone(result))
+            return "(failure: " + ((FailureHolder) result).cause + ')';
+        return "(incomplete)";
+    }
+
+    protected static <F extends io.netty.util.concurrent.Future<?>> void invokeListener(GenericFutureListener<F> listener, F future)
+    {
+        try
+        {
+            listener.operationComplete(future);
+        }
+        catch (Throwable t)
+        {
+            logger.error("Failed to invoke listener {} to {}", listener, future, t);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/concurrent/AsyncPromise.java b/src/java/org/apache/cassandra/utils/concurrent/AsyncPromise.java
new file mode 100644
index 0000000..3336ccc
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/AsyncPromise.java
@@ -0,0 +1,223 @@
+/*
+ * 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.utils.concurrent;
+
+import java.util.concurrent.Executor;
+
+import io.netty.util.concurrent.EventExecutor;
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.GenericFutureListener;
+
+/**
+ * Extends {@link AsyncFuture} to implement the {@link Promise} interface.
+ */
+public class AsyncPromise<V> extends AsyncFuture<V> implements Promise<V>
+{
+    public AsyncPromise() {}
+
+    public AsyncPromise(Executor notifyExecutor)
+    {
+        super(notifyExecutor);
+    }
+
+    AsyncPromise(Executor notifyExecutor, V immediateSuccess)
+    {
+        super(notifyExecutor, immediateSuccess);
+    }
+
+    AsyncPromise(Executor notifyExecutor, Throwable immediateFailure)
+    {
+        super(notifyExecutor, immediateFailure);
+    }
+
+    AsyncPromise(Executor notifyExecutor, FailureHolder initialState)
+    {
+        super(notifyExecutor, initialState);
+    }
+
+    public AsyncPromise(Executor notifyExecutor, GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>> listener)
+    {
+        super(notifyExecutor, listener);
+    }
+
+    AsyncPromise(Executor notifyExecutor, FailureHolder initialState, GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>> listener)
+    {
+        super(notifyExecutor, initialState, listener);
+    }
+
+    public static <V> AsyncPromise<V> uncancellable(EventExecutor executor)
+    {
+        return new AsyncPromise<>(executor, UNCANCELLABLE);
+    }
+
+    public static <V> AsyncPromise<V> uncancellable(EventExecutor executor, GenericFutureListener<? extends Future<? super V>> listener)
+    {
+        return new AsyncPromise<>(executor, UNCANCELLABLE, listener);
+    }
+
+    /**
+     * Complete the promise successfully if not already complete
+     * @throws IllegalStateException if already set
+     */
+    @Override
+    public Promise<V> setSuccess(V v)
+    {
+        if (!trySuccess(v))
+            throw new IllegalStateException("complete already: " + this);
+        return this;
+    }
+
+    /**
+     * Complete the promise successfully if not already complete
+     * @return true iff completed promise
+     */
+    @Override
+    public boolean trySuccess(V v)
+    {
+        return super.trySuccess(v);
+    }
+
+    /**
+     * Complete the promise abnormally if not already complete
+     * @throws IllegalStateException if already set
+     */
+    @Override
+    public Promise<V> setFailure(Throwable throwable)
+    {
+        if (!tryFailure(throwable))
+            throw new IllegalStateException("complete already: " + this);
+        return this;
+    }
+
+    /**
+     * Complete the promise abnormally if not already complete
+     * @return true iff completed promise
+     */
+    @Override
+    public boolean tryFailure(Throwable throwable)
+    {
+        return super.tryFailure(throwable);
+    }
+
+    /**
+     * Prevent a future caller from cancelling this promise
+     * @return true if the promise is now uncancellable (whether or not we did this)
+     */
+    @Override
+    public boolean setUncancellable()
+    {
+        return super.setUncancellable();
+    }
+
+    /**
+     * Prevent a future caller from cancelling this promise
+     * @return true iff this invocation set it to uncancellable, whether or not now uncancellable
+     */
+    @Override
+    public boolean setUncancellableExclusive()
+    {
+        return super.setUncancellableExclusive();
+    }
+
+    @Override
+    public boolean isUncancellable()
+    {
+        return super.isUncancellable();
+    }
+
+    /**
+     * waits for completion; in case of failure rethrows the original exception without a new wrapping exception
+     * so may cause problems for reporting stack traces
+     */
+    @Override
+    public Promise<V> sync() throws InterruptedException
+    {
+        super.sync();
+        return this;
+    }
+
+    /**
+     * waits for completion; in case of failure rethrows the original exception without a new wrapping exception
+     * so may cause problems for reporting stack traces
+     */
+    @Override
+    public Promise<V> syncUninterruptibly()
+    {
+        super.syncUninterruptibly();
+        return this;
+    }
+
+    @Override
+    public Promise<V> addListener(GenericFutureListener<? extends Future<? super V>> listener)
+    {
+        super.addListener(listener);
+        return this;
+    }
+
+    @Override
+    public Promise<V> addListeners(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>>... listeners)
+    {
+        super.addListeners(listeners);
+        return this;
+    }
+
+    @Override
+    public Promise<V> removeListener(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>> listener)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Promise<V> removeListeners(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>>... listeners)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Wait for this promise to complete
+     * @throws InterruptedException if interrupted
+     */
+    @Override
+    public AsyncPromise<V> await() throws InterruptedException
+    {
+        super.await();
+        return this;
+    }
+
+    /**
+     * Wait uninterruptibly for this promise to complete
+     */
+    @Override
+    public AsyncPromise<V> awaitUninterruptibly()
+    {
+        super.awaitUninterruptibly();
+        return this;
+    }
+
+    /**
+     * Wait for this promise to complete, throwing any interrupt as an UnhandledInterruptedException
+     * @throws UncheckedInterruptedException if interrupted
+     */
+    @Override
+    public AsyncPromise<V> awaitThrowUncheckedOnInterrupt() throws UncheckedInterruptedException
+    {
+        super.awaitThrowUncheckedOnInterrupt();
+        return this;
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Awaitable.java b/src/java/org/apache/cassandra/utils/concurrent/Awaitable.java
new file mode 100644
index 0000000..6be7371
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/Awaitable.java
@@ -0,0 +1,403 @@
+/*
+ * 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.utils.concurrent;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Predicate;
+
+import net.nicoulaj.compilecommand.annotations.Inline;
+
+import org.apache.cassandra.utils.Intercept;
+
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue;
+
+/**
+ * A generic signal consumer, supporting all of the typical patterns used in Cassandra.
+ * All of the methods defined in {@link Awaitable} may be waited on without a loop,
+ * as this interface declares that there are no spurious wake-ups.
+ */
+public interface Awaitable
+{
+    /**
+     * Await until the deadline (in nanoTime), throwing any interrupt.
+     * No spurious wakeups.
+     * @return true if we were signalled, false if the deadline elapsed
+     * @throws InterruptedException if interrupted
+     */
+    boolean awaitUntil(long nanoTimeDeadline) throws InterruptedException;
+
+    /**
+     * Await until the deadline (in nanoTime), throwing any interrupt as an unchecked exception.
+     * No spurious wakeups.
+     * @return true if we were signalled, false if the deadline elapsed
+     * @throws UncheckedInterruptedException if interrupted
+     */
+    boolean awaitUntilThrowUncheckedOnInterrupt(long nanoTimeDeadline) throws UncheckedInterruptedException;
+
+    /**
+     * Await until the deadline (in nanoTime), ignoring interrupts (but maintaining the interrupt flag on exit).
+     * No spurious wakeups.
+     * @return true if we were signalled, false if the deadline elapsed
+     */
+    boolean awaitUntilUninterruptibly(long nanoTimeDeadline);
+
+    /**
+     * Await for the specified period, throwing any interrupt.
+     * No spurious wakeups.
+     * @return true if we were signalled, false if the timeout elapses
+     * @throws InterruptedException if interrupted
+     */
+    boolean await(long time, TimeUnit units) throws InterruptedException;
+
+    /**
+     * Await for the specified period, throwing any interrupt as an unchecked exception.
+     * No spurious wakeups.
+     * @return true if we were signalled, false if the timeout elapses
+     * @throws UncheckedInterruptedException if interrupted
+     */
+    boolean awaitThrowUncheckedOnInterrupt(long time, TimeUnit units) throws UncheckedInterruptedException;
+
+    /**
+     * Await until the deadline (in nanoTime), ignoring interrupts (but maintaining the interrupt flag on exit).
+     * No spurious wakeups.
+     * @return true if we were signalled, false if the timeout elapses
+     */
+    boolean awaitUninterruptibly(long time, TimeUnit units);
+
+    /**
+     * Await indefinitely, throwing any interrupt.
+     * No spurious wakeups.
+     * @throws InterruptedException if interrupted
+     */
+    Awaitable await() throws InterruptedException;
+
+    /**
+     * Await indefinitely, throwing any interrupt as an unchecked exception.
+     * No spurious wakeups.
+     * @throws UncheckedInterruptedException if interrupted
+     */
+    Awaitable awaitThrowUncheckedOnInterrupt() throws UncheckedInterruptedException;
+
+    /**
+     * Await indefinitely, ignoring interrupts (but maintaining the interrupt flag on exit).
+     * No spurious wakeups.
+     */
+    Awaitable awaitUninterruptibly();
+
+    public static boolean await(Awaitable await, long time, TimeUnit unit) throws InterruptedException
+    {
+        return await.awaitUntil(nanoTime() + unit.toNanos(time));
+    }
+
+    public static boolean awaitThrowUncheckedOnInterrupt(Awaitable await, long time, TimeUnit units) throws UncheckedInterruptedException
+    {
+        return awaitUntilThrowUncheckedOnInterrupt(await, nanoTime() + units.toNanos(time));
+    }
+
+    public static boolean awaitUninterruptibly(Awaitable await, long time, TimeUnit units)
+    {
+        return awaitUntilUninterruptibly(await, nanoTime() + units.toNanos(time));
+    }
+
+    public static <A extends Awaitable> A awaitThrowUncheckedOnInterrupt(A await) throws UncheckedInterruptedException
+    {
+        try
+        {
+            await.await();
+        }
+        catch (InterruptedException e)
+        {
+            throw new UncheckedInterruptedException();
+        }
+        return await;
+    }
+
+    public static boolean awaitUntilThrowUncheckedOnInterrupt(Awaitable await, long nanoTimeDeadline) throws UncheckedInterruptedException
+    {
+        try
+        {
+            return await.awaitUntil(nanoTimeDeadline);
+        }
+        catch (InterruptedException e)
+        {
+            throw new UncheckedInterruptedException();
+        }
+    }
+
+    /**
+     * {@link Awaitable#awaitUntilUninterruptibly(long)}
+     */
+    public static boolean awaitUntilUninterruptibly(Awaitable await, long nanoTimeDeadline)
+    {
+        boolean interrupted = false;
+        boolean result;
+        while (true)
+        {
+            try
+            {
+                result = await.awaitUntil(nanoTimeDeadline);
+                break;
+            }
+            catch (InterruptedException e)
+            {
+                interrupted = true;
+            }
+        }
+        if (interrupted)
+            Thread.currentThread().interrupt();
+        return result;
+    }
+
+    /**
+     * {@link Awaitable#awaitUninterruptibly()}
+     */
+    public static <A extends Awaitable> A awaitUninterruptibly(A await)
+    {
+        boolean interrupted = false;
+        while (true)
+        {
+            try
+            {
+                await.await();
+                break;
+            }
+            catch (InterruptedException e)
+            {
+                interrupted = true;
+            }
+        }
+        if (interrupted)
+            Thread.currentThread().interrupt();
+        return await;
+    }
+
+    abstract class AbstractAwaitable implements Awaitable
+    {
+        // WARNING: if you extend this class, be sure to consider interaction with WaitManager to ensure Simulator compatibility
+        protected AbstractAwaitable() {}
+
+        /**
+         * {@link Awaitable#await(long, TimeUnit)}
+         */
+        @Override
+        public boolean await(long time, TimeUnit unit) throws InterruptedException
+        {
+            return Awaitable.await(this, time, unit);
+        }
+
+        /**
+         * {@link Awaitable#awaitThrowUncheckedOnInterrupt(long, TimeUnit)}
+         */
+        @Override
+        public boolean awaitThrowUncheckedOnInterrupt(long time, TimeUnit units) throws UncheckedInterruptedException
+        {
+            return Awaitable.awaitThrowUncheckedOnInterrupt(this, time, units);
+        }
+
+        /**
+         * {@link Awaitable#awaitUninterruptibly(long, TimeUnit)}
+         */
+        public boolean awaitUninterruptibly(long time, TimeUnit units)
+        {
+            return awaitUntilUninterruptibly(nanoTime() + units.toNanos(time));
+        }
+
+        /**
+         * {@link Awaitable#awaitThrowUncheckedOnInterrupt()}
+         */
+        public Awaitable awaitThrowUncheckedOnInterrupt() throws UncheckedInterruptedException
+        {
+            return Awaitable.awaitThrowUncheckedOnInterrupt(this);
+        }
+
+        /**
+         * {@link Awaitable#awaitUntilThrowUncheckedOnInterrupt(long)}
+         */
+        public boolean awaitUntilThrowUncheckedOnInterrupt(long nanoTimeDeadline) throws UncheckedInterruptedException
+        {
+            return Awaitable.awaitUntilThrowUncheckedOnInterrupt(this, nanoTimeDeadline);
+        }
+
+        /**
+         * {@link Awaitable#awaitUntilUninterruptibly(long)}
+         */
+        public boolean awaitUntilUninterruptibly(long nanoTimeDeadline)
+        {
+            return Awaitable.awaitUntilUninterruptibly(this, nanoTimeDeadline);
+        }
+
+        /**
+         * {@link Awaitable#awaitUninterruptibly()}
+         */
+        public Awaitable awaitUninterruptibly()
+        {
+            return Awaitable.awaitUninterruptibly(this);
+        }
+    }
+
+    /**
+     * A barebones asynchronous {@link Awaitable}.
+     * If your state is minimal, or can be updated concurrently, extend this class.
+     */
+    abstract class AsyncAwaitable extends AbstractAwaitable
+    {
+        /**
+         * Maintain an internal variable containing a lazily-initialized wait queue
+         * @return null if is done
+         */
+        @Inline
+        private static <A extends Awaitable> WaitQueue.Signal register(AtomicReferenceFieldUpdater<A, WaitQueue> waitingUpdater, Predicate<A> isDone, A awaitable)
+        {
+            if (isDone.test(awaitable))
+                return null;
+
+            WaitQueue waiting = waitingUpdater.get(awaitable);
+            if (waiting == null)
+            {
+                if (!waitingUpdater.compareAndSet(awaitable, null, waiting = newWaitQueue()))
+                {
+                    waiting = waitingUpdater.get(awaitable);
+                    if (waiting == null)
+                    {
+                        assert isDone.test(awaitable);
+                        return null;
+                    }
+                }
+            }
+
+            WaitQueue.Signal s = waiting.register();
+            if (!isDone.test(awaitable))
+                return s;
+
+            s.cancel();
+            return null;
+        }
+
+        @Inline
+        static <A extends Awaitable> A await(AtomicReferenceFieldUpdater<A, WaitQueue> waitingUpdater, Predicate<A> isDone, A awaitable) throws InterruptedException
+        {
+            WaitQueue.Signal s = register(waitingUpdater, isDone, awaitable);
+            if (s != null)
+                s.await();
+            return awaitable;
+        }
+
+        @Inline
+        static <A extends Awaitable> boolean awaitUntil(AtomicReferenceFieldUpdater<A, WaitQueue> waitingUpdater, Predicate<A> isDone, A awaitable, long nanoTimeDeadline) throws InterruptedException
+        {
+            WaitQueue.Signal s = register(waitingUpdater, isDone, awaitable);
+            return s == null || s.awaitUntil(nanoTimeDeadline) || isDone.test(awaitable);
+        }
+
+        @Inline
+        static <A extends Awaitable> void signalAll(AtomicReferenceFieldUpdater<A, WaitQueue> waitingUpdater, A awaitable)
+        {
+            WaitQueue waiting = waitingUpdater.get(awaitable);
+            if (waiting == null)
+                return;
+
+            waiting.signalAll();
+            waitingUpdater.lazySet(awaitable, null);
+        }
+
+        private static final AtomicReferenceFieldUpdater<AsyncAwaitable, WaitQueue> waitingUpdater = AtomicReferenceFieldUpdater.newUpdater(AsyncAwaitable.class, WaitQueue.class, "waiting");
+        private volatile WaitQueue waiting;
+
+        // WARNING: if you extend this class, be sure to consider interaction with WaitManager to ensure Simulator compatibility
+        protected AsyncAwaitable() {}
+
+        /**
+         * {@link Awaitable#await()}
+         */
+        public Awaitable await() throws InterruptedException
+        {
+            return await(waitingUpdater, AsyncAwaitable::isSignalled, this);
+        }
+
+        /**
+         * {@link Awaitable#awaitUntil(long)}
+         */
+        public boolean awaitUntil(long nanoTimeDeadline) throws InterruptedException
+        {
+            return awaitUntil(waitingUpdater, AsyncAwaitable::isSignalled, this, nanoTimeDeadline);
+        }
+
+        /**
+         * Signal any waiting threads; {@link #isSignalled()} must return {@code true} before this method is invoked.
+         */
+        protected void signal()
+        {
+            signalAll(waitingUpdater, this);
+        }
+
+        /**
+         * Return true once signalled. Unidirectional; once true, must never again be false.
+         */
+        protected abstract boolean isSignalled();
+    }
+
+    /**
+     * A barebones {@link Awaitable} that uses mutual exclusion.
+     * If your state will be updated while holding the object monitor, extend this class.
+     */
+    abstract class SyncAwaitable extends AbstractAwaitable
+    {
+        protected SyncAwaitable() {}
+
+        /**
+         * {@link Awaitable#await()}
+         */
+        public synchronized Awaitable await() throws InterruptedException
+        {
+            while (!isSignalled())
+                wait();
+            return this;
+        }
+
+        /**
+         * {@link Awaitable#awaitUntil(long)}
+         */
+        public synchronized boolean awaitUntil(long nanoTimeDeadline) throws InterruptedException
+        {
+            while (true)
+            {
+                if (isSignalled()) return true;
+                if (!waitUntil(this, nanoTimeDeadline)) return false;
+            }
+        }
+
+        /**
+         * Return true once signalled. Unidirectional; once true, must never again be false.
+         */
+        protected abstract boolean isSignalled();
+
+        @Intercept
+        public static boolean waitUntil(Object monitor, long deadlineNanos) throws InterruptedException
+        {
+            long wait = deadlineNanos - nanoTime();
+            if (wait <= 0)
+                return false;
+
+            monitor.wait((wait + 999999) / 1000000);
+            return true;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Blocker.java b/src/java/org/apache/cassandra/utils/concurrent/Blocker.java
deleted file mode 100644
index 5192e98..0000000
--- a/src/java/org/apache/cassandra/utils/concurrent/Blocker.java
+++ /dev/null
@@ -1,63 +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.utils.concurrent;
-
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
-
-public class Blocker
-{
-    private final ReentrantLock lock = new ReentrantLock();
-    private final Condition unblocked = lock.newCondition();
-    private volatile boolean block = false;
-
-    public void block(boolean block)
-    {
-        this.block = block;
-        if (!block)
-        {
-            lock.lock();
-            try
-            {
-                unblocked.signalAll();
-            }
-            finally
-            {
-                lock.unlock();
-            }
-        }
-    }
-
-    public void ask()
-    {
-        if (block)
-        {
-            lock.lock();
-            try
-            {
-                while (block)
-                    unblocked.awaitUninterruptibly();
-            }
-            finally
-            {
-                lock.unlock();
-            }
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/utils/concurrent/BlockingQueues.java b/src/java/org/apache/cassandra/utils/concurrent/BlockingQueues.java
new file mode 100644
index 0000000..a7425d2
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/BlockingQueues.java
@@ -0,0 +1,253 @@
+/*
+ * 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.utils.concurrent;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Queue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.cassandra.utils.Intercept;
+
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+import static org.apache.cassandra.utils.concurrent.Awaitable.SyncAwaitable.waitUntil;
+
+public class BlockingQueues
+{
+    @Intercept
+    public static <T> BlockingQueue<T> newBlockingQueue()
+    {
+        return new LinkedBlockingQueue<>();
+    }
+
+    @Intercept
+    public static <T> BlockingQueue<T> newBlockingQueue(int capacity)
+    {
+        return capacity == 0 ? new SynchronousQueue<>()
+                             : new LinkedBlockingQueue<>(capacity);
+    }
+
+    public static class Sync<T> implements BlockingQueue<T>
+    {
+        final int capacity;
+        final Queue<T> wrapped;
+        public Sync(int capacity, Queue<T> wrapped)
+        {
+            this.capacity = capacity;
+            this.wrapped = wrapped;
+        }
+
+        public synchronized boolean add(T t)
+        {
+            if (!wrapped.add(t))
+                throw new IllegalStateException();
+            notify();
+            return true;
+        }
+
+        public synchronized boolean offer(T t)
+        {
+            if (wrapped.size() == capacity)
+                return false;
+            return add(t);
+        }
+
+        public synchronized T remove()
+        {
+            return poll();
+        }
+
+        public synchronized T poll()
+        {
+            if (wrapped.size() == capacity)
+                notify();
+
+            return wrapped.poll();
+        }
+
+        public synchronized T element()
+        {
+            return wrapped.element();
+        }
+
+        public synchronized T peek()
+        {
+            return wrapped.peek();
+        }
+
+        public synchronized void put(T t) throws InterruptedException
+        {
+            while (!offer(t))
+                wait();
+        }
+
+        public synchronized boolean offer(T t, long timeout, TimeUnit unit) throws InterruptedException
+        {
+            if (offer(t))
+                return true;
+
+            long deadline = nanoTime() + unit.toNanos(timeout);
+            while (true)
+            {
+                if (offer(t))
+                    return true;
+
+                if (!waitUntil(this, deadline))
+                    return false;
+            }
+        }
+
+        public synchronized T take() throws InterruptedException
+        {
+            T result;
+            while (null == (result = poll()))
+                wait();
+
+            return result;
+        }
+
+        public synchronized T poll(long timeout, TimeUnit unit) throws InterruptedException
+        {
+            T result = poll();
+            if (result != null)
+                return result;
+
+            long deadline = nanoTime() + unit.toNanos(timeout);
+            while (null == (result = poll()))
+            {
+                if (!waitUntil(this, deadline))
+                    return null;
+            }
+            return result;
+        }
+
+        public synchronized int remainingCapacity()
+        {
+            return capacity - wrapped.size();
+        }
+
+        public synchronized boolean remove(Object o)
+        {
+            if (!wrapped.remove(o))
+                return false;
+            if (wrapped.size() == capacity - 1)
+                notify();
+            return true;
+        }
+
+        public synchronized boolean containsAll(Collection<?> c)
+        {
+            return wrapped.containsAll(c);
+        }
+
+        public synchronized boolean addAll(Collection<? extends T> c)
+        {
+            c.forEach(this::add);
+            return true;
+        }
+
+        public synchronized boolean removeAll(Collection<?> c)
+        {
+            boolean result = wrapped.removeAll(c);
+            notifyAll();
+            return result;
+        }
+
+        public synchronized boolean retainAll(Collection<?> c)
+        {
+            boolean result = wrapped.retainAll(c);
+            notifyAll();
+            return result;
+        }
+
+        public synchronized void clear()
+        {
+            wrapped.clear();
+            notifyAll();
+        }
+
+        public synchronized int size()
+        {
+            return wrapped.size();
+        }
+
+        public synchronized boolean isEmpty()
+        {
+            return wrapped.isEmpty();
+        }
+
+        public synchronized boolean contains(Object o)
+        {
+            return wrapped.contains(o);
+        }
+
+        public synchronized Iterator<T> iterator()
+        {
+            Iterator<T> iter = wrapped.iterator();
+            return new Iterator<T>()
+            {
+                public boolean hasNext()
+                {
+                    synchronized (Sync.this)
+                    {
+                        return iter.hasNext();
+                    }
+                }
+
+                public T next()
+                {
+                    synchronized (Sync.this)
+                    {
+                        return iter.next();
+                    }
+                }
+            };
+        }
+
+        public synchronized Object[] toArray()
+        {
+            return wrapped.toArray();
+        }
+
+        public synchronized <T1> T1[] toArray(T1[] a)
+        {
+            return wrapped.toArray(a);
+        }
+
+        public synchronized int drainTo(Collection<? super T> c)
+        {
+            return drainTo(c, Integer.MAX_VALUE);
+        }
+
+        public synchronized int drainTo(Collection<? super T> c, int maxElements)
+        {
+            int count = 0;
+            while (count < maxElements && !isEmpty())
+            {
+                c.add(poll());
+                ++count;
+            }
+
+            return count;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Condition.java b/src/java/org/apache/cassandra/utils/concurrent/Condition.java
new file mode 100644
index 0000000..f47e20f
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/Condition.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.utils.concurrent;
+
+import org.apache.cassandra.utils.Intercept;
+
+/**
+ * Simpler API than java.util.concurrent.Condition; would be nice to extend it, but also nice
+ * to share API with Future, for which Netty's API is incompatible with java.util.concurrent.Condition
+ *
+ * {@link Awaitable} for explicit external signals.
+ */
+public interface Condition extends Awaitable
+{
+    /**
+     * Returns true once signalled. Unidirectional; once true, will never again be false.
+     */
+    boolean isSignalled();
+
+    /**
+     * Signal the condition as met, and wake all waiting threads.
+     */
+    void signal();
+
+    /**
+     * Signal the condition as met, and wake all waiting threads.
+     */
+    default void signalAll() { signal(); }
+
+    /**
+     * Factory method used to capture and redirect instantiations for simulation
+     */
+    @Intercept
+    static Condition newOneTimeCondition()
+    {
+        return new Async();
+    }
+
+    /**
+     * An asynchronous {@link Condition}. Typically lower overhead than {@link Sync}.
+     */
+    public static class Async extends AsyncAwaitable implements Condition
+    {
+        private volatile boolean signaled = false;
+
+        // WARNING: if extending this class, consider simulator interactions
+        protected Async() {}
+
+        public boolean isSignalled()
+        {
+            return signaled;
+        }
+
+        public void signal()
+        {
+            signaled = true;
+            super.signal();
+        }
+    }
+
+    /**
+     * A {@link Condition} based on its object monitor.
+     * WARNING: lengthy operations performed while holding the lock may prevent timely notification of waiting threads
+     * that a deadline has passed.
+     */
+    public static class Sync extends SyncAwaitable implements Condition
+    {
+        private boolean signaled = false;
+
+        // this can be instantiated directly, as we intercept monitors directly with byte weaving
+        public Sync() {}
+
+        public synchronized boolean isSignalled()
+        {
+            return signaled;
+        }
+
+        public synchronized void signal()
+        {
+            signaled = true;
+            notifyAll();
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/concurrent/CountDownLatch.java b/src/java/org/apache/cassandra/utils/concurrent/CountDownLatch.java
new file mode 100644
index 0000000..5988375
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/CountDownLatch.java
@@ -0,0 +1,107 @@
+/*
+ * 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.utils.concurrent;
+
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+
+import org.apache.cassandra.utils.Intercept;
+
+public interface CountDownLatch extends Awaitable
+{
+    /**
+     * Count down by 1, signalling waiters if we have reached zero
+     */
+    void decrement();
+
+    /**
+     * @return the current count
+     */
+    int count();
+
+    /**
+     * Factory method used to capture and redirect instantiations for simulation
+     */
+    @Intercept
+    static CountDownLatch newCountDownLatch(int count)
+    {
+        return new Async(count);
+    }
+
+    static class Async extends AsyncAwaitable implements CountDownLatch
+    {
+        private static final AtomicIntegerFieldUpdater<CountDownLatch.Async> countUpdater = AtomicIntegerFieldUpdater.newUpdater(CountDownLatch.Async.class, "count");
+        private volatile int count;
+
+        // WARNING: if extending this class, consider simulator interactions
+        protected Async(int count)
+        {
+            this.count = count;
+            if (count == 0)
+                signal();
+        }
+
+        public void decrement()
+        {
+            if (countUpdater.decrementAndGet(this) == 0)
+                signal();
+        }
+
+        public int count()
+        {
+            return count;
+        }
+
+        @Override
+        protected boolean isSignalled()
+        {
+            return count <= 0;
+        }
+    }
+
+    static final class Sync extends SyncAwaitable implements CountDownLatch
+    {
+        private int count;
+
+        public Sync(int count)
+        {
+            this.count = count;
+        }
+
+        public synchronized void decrement()
+        {
+            if (count > 0 && --count == 0)
+                notifyAll();
+        }
+
+        public synchronized int count()
+        {
+            return count;
+        }
+
+        /**
+         * not synchronized as only intended for internal usage by externally synchronized methods
+         */
+
+        @Override
+        protected boolean isSignalled()
+        {
+            return count <= 0;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Future.java b/src/java/org/apache/cassandra/utils/concurrent/Future.java
new file mode 100644
index 0000000..c23f9e8
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/Future.java
@@ -0,0 +1,121 @@
+/*
+ * 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.utils.concurrent;
+
+import java.util.concurrent.Executor;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+import io.netty.util.internal.PlatformDependent;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
+/**
+ * A Future that integrates several different (but equivalent) APIs used within Cassandra into a single concept,
+ * integrating also with our {@link Awaitable} abstraction, to overall improve coherency and clarity in the codebase.
+ */
+public interface Future<V> extends io.netty.util.concurrent.Future<V>, ListenableFuture<V>, Awaitable
+{
+    /**
+     * Wait indefinitely for this future to complete, throwing any interrupt
+     * @throws InterruptedException if interrupted
+     */
+    @Override
+    Future<V> await() throws InterruptedException;
+
+    /**
+     * Wait indefinitely for this future to complete
+     */
+    @Override
+    Future<V> awaitUninterruptibly();
+
+    /**
+     * Wait indefinitely for this promise to complete, throwing any interrupt as an UnhandledInterruptedException
+     * @throws UncheckedInterruptedException if interrupted
+     */
+    @Override
+    Future<V> awaitThrowUncheckedOnInterrupt();
+
+    default void rethrowIfFailed()
+    {
+        Throwable cause = this.cause();
+        if (cause != null)
+        {
+            PlatformDependent.throwException(cause);
+        }
+    }
+
+    /**
+     * waits for completion; in case of failure rethrows the original exception without a new wrapping exception
+     * so may cause problems for reporting stack traces
+     */
+    @Override
+    default Future<V> sync() throws InterruptedException
+    {
+        await();
+        rethrowIfFailed();
+        return this;
+    }
+
+    /**
+     * waits for completion; in case of failure rethrows the original exception without a new wrapping exception
+     * so may cause problems for reporting stack traces
+     */
+    @Override
+    default Future<V> syncUninterruptibly()
+    {
+        awaitUninterruptibly();
+        rethrowIfFailed();
+        return this;
+    }
+
+    @Deprecated
+    @Override
+    default boolean await(long l) throws InterruptedException
+    {
+        return await(l, MILLISECONDS);
+    }
+
+    @Deprecated
+    @Override
+    default boolean awaitUninterruptibly(long l)
+    {
+        return awaitUninterruptibly(l, MILLISECONDS);
+    }
+
+    /**
+     * Invoke {@code runnable} on completion, using {@code executor}.
+     * Tasks are submitted to their executors in the order they were added to this Future.
+     */
+    @Override
+    default void addListener(Runnable runnable, Executor executor)
+    {
+        addListener(future -> executor.execute(runnable));
+    }
+
+    /**
+     * Invoke {@code runnable} on completion. Depending on the implementation and its configuration, this
+     * may be executed immediately by the notifying/completing thread, or asynchronously by an executor.
+     * Tasks are executed, or submitted to the executor, in the order they were added to this Future.
+     */
+    default void addListener(Runnable runnable)
+    {
+        addListener(future -> runnable.run());
+    }
+}
diff --git a/test/microbench/org/apache/cassandra/test/microbench/FastThreadExecutor.java b/src/java/org/apache/cassandra/utils/concurrent/NotScheduledFuture.java
similarity index 56%
copy from test/microbench/org/apache/cassandra/test/microbench/FastThreadExecutor.java
copy to src/java/org/apache/cassandra/utils/concurrent/NotScheduledFuture.java
index 5644e4f..a81f9a9 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/FastThreadExecutor.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/NotScheduledFuture.java
@@ -16,24 +16,46 @@
  * limitations under the License.
  */
 
-package org.apache.cassandra.test.microbench;
+package org.apache.cassandra.utils.concurrent;
 
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 
-import io.netty.util.concurrent.DefaultThreadFactory;
-
-/**
- * Created to test perf of FastThreadLocal
- *
- * Used in MutationBench via:
- * jvmArgsAppend = {"-Djmh.executor=CUSTOM", "-Djmh.executor.class=org.apache.cassandra.test.microbench.FastThreadExecutor"}
- */
-public class FastThreadExecutor extends ThreadPoolExecutor
+public class NotScheduledFuture<T> implements ScheduledFuture<T>
 {
-    public FastThreadExecutor(int size, String name)
+    public long getDelay(TimeUnit unit)
+    {
+        return 0;
+    }
+
+    public int compareTo(Delayed o)
+    {
+        return 0;
+    }
+
+    public boolean cancel(boolean mayInterruptIfRunning)
+    {
+        return false;
+    }
+
+    public boolean isCancelled()
+    {
+        return false;
+    }
+
+    public boolean isDone()
+    {
+        return false;
+    }
+
+    public T get()
+    {
+        return null;
+    }
+
+    public T get(long timeout, TimeUnit unit)
     {
-        super(size, size, 10, TimeUnit.SECONDS, new LinkedBlockingQueue<>(), new DefaultThreadFactory(name, true));
+        return null;
     }
 }
diff --git a/src/java/org/apache/cassandra/utils/concurrent/OpOrder.java b/src/java/org/apache/cassandra/utils/concurrent/OpOrder.java
index 863f038..7f18a0c 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/OpOrder.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/OpOrder.java
@@ -18,7 +18,11 @@
  */
 package org.apache.cassandra.utils.concurrent;
 
+import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+
+import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue;
 
 /**
  * <p>A class for providing synchronization between producers and consumers that do not
@@ -113,8 +117,6 @@ public class OpOrder
      * after which all new operations will start against a new Group that will not be accepted
      * by barrier.isAfter(), and barrier.await() will return only once all operations started prior to the issue
      * have completed.
-     *
-     * @return
      */
     public Barrier newBarrier()
     {
@@ -162,10 +164,11 @@ public class OpOrder
         private final long id; // monotonically increasing id for compareTo()
         private volatile int running = 0; // number of operations currently running.  < 0 means we're expired, and the count of tasks still running is -(running + 1)
         private volatile boolean isBlocking; // indicates running operations are blocking future barriers
-        private final WaitQueue isBlockingSignal = new WaitQueue(); // signal to wait on to indicate isBlocking is true
-        private final WaitQueue waiting = new WaitQueue(); // signal to wait on for completion
+        private volatile ConcurrentLinkedQueue<WaitQueue.Signal> blocking; // signal to wait on to indicate isBlocking is true
+        private final WaitQueue waiting = newWaitQueue(); // signal to wait on for completion
 
         static final AtomicIntegerFieldUpdater<Group> runningUpdater = AtomicIntegerFieldUpdater.newUpdater(Group.class, "running");
+        static final AtomicReferenceFieldUpdater<Group, ConcurrentLinkedQueue> blockingUpdater = AtomicReferenceFieldUpdater.newUpdater(Group.class, ConcurrentLinkedQueue.class, "blocking");
 
         // constructs first instance only
         private Group()
@@ -318,21 +321,21 @@ public class OpOrder
             return isBlocking;
         }
 
-        /**
-         * register to be signalled when a barrier waiting on us is, or maybe, blocking general progress,
-         * so we should try more aggressively to progress
-         */
-        public WaitQueue.Signal isBlockingSignal()
+        public void notifyIfBlocking(WaitQueue.Signal signal)
         {
-            return isBlockingSignal.register();
+            if (blocking == null)
+                blockingUpdater.compareAndSet(this, null, new ConcurrentLinkedQueue<>());
+            blocking.add(signal);
+            if (isBlocking() && blocking.remove(signal))
+                signal.signal();
         }
 
-        /**
-         * wrap the provided signal to also be signalled if the operation gets marked blocking
-         */
-        public WaitQueue.Signal isBlockingSignal(WaitQueue.Signal signal)
+        private void markBlocking()
         {
-            return WaitQueue.any(signal, isBlockingSignal());
+            isBlocking = true;
+            ConcurrentLinkedQueue<WaitQueue.Signal> blocking = this.blocking;
+            if (blocking != null)
+                blocking.forEach(WaitQueue.Signal::signal);
         }
 
         public int compareTo(Group that)
@@ -406,21 +409,12 @@ public class OpOrder
             Group current = orderOnOrBefore;
             while (current != null)
             {
-                current.isBlocking = true;
-                current.isBlockingSignal.signalAll();
+                current.markBlocking();
                 current = current.prev;
             }
         }
 
         /**
-         * Register to be signalled once allPriorOpsAreFinished() or allPriorOpsAreFinishedOrSafe() may return true
-         */
-        public WaitQueue.Signal register()
-        {
-            return orderOnOrBefore.waiting.register();
-        }
-
-        /**
          * wait for all operations started prior to issuing the barrier to complete
          */
         public void await()
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Promise.java b/src/java/org/apache/cassandra/utils/concurrent/Promise.java
new file mode 100644
index 0000000..06620fe
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/Promise.java
@@ -0,0 +1,114 @@
+/*
+ * 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.utils.concurrent;
+
+import java.util.concurrent.ExecutorService;
+
+import com.google.common.util.concurrent.FutureCallback;
+
+import io.netty.util.concurrent.GenericFutureListener;
+
+/**
+ * A Promise that integrates {@link io.netty.util.concurrent.Promise} with our {@link Future} API
+ * to improve clarity and coherence in the codebase.
+ */
+public interface Promise<V> extends io.netty.util.concurrent.Promise<V>, Future<V>
+{
+    public static <V> GenericFutureListener<? extends Future<V>> listener(FutureCallback<V> callback)
+    {
+        return future -> {
+            if (future.isSuccess()) callback.onSuccess(future.getNow());
+            else callback.onFailure(future.cause());
+        };
+    }
+
+    public static <V> GenericFutureListener<? extends Future<V>> listener(ExecutorService executor, FutureCallback<V> callback)
+    {
+        return future -> executor.execute(() -> {
+            if (future.isSuccess()) callback.onSuccess(future.getNow());
+            else callback.onFailure(future.cause());
+        });
+    }
+
+    @Override
+    Promise<V> addListener(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>> var1);
+
+    @Override
+    Promise<V> addListeners(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>>... var1);
+
+    @Override
+    Promise<V> removeListener(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>> var1);
+
+    @Override
+    Promise<V> removeListeners(GenericFutureListener<? extends io.netty.util.concurrent.Future<? super V>>... var1);
+
+    /**
+     * Complete the promise successfully if not already complete
+     * @throws IllegalStateException if already set
+     */
+    @Override
+    Promise<V> setSuccess(V v) throws IllegalStateException;
+
+    /**
+     * Complete the promise abnormally if not already complete
+     * @throws IllegalStateException if already set
+     */
+    @Override
+    Promise<V> setFailure(Throwable throwable) throws IllegalStateException;
+
+    /**
+     * Prevent a future caller from cancelling this promise
+     * @return true iff this invocation set it to uncancellable, whether or not now uncancellable
+     */
+    boolean setUncancellableExclusive();
+
+    /**
+     * Wait indefinitely for this promise to complete, throwing any interrupt
+     * @throws InterruptedException if interrupted
+     */
+    @Override
+    Promise<V> await() throws InterruptedException;
+
+    /**
+     * Wait indefinitely for this promise to complete
+     */
+    @Override
+    Promise<V> awaitUninterruptibly();
+
+    /**
+     * Wait indefinitely for this promise to complete, throwing any interrupt as an UnhandledInterruptedException
+     * @throws UncheckedInterruptedException if interrupted
+     */
+    @Override
+    Promise<V> awaitThrowUncheckedOnInterrupt();
+
+    /**
+     * waits for completion; in case of failure rethrows the original exception without a new wrapping exception
+     * so may cause problems for reporting stack traces
+     */
+    @Override
+    Promise<V> sync() throws InterruptedException;
+
+    /**
+     * waits for completion; in case of failure rethrows the original exception without a new wrapping exception
+     * so may cause problems for reporting stack traces
+     */
+    @Override
+    Promise<V> syncUninterruptibly();
+}
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Semaphore.java b/src/java/org/apache/cassandra/utils/concurrent/Semaphore.java
new file mode 100644
index 0000000..f16ffdd
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/Semaphore.java
@@ -0,0 +1,333 @@
+/*
+ * 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.utils.concurrent;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+
+import net.openhft.chronicle.core.util.ThrowingConsumer;
+import org.apache.cassandra.utils.Intercept;
+
+import static java.lang.System.nanoTime;
+import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue;
+
+public interface Semaphore
+{
+    /**
+     * @return the number of permits presently in this semaphore
+     */
+    int permits();
+
+    /**
+     * set the number of permits in this semaphore to zero
+     */
+    int drain();
+
+    /**
+     * Increase the number of available permits and signal any waiters that may be served by the release
+     */
+    void release(int permits);
+
+    /**
+     * Try to take permits, returning immediately
+     * @return true iff permits acquired
+     */
+    boolean tryAcquire(int acquire);
+
+    /**
+     * Try to take permits, waiting up to timeout
+     * @return true iff permits acquired
+     * @throws InterruptedException if interrupted
+     */
+    boolean tryAcquire(int acquire, long time, TimeUnit unit) throws InterruptedException;
+
+    /**
+     * Try to take permits, waiting until the deadline
+     * @return true iff permits acquired
+     * @throws InterruptedException if interrupted
+     */
+    boolean tryAcquireUntil(int acquire, long nanoTimeDeadline) throws InterruptedException;
+
+    /**
+     * Take permits, waiting indefinitely until available
+     * @throws InterruptedException if interrupted
+     */
+    void acquire(int acquire) throws InterruptedException;
+
+    /**
+     * Take permits, waiting indefinitely until available
+     * @throws UncheckedInterruptedException if interrupted
+     */
+    void acquireThrowUncheckedOnInterrupt(int acquire) throws UncheckedInterruptedException;
+
+    /**
+     * Factory method used to capture and redirect instantiations for simulation
+     *
+     * Construct an unfair Semaphore initially holding the specified number of permits
+     */
+    @Intercept
+    public static Semaphore newSemaphore(int permits)
+    {
+        return new UnfairAsync(permits);
+    }
+
+    /**
+     * Factory method used to capture and redirect instantiations for simulation
+     *
+     * Construct a fair Semaphore initially holding the specified number of permits
+     */
+    @Intercept
+    public static Semaphore newFairSemaphore(int permits)
+    {
+        return new FairJDK(permits);
+    }
+
+    /**
+     * An unfair semaphore, making no guarantees about thread starvation.
+     *
+     * TODO this Semaphore is potentially inefficient if used with release quantities other than 1
+     *      (this is unimportant at time of authoring)
+     */
+    public static class UnfairAsync implements Semaphore
+    {
+        private static final AtomicReferenceFieldUpdater<UnfairAsync, WaitQueue> waitingUpdater = AtomicReferenceFieldUpdater.newUpdater(UnfairAsync.class, WaitQueue.class, "waiting");
+        private static final AtomicIntegerFieldUpdater<UnfairAsync> permitsUpdater = AtomicIntegerFieldUpdater.newUpdater(UnfairAsync.class, "permits");
+        private volatile WaitQueue waiting;
+        private volatile int permits;
+
+        // WARNING: if extending this class, consider simulator interactions
+        public UnfairAsync(int permits)
+        {
+            this.permits = permits;
+        }
+
+        /**
+         * {@link Semaphore#drain()}
+         */
+        public int drain()
+        {
+            return permitsUpdater.getAndSet(this, 0);
+        }
+
+        /**
+         * {@link Semaphore#permits()}
+         */
+        public int permits()
+        {
+            return permits;
+        }
+
+        /**
+         * {@link Semaphore#release(int)}
+         */
+        public void release(int permits)
+        {
+            if (permits < 0) throw new IllegalArgumentException();
+            if (permits > 0 && permitsUpdater.getAndAdd(this, permits) == 0)
+            {
+                if (waiting != null)
+                {
+                    if (permits > 1) waiting.signalAll();
+                    else waiting.signal();
+                }
+            }
+        }
+
+        /**
+         * {@link Semaphore#tryAcquire(int)}
+         */
+        public boolean tryAcquire(int acquire)
+        {
+            if (acquire < 0)
+                throw new IllegalArgumentException();
+            while (true)
+            {
+                int cur = permits;
+                if (cur < acquire)
+                    return false;
+                if (permitsUpdater.compareAndSet(this, cur, cur - acquire))
+                    return true;
+            }
+        }
+
+        /**
+         * {@link Semaphore#tryAcquire(int, long, TimeUnit)}
+         */
+        public boolean tryAcquire(int acquire, long time, TimeUnit unit) throws InterruptedException
+        {
+            return tryAcquireUntil(acquire, nanoTime() + unit.toNanos(time));
+        }
+
+        /**
+         * {@link Semaphore#tryAcquireUntil(int, long)}
+         */
+        public boolean tryAcquireUntil(int acquire, long nanoTimeDeadline) throws InterruptedException
+        {
+            boolean wait = true;
+            while (true)
+            {
+                int cur = permits;
+                if (cur < acquire)
+                {
+                    if (!wait) return false;
+
+                    WaitQueue.Signal signal = register();
+                    if (permits < acquire) wait = signal.awaitUntil(nanoTimeDeadline);
+                    else signal.cancel();
+                }
+                else if (permitsUpdater.compareAndSet(this, cur, cur - acquire))
+                    return true;
+            }
+        }
+
+        /**
+         * {@link Semaphore#acquire(int)}
+         */
+        public void acquire(int acquire) throws InterruptedException
+        {
+            acquire(acquire, WaitQueue.Signal::await);
+        }
+
+        /**
+         * {@link Semaphore#acquireThrowUncheckedOnInterrupt(int)}
+         */
+        public void acquireThrowUncheckedOnInterrupt(int acquire)
+        {
+            acquire(acquire, WaitQueue.Signal::awaitThrowUncheckedOnInterrupt);
+        }
+
+        private <T extends Throwable> void acquire(int acquire, ThrowingConsumer<WaitQueue.Signal, T> wait) throws T
+        {
+            while (true)
+            {
+                int cur = permits;
+                if (cur < acquire)
+                {
+                    WaitQueue.Signal signal = register();
+                    if (permits < acquire) wait.accept(signal);
+                    else signal.cancel();
+                }
+                else if (permitsUpdater.compareAndSet(this, cur, cur - acquire))
+                    return;
+            }
+        }
+
+        private WaitQueue.Signal register()
+        {
+            if (waiting == null)
+                waitingUpdater.compareAndSet(this, null, newWaitQueue());
+            return waiting.register();
+        }
+    }
+
+    /**
+     * A fair semaphore, guaranteeing threads are signalled in the order they request permits.
+     *
+     * Unlike {@link UnfairAsync} this class is efficient for arbitrarily-sized increments and decrements,
+     * however it has the normal throughput bottleneck of fairness.
+     */
+    public static class FairJDK implements Semaphore
+    {
+        final java.util.concurrent.Semaphore wrapped;
+
+        public FairJDK(int permits)
+        {
+            wrapped = new java.util.concurrent.Semaphore(permits, true);
+        }
+
+        /**
+         * {@link Semaphore#drain()}
+         */
+        public int drain()
+        {
+            return wrapped.drainPermits();
+        }
+
+        /**
+         * Number of permits that are available to be acquired. {@link Semaphore#permits()}
+         */
+        public int permits()
+        {
+            return wrapped.availablePermits();
+        }
+
+        /**
+         * Number of permits that have been acquired in excess of available. {@link Semaphore#permits()}
+         */
+        public int waiting()
+        {
+            return wrapped.getQueueLength();
+        }
+
+        /**
+         * {@link Semaphore#release(int)}
+         */
+        public void release(int permits)
+        {
+            wrapped.release(permits);
+        }
+
+        /**
+         * {@link Semaphore#tryAcquire(int)}
+         */
+        public boolean tryAcquire(int permits)
+        {
+            return wrapped.tryAcquire(permits);
+        }
+
+        /**
+         * {@link Semaphore#tryAcquire(int, long, TimeUnit)}
+         */
+        public boolean tryAcquire(int acquire, long time, TimeUnit unit) throws InterruptedException
+        {
+            return wrapped.tryAcquire(acquire, time, unit);
+        }
+
+        /**
+         * {@link Semaphore#tryAcquireUntil(int, long)}
+         */
+        public boolean tryAcquireUntil(int acquire, long nanoTimeDeadline) throws InterruptedException
+        {
+            long wait = nanoTimeDeadline - System.nanoTime();
+            return wrapped.tryAcquire(acquire, Math.max(0, wait), TimeUnit.NANOSECONDS);
+        }
+
+        /**
+         * {@link Semaphore#acquire(int)}
+         */
+        public void acquire(int acquire) throws InterruptedException
+        {
+            wrapped.acquire(acquire);
+        }
+
+        @Override
+        public void acquireThrowUncheckedOnInterrupt(int acquire) throws UncheckedInterruptedException
+        {
+            try
+            {
+                acquire(acquire);
+            }
+            catch (InterruptedException e)
+            {
+                throw new UncheckedInterruptedException(e);
+            }
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/concurrent/SimpleCondition.java b/src/java/org/apache/cassandra/utils/concurrent/SimpleCondition.java
deleted file mode 100644
index 844cfda..0000000
--- a/src/java/org/apache/cassandra/utils/concurrent/SimpleCondition.java
+++ /dev/null
@@ -1,107 +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.utils.concurrent;
-
-import java.util.Date;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
-import java.util.concurrent.locks.Condition;
-
-import static org.apache.cassandra.utils.Clock.Global.nanoTime;
-
-// fulfils the Condition interface without spurious wakeup problems
-// (or lost notify problems either: that is, even if you call await()
-// _after_ signal(), it will work as desired.)
-public class SimpleCondition implements Condition
-{
-    private static final AtomicReferenceFieldUpdater<SimpleCondition, WaitQueue> waitingUpdater = AtomicReferenceFieldUpdater.newUpdater(SimpleCondition.class, WaitQueue.class, "waiting");
-
-    private volatile WaitQueue waiting;
-    private volatile boolean signaled = false;
-
-    @Override
-    public void await() throws InterruptedException
-    {
-        if (isSignaled())
-            return;
-        if (waiting == null)
-            waitingUpdater.compareAndSet(this, null, new WaitQueue());
-        WaitQueue.Signal s = waiting.register();
-        if (isSignaled())
-            s.cancel();
-        else
-            s.await();
-        assert isSignaled();
-    }
-
-    public boolean await(long time, TimeUnit unit) throws InterruptedException
-    {
-        long start = nanoTime();
-        long until = start + unit.toNanos(time);
-        return awaitUntil(until);
-    }
-
-    public boolean awaitUntil(long deadlineNanos) throws InterruptedException
-    {
-        if (isSignaled())
-            return true;
-        if (waiting == null)
-            waitingUpdater.compareAndSet(this, null, new WaitQueue());
-        WaitQueue.Signal s = waiting.register();
-        if (isSignaled())
-        {
-            s.cancel();
-            return true;
-        }
-        return s.awaitUntil(deadlineNanos) || isSignaled();
-    }
-
-    public void signal()
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public boolean isSignaled()
-    {
-        return signaled;
-    }
-
-    public void signalAll()
-    {
-        signaled = true;
-        if (waiting != null)
-            waiting.signalAll();
-    }
-
-    public void awaitUninterruptibly()
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public long awaitNanos(long nanosTimeout)
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public boolean awaitUntil(Date deadline)
-    {
-        throw new UnsupportedOperationException();
-    }
-}
diff --git a/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java b/src/java/org/apache/cassandra/utils/concurrent/UncheckedInterruptedException.java
similarity index 65%
copy from src/java/org/apache/cassandra/streaming/StreamingMessageSender.java
copy to src/java/org/apache/cassandra/utils/concurrent/UncheckedInterruptedException.java
index 96e7626..8e85b84 100644
--- a/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/UncheckedInterruptedException.java
@@ -16,22 +16,18 @@
  * limitations under the License.
  */
 
-package org.apache.cassandra.streaming;
+package org.apache.cassandra.utils.concurrent;
 
-import java.io.IOException;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.streaming.messages.StreamMessage;
-
-public interface StreamingMessageSender
+/**
+ * Unchecked {@link InterruptedException}, to be thrown in places where an interrupt is unexpected
+ */
+public class UncheckedInterruptedException extends RuntimeException
 {
-    void initialize() throws IOException;
-
-    void sendMessage(StreamMessage message) throws IOException;
-
-    boolean connected();
-
-    void close();
+    public UncheckedInterruptedException()
+    {
+    }
+    public UncheckedInterruptedException(InterruptedException cause)
+    {
+        super(cause);
+    }
 }
diff --git a/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java b/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java
index 295af1b..53e7e8d 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java
@@ -23,11 +23,12 @@ import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 import java.util.concurrent.locks.LockSupport;
 import java.util.function.BooleanSupplier;
+import java.util.function.Consumer;
 
-import com.codahale.metrics.Timer;
+import org.apache.cassandra.utils.Intercept;
+import org.apache.cassandra.utils.concurrent.Awaitable.AbstractAwaitable;
 
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
-
 /**
  * <p>A relatively easy to use utility for general purpose thread signalling.</p>
  * <p>Usage on a thread awaiting a state change using a WaitQueue q is:</p>
@@ -69,131 +70,11 @@ import static org.apache.cassandra.utils.Clock.Global.nanoTime;
  * to be met that we no longer need.
  * <p>5. This scheme is not fair</p>
  * <p>6. Only the thread that calls register() may call await()</p>
+ *
+ * TODO: this class should not be backed by CLQ (should use an intrusive linked-list with lower overhead)
  */
-public final class WaitQueue
+public interface WaitQueue
 {
-
-    private static final int CANCELLED = -1;
-    private static final int SIGNALLED = 1;
-    private static final int NOT_SET = 0;
-
-    private static final AtomicIntegerFieldUpdater signalledUpdater = AtomicIntegerFieldUpdater.newUpdater(RegisteredSignal.class, "state");
-
-    // the waiting signals
-    private final ConcurrentLinkedQueue<RegisteredSignal> queue = new ConcurrentLinkedQueue<>();
-
-    /**
-     * The calling thread MUST be the thread that uses the signal
-     * @return                                x
-     */
-    public Signal register()
-    {
-        RegisteredSignal signal = new RegisteredSignal();
-        queue.add(signal);
-        return signal;
-    }
-
-    /**
-     * The calling thread MUST be the thread that uses the signal.
-     * If the Signal is waited on, context.stop() will be called when the wait times out, the Signal is signalled,
-     * or the waiting thread is interrupted.
-     * @return
-     */
-    public Signal register(Timer.Context context)
-    {
-        assert context != null;
-        RegisteredSignal signal = new TimedSignal(context);
-        queue.add(signal);
-        return signal;
-    }
-
-    /**
-     * Signal one waiting thread
-     */
-    public boolean signal()
-    {
-        if (!hasWaiters())
-            return false;
-        while (true)
-        {
-            RegisteredSignal s = queue.poll();
-            if (s == null || s.signal() != null)
-                return s != null;
-        }
-    }
-
-    /**
-     * Signal all waiting threads
-     */
-    public void signalAll()
-    {
-        if (!hasWaiters())
-            return;
-
-        // to avoid a race where the condition is not met and the woken thread managed to wait on the queue before
-        // we finish signalling it all, we pick a random thread we have woken-up and hold onto it, so that if we encounter
-        // it again we know we're looping. We reselect a random thread periodically, progressively less often.
-        // the "correct" solution to this problem is to use a queue that permits snapshot iteration, but this solution is sufficient
-        int i = 0, s = 5;
-        Thread randomThread = null;
-        Iterator<RegisteredSignal> iter = queue.iterator();
-        while (iter.hasNext())
-        {
-            RegisteredSignal signal = iter.next();
-            Thread signalled = signal.signal();
-
-            if (signalled != null)
-            {
-                if (signalled == randomThread)
-                    break;
-
-                if (++i == s)
-                {
-                    randomThread = signalled;
-                    s <<= 1;
-                }
-            }
-
-            iter.remove();
-        }
-    }
-
-    private void cleanUpCancelled()
-    {
-        // TODO: attempt to remove the cancelled from the beginning only (need atomic cas of head)
-        Iterator<RegisteredSignal> iter = queue.iterator();
-        while (iter.hasNext())
-        {
-            RegisteredSignal s = iter.next();
-            if (s.isCancelled())
-                iter.remove();
-        }
-    }
-
-    public boolean hasWaiters()
-    {
-        return !queue.isEmpty();
-    }
-
-    /**
-     * Return how many threads are waiting
-     * @return
-     */
-    public int getWaiting()
-    {
-        if (!hasWaiters())
-            return 0;
-        Iterator<RegisteredSignal> iter = queue.iterator();
-        int count = 0;
-        while (iter.hasNext())
-        {
-            Signal next = iter.next();
-            if (!next.isCancelled())
-                count++;
-        }
-        return count;
-    }
-
     /**
      * A Signal is a one-time-use mechanism for a thread to wait for notification that some condition
      * state has transitioned that it may be interested in (and hence should check if it is).
@@ -209,14 +90,8 @@ public final class WaitQueue
      * thread that registered itself with WaitQueue(s) to obtain the underlying RegisteredSignal(s);
      * only the owning thread should use a Signal.
      */
-    public static interface Signal
+    public static interface Signal extends Condition
     {
-
-        /**
-         * @return true if signalled; once true, must be discarded by the owning thread.
-         */
-        public boolean isSignalled();
-
         /**
          * @return true if cancelled; once cancelled, must be discarded by the owning thread.
          */
@@ -240,324 +115,316 @@ public final class WaitQueue
          * and if signalled propagates the signal to another waiting thread
          */
         public abstract void cancel();
+    }
 
-        /**
-         * Wait, without throwing InterruptedException, until signalled. On exit isSignalled() must be true.
-         * If the thread is interrupted in the meantime, the interrupted flag will be set.
-         */
-        public void awaitUninterruptibly();
+    /**
+     * The calling thread MUST be the thread that uses the signal
+     */
+    public Signal register();
 
-        /**
-         * Wait until signalled, or throw an InterruptedException if interrupted before this happens.
-         * On normal exit isSignalled() must be true; however if InterruptedException is thrown isCancelled()
-         * will be true.
-         * @throws InterruptedException
-         */
-        public void await() throws InterruptedException;
+    /**
+     * The calling thread MUST be the thread that uses the signal.
+     * If the Signal is waited on, context.stop() will be called when the wait times out, the Signal is signalled,
+     * or the waiting thread is interrupted.
+     */
+    public <V> Signal register(V supplyOnDone, Consumer<V> receiveOnDone);
 
-        /**
-         * Wait until signalled, or the provided time is reached, or the thread is interrupted. If signalled,
-         * isSignalled() will be true on exit, and the method will return true; if timedout, the method will return
-         * false and isCancelled() will be true; if interrupted an InterruptedException will be thrown and isCancelled()
-         * will be true.
-         * @param nanos nanoTime() to wait until
-         * @return true if signalled, false if timed out
-         * @throws InterruptedException
-         */
-        public boolean awaitUntil(long nanos) throws InterruptedException;
+    /**
+     * Signal one waiting thread
+     */
+    public boolean signal();
 
-        /**
-         * Wait until signalled, or the provided time is reached, or the thread is interrupted. If signalled,
-         * isSignalled() will be true on exit, and the method will return true; if timedout, the method will return
-         * false and isCancelled() will be true
-         * @param nanos nanoTime() to wait until
-         * @return true if signalled, false if timed out
-         */
-        public boolean awaitUntilUninterruptibly(long nanos);
-    }
+    /**
+     * Signal all waiting threads
+     */
+    public void signalAll();
+
+    /** getWaiting() > 0 */
+    public boolean hasWaiters();
+
+    /** Return how many threads are waiting */
+    public int getWaiting();
 
     /**
-     * An abstract signal implementation
+     * Factory method used to capture and redirect instantiations for simulation
      */
-    public static abstract class AbstractSignal implements Signal
+    @Intercept
+    public static WaitQueue newWaitQueue()
     {
-        public void awaitUninterruptibly()
-        {
-            boolean interrupted = false;
-            while (!isSignalled())
-            {
-                if (Thread.interrupted())
-                    interrupted = true;
-                LockSupport.park();
-            }
-            if (interrupted)
-                Thread.currentThread().interrupt();
-            checkAndClear();
-        }
+        return new Standard();
+    }
 
-        public void await() throws InterruptedException
+    class Standard implements WaitQueue
+    {
+        private static final int CANCELLED = -1;
+        private static final int SIGNALLED = 1;
+        private static final int NOT_SET = 0;
+
+        private static final AtomicIntegerFieldUpdater<RegisteredSignal> signalledUpdater = AtomicIntegerFieldUpdater.newUpdater(RegisteredSignal.class, "state");
+
+        // the waiting signals
+        private final ConcurrentLinkedQueue<RegisteredSignal> queue = new ConcurrentLinkedQueue<>();
+
+        /**
+         * The calling thread MUST be the thread that uses the signal
+         * @return                                x
+         */
+        public Signal register()
         {
-            while (!isSignalled())
-            {
-                checkInterrupted();
-                LockSupport.park();
-            }
-            checkAndClear();
+            RegisteredSignal signal = new RegisteredSignal();
+            queue.add(signal);
+            return signal;
         }
 
-        public boolean awaitUntil(long until) throws InterruptedException
+        /**
+         * The calling thread MUST be the thread that uses the signal.
+         * If the Signal is waited on, context.stop() will be called when the wait times out, the Signal is signalled,
+         * or the waiting thread is interrupted.
+         */
+        public <V> Signal register(V supplyOnDone, Consumer<V> receiveOnDone)
         {
-            long now;
-            while (until > (now = nanoTime()) && !isSignalled())
-            {
-                checkInterrupted();
-                long delta = until - now;
-                LockSupport.parkNanos(delta);
-            }
-            return checkAndClear();
+            RegisteredSignal signal = new SignalWithListener<>(supplyOnDone, receiveOnDone);
+            queue.add(signal);
+            return signal;
         }
 
-        public boolean awaitUntilUninterruptibly(long until)
+        /**
+         * Signal one waiting thread
+         */
+        public boolean signal()
         {
-            long now;
-            while (until > (now = nanoTime()) && !isSignalled())
+            while (true)
             {
-                long delta = until - now;
-                LockSupport.parkNanos(delta);
+                RegisteredSignal s = queue.poll();
+                if (s == null || s.doSignal() != null)
+                    return s != null;
             }
-            return checkAndClear();
         }
 
-        private void checkInterrupted() throws InterruptedException
+        /**
+         * Signal all waiting threads
+         */
+        public void signalAll()
         {
-            if (Thread.interrupted())
+            if (!hasWaiters())
+                return;
+
+            // to avoid a race where the condition is not met and the woken thread managed to wait on the queue before
+            // we finish signalling it all, we pick a random thread we have woken-up and hold onto it, so that if we encounter
+            // it again we know we're looping. We reselect a random thread periodically, progressively less often.
+            // the "correct" solution to this problem is to use a queue that permits snapshot iteration, but this solution is sufficient
+            // TODO: this is only necessary because we use CLQ - which is only for historical any-NIH reasons
+            int i = 0, s = 5;
+            Thread randomThread = null;
+            Iterator<RegisteredSignal> iter = queue.iterator();
+            while (iter.hasNext())
             {
-                cancel();
-                throw new InterruptedException();
-            }
-        }
-    }
+                RegisteredSignal signal = iter.next();
+                Thread signalled = signal.doSignal();
 
-    /**
-     * A signal registered with this WaitQueue
-     */
-    private class RegisteredSignal extends AbstractSignal
-    {
-        private volatile Thread thread = Thread.currentThread();
-        volatile int state;
+                if (signalled != null)
+                {
+                    if (signalled == randomThread)
+                        break;
+
+                    if (++i == s)
+                    {
+                        randomThread = signalled;
+                        s <<= 1;
+                    }
+                }
 
-        public boolean isSignalled()
-        {
-            return state == SIGNALLED;
+                iter.remove();
+            }
         }
 
-        public boolean isCancelled()
+        private void cleanUpCancelled()
         {
-            return state == CANCELLED;
+            // TODO: attempt to remove the cancelled from the beginning only (need atomic cas of head)
+            queue.removeIf(RegisteredSignal::isCancelled);
         }
 
-        public boolean isSet()
+        public boolean hasWaiters()
         {
-            return state != NOT_SET;
+            return !queue.isEmpty();
         }
 
-        private Thread signal()
+        /**
+         * @return how many threads are waiting
+         */
+        public int getWaiting()
         {
-            if (!isSet() && signalledUpdater.compareAndSet(this, NOT_SET, SIGNALLED))
+            if (!hasWaiters())
+                return 0;
+            Iterator<RegisteredSignal> iter = queue.iterator();
+            int count = 0;
+            while (iter.hasNext())
             {
-                Thread thread = this.thread;
-                LockSupport.unpark(thread);
-                this.thread = null;
-                return thread;
+                Signal next = iter.next();
+                if (!next.isCancelled())
+                    count++;
             }
-            return null;
+            return count;
         }
 
-        public boolean checkAndClear()
+        /**
+         * An abstract signal implementation
+         *
+         * TODO: use intrusive linked list
+         */
+        public static abstract class AbstractSignal extends AbstractAwaitable implements Signal
         {
-            if (!isSet() && signalledUpdater.compareAndSet(this, NOT_SET, CANCELLED))
+            public Signal await() throws InterruptedException
             {
-                thread = null;
-                cleanUpCancelled();
-                return false;
+                while (!isSignalled())
+                {
+                    checkInterrupted();
+                    LockSupport.park();
+                }
+                checkAndClear();
+                return this;
+            }
+
+            public boolean awaitUntil(long nanoTimeDeadline) throws InterruptedException
+            {
+                long now;
+                while (nanoTimeDeadline > (now = nanoTime()) && !isSignalled())
+                {
+                    checkInterrupted();
+                    long delta = nanoTimeDeadline - now;
+                    LockSupport.parkNanos(delta);
+                }
+                return checkAndClear();
+            }
+
+            private void checkInterrupted() throws InterruptedException
+            {
+                if (Thread.interrupted())
+                {
+                    cancel();
+                    throw new InterruptedException();
+                }
             }
-            // must now be signalled assuming correct API usage
-            return true;
         }
 
         /**
-         * Should only be called by the registered thread. Indicates the signal can be retired,
-         * and if signalled propagates the signal to another waiting thread
+         * A signal registered with this WaitQueue
          */
-        public void cancel()
+        private class RegisteredSignal extends AbstractSignal
         {
-            if (isCancelled())
-                return;
-            if (!signalledUpdater.compareAndSet(this, NOT_SET, CANCELLED))
+            private volatile Thread thread = Thread.currentThread();
+            volatile int state;
+
+            public boolean isSignalled()
             {
-                // must already be signalled - switch to cancelled and
-                state = CANCELLED;
-                // propagate the signal
-                WaitQueue.this.signal();
+                return state == SIGNALLED;
             }
-            thread = null;
-            cleanUpCancelled();
-        }
-    }
-
-    /**
-     * A RegisteredSignal that stores a TimerContext, and stops the timer when either cancelled or
-     * finished waiting. i.e. if the timer is started when the signal is registered it tracks the
-     * time in between registering and invalidating the signal.
-     */
-    private final class TimedSignal extends RegisteredSignal
-    {
-        private final Timer.Context context;
 
-        private TimedSignal(Timer.Context context)
-        {
-            this.context = context;
-        }
+            public boolean isCancelled()
+            {
+                return state == CANCELLED;
+            }
 
-        @Override
-        public boolean checkAndClear()
-        {
-            context.stop();
-            return super.checkAndClear();
-        }
+            public boolean isSet()
+            {
+                return state != NOT_SET;
+            }
 
-        @Override
-        public void cancel()
-        {
-            if (!isCancelled())
+            private Thread doSignal()
             {
-                context.stop();
-                super.cancel();
+                if (!isSet() && signalledUpdater.compareAndSet(this, NOT_SET, SIGNALLED))
+                {
+                    Thread thread = this.thread;
+                    LockSupport.unpark(thread);
+                    this.thread = null;
+                    return thread;
+                }
+                return null;
             }
-        }
-    }
 
-    /**
-     * An abstract signal wrapping multiple delegate signals
-     */
-    private abstract static class MultiSignal extends AbstractSignal
-    {
-        final Signal[] signals;
-        protected MultiSignal(Signal[] signals)
-        {
-            this.signals = signals;
-        }
+            public void signal()
+            {
+                doSignal();
+            }
 
-        public boolean isCancelled()
-        {
-            for (Signal signal : signals)
-                if (!signal.isCancelled())
+            public boolean checkAndClear()
+            {
+                if (!isSet() && signalledUpdater.compareAndSet(this, NOT_SET, CANCELLED))
+                {
+                    thread = null;
+                    cleanUpCancelled();
                     return false;
-            return true;
-        }
-
-        public boolean checkAndClear()
-        {
-            for (Signal signal : signals)
-                signal.checkAndClear();
-            return isSignalled();
-        }
-
-        public void cancel()
-        {
-            for (Signal signal : signals)
-                signal.cancel();
-        }
-    }
+                }
+                // must now be signalled assuming correct API usage
+                return true;
+            }
 
-    /**
-     * A Signal that wraps multiple Signals and returns when any single one of them would have returned
-     */
-    private static class AnySignal extends MultiSignal
-    {
-        protected AnySignal(Signal ... signals)
-        {
-            super(signals);
+            /**
+             * Should only be called by the registered thread. Indicates the signal can be retired,
+             * and if signalled propagates the signal to another waiting thread
+             */
+            public void cancel()
+            {
+                if (isCancelled())
+                    return;
+                if (!signalledUpdater.compareAndSet(this, NOT_SET, CANCELLED))
+                {
+                    // must already be signalled - switch to cancelled and
+                    state = CANCELLED;
+                    // propagate the signal
+                    WaitQueue.Standard.this.signal();
+                }
+                thread = null;
+                cleanUpCancelled();
+            }
         }
 
-        public boolean isSignalled()
+        /**
+         * A RegisteredSignal that stores a TimerContext, and stops the timer when either cancelled or
+         * finished waiting. i.e. if the timer is started when the signal is registered it tracks the
+         * time in between registering and invalidating the signal.
+         */
+        private final class SignalWithListener<V> extends RegisteredSignal
         {
-            for (Signal signal : signals)
-                if (signal.isSignalled())
-                    return true;
-            return false;
-        }
+            private final V supplyOnDone;
+            private final Consumer<V> receiveOnDone;
 
-        public boolean isSet()
-        {
-            for (Signal signal : signals)
-                if (signal.isSet())
-                    return true;
-            return false;
-        }
-    }
+            private SignalWithListener(V supplyOnDone, Consumer<V> receiveOnDone)
+            {
+                this.receiveOnDone = receiveOnDone;
+                this.supplyOnDone = supplyOnDone;
+            }
 
-    /**
-     * A Signal that wraps multiple Signals and returns when all of them would have finished returning
-     */
-    private static class AllSignal extends MultiSignal
-    {
-        protected AllSignal(Signal ... signals)
-        {
-            super(signals);
-        }
 
-        public boolean isSignalled()
-        {
-            for (Signal signal : signals)
-                if (!signal.isSignalled())
-                    return false;
-            return true;
-        }
+            @Override
+            public boolean checkAndClear()
+            {
+                receiveOnDone.accept(supplyOnDone);
+                return super.checkAndClear();
+            }
 
-        public boolean isSet()
-        {
-            for (Signal signal : signals)
-                if (!signal.isSet())
-                    return false;
-            return true;
+            @Override
+            public void cancel()
+            {
+                if (!isCancelled())
+                {
+                    receiveOnDone.accept(supplyOnDone);
+                    super.cancel();
+                }
+            }
         }
     }
 
     /**
-     * @param signals
-     * @return a signal that returns only when any of the provided signals would have returned
-     */
-    public static Signal any(Signal ... signals)
-    {
-        return new AnySignal(signals);
-    }
-
-    /**
-     * @param signals
-     * @return a signal that returns only when all provided signals would have returned
-     */
-    public static Signal all(Signal ... signals)
-    {
-        return new AllSignal(signals);
-    }
-
-    /**
      * Loops waiting on the supplied condition and WaitQueue and will not return until the condition is true
      */
-    public static void waitOnCondition(BooleanSupplier condition, WaitQueue queue)
+    public static void waitOnCondition(BooleanSupplier condition, WaitQueue queue) throws InterruptedException
     {
         while (!condition.getAsBoolean())
         {
             Signal s = queue.register();
-            if (!condition.getAsBoolean())
-            {
-                s.awaitUninterruptibly();
-            }
-            else
-            {
-                s.cancel();
-            }
+            if (!condition.getAsBoolean()) s.await();
+            else s.cancel();
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/utils/concurrent/WeightedQueue.java b/src/java/org/apache/cassandra/utils/concurrent/WeightedQueue.java
index 3a6505e..b1def45 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/WeightedQueue.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/WeightedQueue.java
@@ -22,14 +22,15 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.Objects;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
+import static org.apache.cassandra.utils.concurrent.Semaphore.newSemaphore;
+
 /**
  * Weighted queue is a wrapper around any blocking queue that turns it into a blocking weighted queue. The queue
  * will weigh each element being added and removed. Adding to the queue is blocked if adding would violate
@@ -262,7 +263,7 @@ public class WeightedQueue<T> implements BlockingQueue<T>
 
     public WeightedQueue(int maxWeight)
     {
-        this(maxWeight, new LinkedBlockingQueue<T>(), NATURAL_WEIGHER);
+        this(maxWeight, newBlockingQueue(), NATURAL_WEIGHER);
     }
 
     public WeightedQueue(int maxWeight, BlockingQueue<T> queue, Weigher<T> weigher)
@@ -273,7 +274,7 @@ public class WeightedQueue<T> implements BlockingQueue<T>
         this.maxWeight = maxWeight;
         this.queue = queue;
         this.weigher = weigher;
-        availableWeight = new Semaphore(maxWeight);
+        availableWeight = newSemaphore(maxWeight);
     }
 
     boolean acquireWeight(T weighable, long timeout, TimeUnit unit) throws InterruptedException
diff --git a/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java b/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java
index 9576ccf..2883830 100644
--- a/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java
@@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicLongFieldUpdater;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.codahale.metrics.Timer;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.utils.concurrent.OpOrder;
@@ -182,19 +183,17 @@ public abstract class MemtableAllocator
                     allocated(size);
                     return;
                 }
-                WaitQueue.Signal signal = opGroup.isBlockingSignal(parent.hasRoom().register(parent.blockedTimerContext()));
+                WaitQueue.Signal signal = parent.hasRoom().register(parent.blockedTimerContext(), Timer.Context::stop);
+                opGroup.notifyIfBlocking(signal);
                 boolean allocated = parent.tryAllocate(size);
-                if (allocated || opGroup.isBlocking())
+                if (allocated)
                 {
                     signal.cancel();
-                    if (allocated) // if we allocated, take ownership
-                        acquired(size);
-                    else // otherwise we're blocking so we're permitted to overshoot our constraints, to just allocate without blocking
-                        allocated(size);
+                    acquired(size);
                     return;
                 }
                 else
-                    signal.awaitUninterruptibly();
+                    signal.awaitThrowUncheckedOnInterrupt();
             }
         }
 
diff --git a/src/java/org/apache/cassandra/utils/memory/MemtableCleanerThread.java b/src/java/org/apache/cassandra/utils/memory/MemtableCleanerThread.java
index f6fccc6..78c0f0e 100644
--- a/src/java/org/apache/cassandra/utils/memory/MemtableCleanerThread.java
+++ b/src/java/org/apache/cassandra/utils/memory/MemtableCleanerThread.java
@@ -27,6 +27,8 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.concurrent.InfiniteLoopExecutor;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
+import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue;
+
 /**
  * A thread that reclaims memory from a MemtablePool on demand.  The actual reclaiming work is delegated to the
  * cleaner Runnable, e.g., FlushLargestColumnFamily
@@ -47,7 +49,7 @@ public class MemtableCleanerThread<P extends MemtablePool> extends InfiniteLoopE
         final MemtableCleaner cleaner;
 
         /** signalled whenever needsCleaning() may return true */
-        final WaitQueue wait = new WaitQueue();
+        final WaitQueue wait = newWaitQueue();
 
         private Clean(P pool, MemtableCleaner cleaner)
         {
diff --git a/src/java/org/apache/cassandra/utils/memory/MemtablePool.java b/src/java/org/apache/cassandra/utils/memory/MemtablePool.java
index 58b2910..6793c31 100644
--- a/src/java/org/apache/cassandra/utils/memory/MemtablePool.java
+++ b/src/java/org/apache/cassandra/utils/memory/MemtablePool.java
@@ -32,6 +32,8 @@ import org.apache.cassandra.metrics.DefaultNameFactory;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 import org.apache.cassandra.utils.ExecutorUtils;
 
+import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue;
+
 
 /**
  * Represents an amount of memory used for a given purpose, that can be allocated to specific tasks through
@@ -48,7 +50,7 @@ public abstract class MemtablePool
     public final Timer blockedOnAllocating;
     public final Gauge<Long> numPendingTasks;
 
-    final WaitQueue hasRoom = new WaitQueue();
+    final WaitQueue hasRoom = newWaitQueue();
 
     MemtablePool(long maxOnHeapMemory, long maxOffHeapMemory, float cleanThreshold, MemtableCleaner cleaner)
     {
diff --git a/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java b/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
index 9aecf85..cb5c425 100644
--- a/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
@@ -20,13 +20,15 @@ package org.apache.cassandra.utils.memory;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.Semaphore;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.concurrent.Semaphore;
+
+import static org.apache.cassandra.utils.concurrent.Semaphore.newSemaphore;
 
 /**
  * This NativeAllocator uses global slab allocation strategy
@@ -177,10 +179,10 @@ public class NativeAllocator extends MemtableAllocator
     private static class RaceAllocated
     {
         final ConcurrentLinkedQueue<Region> stash = new ConcurrentLinkedQueue<>();
-        final Semaphore permits = new Semaphore(8);
+        final Semaphore permits = newSemaphore(8);
         boolean stash(Region region)
         {
-            if (!permits.tryAcquire())
+            if (!permits.tryAcquire(1))
                 return false;
             stash.add(region);
             return true;
@@ -189,7 +191,7 @@ public class NativeAllocator extends MemtableAllocator
         {
             Region next = stash.poll();
             if (next != null)
-                permits.release();
+                permits.release(1);
             return next;
         }
     }
diff --git a/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java b/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
index 486ec38..8d118e7 100644
--- a/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
+++ b/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
@@ -17,9 +17,7 @@
  */
 package org.apache.cassandra.utils.obs;
 
-import java.io.DataInput;
 import java.io.DataInputStream;
-import java.io.DataOutput;
 import java.io.IOException;
 
 import com.google.common.annotations.VisibleForTesting;
diff --git a/test/burn/org/apache/cassandra/net/Verifier.java b/test/burn/org/apache/cassandra/net/Verifier.java
index 60b014a..2776dcb 100644
--- a/test/burn/org/apache/cassandra/net/Verifier.java
+++ b/test/burn/org/apache/cassandra/net/Verifier.java
@@ -32,14 +32,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.carrotsearch.hppc.LongObjectHashMap;
-import com.carrotsearch.hppc.predicates.LongObjectPredicate;
 import com.carrotsearch.hppc.procedures.LongObjectProcedure;
-import com.carrotsearch.hppc.procedures.LongProcedure;
 import org.apache.cassandra.net.Verifier.ExpiredMessageEvent.ExpirationType;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
 import static java.util.concurrent.TimeUnit.*;
-import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static org.apache.cassandra.net.MessagingService.VERSION_40;
 import static org.apache.cassandra.net.MessagingService.current_version;
 import static org.apache.cassandra.net.ConnectionType.LARGE_MESSAGES;
@@ -67,6 +64,7 @@ import static org.apache.cassandra.net.Verifier.EventType.SERIALIZE;
 import static org.apache.cassandra.net.Verifier.ExpiredMessageEvent.ExpirationType.ON_SENT;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
 import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue;
 
 /**
  * This class is a single-threaded verifier monitoring a single link, with events supplied by inbound and outbound threads
@@ -1282,7 +1280,7 @@ public class Verifier
 
         // we use a concurrent skip list to permit efficient searching, even if we always append
         final ConcurrentSkipListMap<Long, Chunk> chunkList = new ConcurrentSkipListMap<>();
-        final WaitQueue writerWaiting = new WaitQueue();
+        final WaitQueue writerWaiting = newWaitQueue();
 
         volatile Chunk writerChunk = new Chunk(0);
         Chunk readerChunk = writerChunk;
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
index 8a4ed7f..201555f 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
@@ -45,6 +45,7 @@ import java.util.stream.Stream;
 import javax.annotation.concurrent.GuardedBy;
 
 import com.google.common.collect.Sets;
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -79,11 +80,12 @@ import org.apache.cassandra.distributed.shared.Versions;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.net.Verb;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
 import org.reflections.Reflections;
 import org.reflections.util.ConfigurationBuilder;
 
 import static org.apache.cassandra.distributed.shared.NetworkTopology.addressAndPort;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
+import static org.reflections.ReflectionUtils.forNames;
 
 /**
  * AbstractCluster creates, initializes and manages Cassandra instances ({@link Instance}.
@@ -672,7 +674,7 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
     public abstract class ChangeMonitor implements AutoCloseable
     {
         final List<IListen.Cancel> cleanup;
-        final SimpleCondition completed;
+        final Condition completed;
         private final long timeOut;
         private final TimeUnit timeoutUnit;
         protected Predicate<IInstance> instanceFilter;
@@ -684,7 +686,7 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
             this.timeoutUnit = timeoutUnit;
             this.instanceFilter = i -> true;
             this.cleanup = new ArrayList<>(instances.size());
-            this.completed = new SimpleCondition();
+            this.completed = newOneTimeCondition();
         }
 
         public void ignoreStoppedInstances()
@@ -694,7 +696,7 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
 
         protected void signal()
         {
-            if (initialized && !completed.isSignaled() && isCompleted())
+            if (initialized && !completed.isSignalled() && isCompleted())
                 completed.signalAll();
         }
 
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java b/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java
index 852b3e8..9c4f255 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java
@@ -49,9 +49,6 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
-import static java.lang.Integer.MAX_VALUE;
-import static org.apache.cassandra.cql3.QueryOptions.create;
-import static org.apache.cassandra.service.QueryState.forInternalCalls;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
 
 public class Coordinator implements ICoordinator
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/DelegatingInvokableInstance.java b/test/distributed/org/apache/cassandra/distributed/impl/DelegatingInvokableInstance.java
index 9948ce5..a6bd0f6 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/DelegatingInvokableInstance.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/DelegatingInvokableInstance.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.distributed.impl;
 
 import java.io.Serializable;
 import java.net.InetSocketAddress;
-import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.Future;
 import java.util.function.BiConsumer;
@@ -35,7 +34,6 @@ import org.apache.cassandra.distributed.api.IInvokableInstance;
 import org.apache.cassandra.distributed.api.IListen;
 import org.apache.cassandra.distributed.api.IMessage;
 import org.apache.cassandra.distributed.api.SimpleQueryResult;
-import org.apache.cassandra.distributed.shared.NetworkTopology;
 
 public abstract class DelegatingInvokableInstance implements IInvokableInstance
 {
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java
index c3b8019..a612781 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java
@@ -49,6 +49,8 @@ import org.apache.cassandra.distributed.api.IIsolatedExecutor;
 import org.apache.cassandra.utils.ExecutorUtils;
 import org.apache.cassandra.utils.Throwables;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
+
 public class IsolatedExecutor implements IIsolatedExecutor
 {
     final ExecutorService isolatedExecutor;
@@ -78,7 +80,7 @@ public class IsolatedExecutor implements IIsolatedExecutor
             t.setDaemon(true);
             return t;
         };
-        ExecutorService shutdownExecutor = new ThreadPoolExecutor(0, Integer.MAX_VALUE, 0, TimeUnit.SECONDS,
+        ExecutorService shutdownExecutor = new ThreadPoolExecutor(0, Integer.MAX_VALUE, 0, SECONDS,
                                                                   new LinkedBlockingQueue<>(), threadFactory);
         return shutdownExecutor.submit(() -> {
             try
diff --git a/test/distributed/org/apache/cassandra/distributed/test/AbstractEncryptionOptionsImpl.java b/test/distributed/org/apache/cassandra/distributed/test/AbstractEncryptionOptionsImpl.java
index 24d0b75..0d12c7c 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/AbstractEncryptionOptionsImpl.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/AbstractEncryptionOptionsImpl.java
@@ -21,7 +21,6 @@ package org.apache.cassandra.distributed.test;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -29,6 +28,7 @@ import javax.net.ssl.SSLHandshakeException;
 import javax.net.ssl.SSLSession;
 
 import com.google.common.collect.ImmutableMap;
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,7 +52,12 @@ import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.security.ISslContextFactory;
 import org.apache.cassandra.security.SSLFactory;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.distributed.test.AbstractEncryptionOptionsImpl.ConnectResult.CONNECTING;
+import static org.apache.cassandra.distributed.test.AbstractEncryptionOptionsImpl.ConnectResult.UNINITIALIZED;
+import static org.apache.cassandra.security.SSLFactory.getOrCreateSslContext;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
 
 public class AbstractEncryptionOptionsImpl extends TestBaseImpl
 {
@@ -190,17 +195,17 @@ public class AbstractEncryptionOptionsImpl extends TestBaseImpl
         ConnectResult connect() throws Throwable
         {
             AtomicInteger connectAttempts = new AtomicInteger(0);
-            result.set(ConnectResult.UNINITIALIZED);
+            result.set(UNINITIALIZED);
             setLastThrowable(null);
             setProtocolAndCipher(null, null);
 
             SslContext sslContext = SSLFactory.getOrCreateSslContext(
-            encryptionOptions.withAcceptedProtocols(acceptedProtocols).withCipherSuites(cipherSuites),
-            true, ISslContextFactory.SocketType.CLIENT);
+                encryptionOptions.withAcceptedProtocols(acceptedProtocols).withCipherSuites(cipherSuites),
+                true, ISslContextFactory.SocketType.CLIENT);
 
             EventLoopGroup workerGroup = new NioEventLoopGroup();
             Bootstrap b = new Bootstrap();
-            SimpleCondition attemptCompleted = new SimpleCondition();
+            Condition attemptCompleted = newOneTimeCondition();
 
             // Listener on the SSL handshake makes sure that the test completes immediately as
             // the server waits to receive a message over the TLS connection, so the discardHandler.decode
@@ -297,12 +302,12 @@ public class AbstractEncryptionOptionsImpl extends TestBaseImpl
                 }
             });
 
-            result.set(ConnectResult.CONNECTING);
+            result.set(CONNECTING);
             ChannelFuture f = b.connect(host, port);
             try
             {
                 f.sync();
-                attemptCompleted.await(15, TimeUnit.SECONDS);
+                attemptCompleted.await(15, SECONDS);
             }
             finally
             {
diff --git a/test/distributed/org/apache/cassandra/distributed/test/GossipShutdownTest.java b/test/distributed/org/apache/cassandra/distributed/test/GossipShutdownTest.java
index ad21c21..d6a4188 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/GossipShutdownTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/GossipShutdownTest.java
@@ -20,29 +20,32 @@ package org.apache.cassandra.distributed.test;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.net.InetSocketAddress;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.junit.Test;
 
 import org.apache.cassandra.distributed.Cluster;
-import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.EndpointState;
-import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.gms.IEndpointStateChangeSubscriber;
 import org.apache.cassandra.gms.VersionedValue;
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.net.Verb;
 import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
 
+
+import static java.lang.Thread.sleep;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
 import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
 import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.apache.cassandra.gms.Gossiper.instance;
+import static org.apache.cassandra.net.Verb.GOSSIP_DIGEST_ACK;
+import static org.apache.cassandra.net.Verb.GOSSIP_DIGEST_SYN;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
 
 public class GossipShutdownTest extends TestBaseImpl
 {
@@ -61,27 +64,27 @@ public class GossipShutdownTest extends TestBaseImpl
             cluster.schemaChange("create table "+KEYSPACE+".tbl (id int primary key, v int)");
 
             for (int i = 0; i < 10; i++)
-                cluster.coordinator(1).execute("insert into "+KEYSPACE+".tbl (id, v) values (?,?)", ConsistencyLevel.ALL, i, i);
+                cluster.coordinator(1).execute("insert into "+KEYSPACE+".tbl (id, v) values (?,?)", ALL, i, i);
 
-            SimpleCondition timeToShutdown = new SimpleCondition();
-            SimpleCondition waitForShutdown = new SimpleCondition();
+            Condition timeToShutdown = newOneTimeCondition();
+            Condition waitForShutdown = newOneTimeCondition();
             AtomicBoolean signalled = new AtomicBoolean(false);
             Future f = es.submit(() -> {
                 await(timeToShutdown);
 
                 cluster.get(1).runOnInstance(() -> {
-                    Gossiper.instance.register(new EPChanges());
+                    instance.register(new EPChanges());
                 });
 
                 cluster.get(2).runOnInstance(() -> {
                     StorageService.instance.setIsShutdownUnsafeForTests(true);
-                    Gossiper.instance.stop();
+                    instance.stop();
                 });
                 waitForShutdown.signalAll();
             });
 
-            cluster.filters().outbound().from(2).to(1).verbs(Verb.GOSSIP_DIGEST_SYN.id).messagesMatching((from, to, message) -> true).drop();
-            cluster.filters().outbound().from(2).to(1).verbs(Verb.GOSSIP_DIGEST_ACK.id).messagesMatching((from, to, message) ->
+            cluster.filters().outbound().from(2).to(1).verbs(GOSSIP_DIGEST_SYN.id).messagesMatching((from, to, message) -> true).drop();
+            cluster.filters().outbound().from(2).to(1).verbs(GOSSIP_DIGEST_ACK.id).messagesMatching((from, to, message) ->
                                                                                                          {
                                                                                                              if (signalled.compareAndSet(false, true))
                                                                                                              {
@@ -92,7 +95,7 @@ public class GossipShutdownTest extends TestBaseImpl
                                                                                                              return true;
                                                                                                          }).drop();
 
-            Thread.sleep(10000); // wait for gossip to exchange a few messages
+            sleep(10000); // wait for gossip to exchange a few messages
             f.get();
         }
         finally
@@ -101,7 +104,7 @@ public class GossipShutdownTest extends TestBaseImpl
         }
     }
 
-    private static void await(SimpleCondition sc)
+    private static void await(Condition sc)
     {
         try
         {
diff --git a/test/distributed/org/apache/cassandra/distributed/test/IncRepairTruncationTest.java b/test/distributed/org/apache/cassandra/distributed/test/IncRepairTruncationTest.java
index 4ffdaa9..989e0ab 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/IncRepairTruncationTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/IncRepairTruncationTest.java
@@ -26,6 +26,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.junit.Test;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -33,14 +34,15 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.IMessage;
-import org.apache.cassandra.distributed.api.IMessageFilters;
 import org.apache.cassandra.distributed.api.NodeToolResult;
 import org.apache.cassandra.net.Verb;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
+
 
 import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
 import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.apache.cassandra.distributed.api.IMessageFilters.Matcher;
 import static org.apache.cassandra.distributed.test.PreviewRepairTest.insert;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
 
 public class IncRepairTruncationTest extends TestBaseImpl
 {
@@ -133,10 +135,10 @@ public class IncRepairTruncationTest extends TestBaseImpl
         }
     }
 
-    private static class BlockMessage implements IMessageFilters.Matcher
+    private static class BlockMessage implements Matcher
     {
-        private final SimpleCondition gotMessage = new SimpleCondition();
-        private final SimpleCondition allowMessage = new SimpleCondition();
+        private final Condition gotMessage = newOneTimeCondition();
+        private final Condition allowMessage = newOneTimeCondition();
 
         public boolean matches(int from, int to, IMessage message)
         {
diff --git a/test/distributed/org/apache/cassandra/distributed/test/PreviewRepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/PreviewRepairTest.java
index e57c1f8..667422a 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/PreviewRepairTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/PreviewRepairTest.java
@@ -33,8 +33,10 @@ import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import com.google.common.collect.ImmutableList;
 import com.google.common.util.concurrent.Uninterruptibles;
+
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -50,9 +52,7 @@ import org.apache.cassandra.distributed.api.ICoordinator;
 import org.apache.cassandra.distributed.api.IInvokableInstance;
 import org.apache.cassandra.distributed.api.IIsolatedExecutor;
 import org.apache.cassandra.distributed.api.IMessage;
-import org.apache.cassandra.distributed.api.IMessageFilters;
 import org.apache.cassandra.distributed.api.NodeToolResult;
-import org.apache.cassandra.distributed.impl.Instance;
 import org.apache.cassandra.distributed.shared.RepairResult;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.Verb;
@@ -63,14 +63,22 @@ import org.apache.cassandra.repair.messages.RepairMessage;
 import org.apache.cassandra.repair.messages.RepairOption;
 import org.apache.cassandra.repair.messages.ValidationRequest;
 import org.apache.cassandra.service.ActiveRepairService;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
-import org.apache.cassandra.utils.progress.ProgressEventType;
 
+import static com.google.common.collect.ImmutableList.of;
+import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
 import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.apache.cassandra.distributed.api.IMessageFilters.Matcher;
+import static org.apache.cassandra.distributed.impl.Instance.deserializeMessage;
+import static org.apache.cassandra.distributed.test.PreviewRepairTest.DelayFirstRepairTypeMessageFilter.finalizePropose;
+import static org.apache.cassandra.distributed.test.PreviewRepairTest.DelayFirstRepairTypeMessageFilter.validationRequest;
+import static org.apache.cassandra.net.Verb.FINALIZE_PROPOSE_MSG;
+import static org.apache.cassandra.net.Verb.VALIDATION_REQ;
+import static org.apache.cassandra.service.StorageService.instance;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
+import static org.apache.cassandra.utils.progress.ProgressEventType.*;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -168,11 +176,11 @@ public class PreviewRepairTest extends TestBaseImpl
             insert(cluster.coordinator(1), 100, 100);
             cluster.forEach((node) -> node.flush(KEYSPACE));
             
-            SimpleCondition previewRepairStarted = new SimpleCondition();
-            SimpleCondition continuePreviewRepair = new SimpleCondition();
-            DelayFirstRepairTypeMessageFilter filter = DelayFirstRepairTypeMessageFilter.validationRequest(previewRepairStarted, continuePreviewRepair);
+            Condition previewRepairStarted = newOneTimeCondition();
+            Condition continuePreviewRepair = newOneTimeCondition();
+            DelayFirstRepairTypeMessageFilter filter = validationRequest(previewRepairStarted, continuePreviewRepair);
             // this pauses the validation request sent from node1 to node2 until we have run a full inc repair below
-            cluster.filters().outbound().verbs(Verb.VALIDATION_REQ.id).from(1).to(2).messagesMatching(filter).drop();
+            cluster.filters().outbound().verbs(VALIDATION_REQ.id).from(1).to(2).messagesMatching(filter).drop();
 
             Future<RepairResult> rsFuture = es.submit(() -> cluster.get(1).callOnInstance(repair(options(true, false))));
             previewRepairStarted.await();
@@ -207,24 +215,24 @@ public class PreviewRepairTest extends TestBaseImpl
             insert(cluster.coordinator(1), 100, 100);
             cluster.forEach((node) -> node.flush(KEYSPACE));
 
-            SimpleCondition previewRepairStarted = new SimpleCondition();
-            SimpleCondition continuePreviewRepair = new SimpleCondition();
+            Condition previewRepairStarted = newOneTimeCondition();
+            Condition continuePreviewRepair = newOneTimeCondition();
             // this pauses the validation request sent from node1 to node2 until the inc repair below has run
             cluster.filters()
                    .outbound()
-                   .verbs(Verb.VALIDATION_REQ.id)
+                   .verbs(VALIDATION_REQ.id)
                    .from(1).to(2)
-                   .messagesMatching(DelayFirstRepairTypeMessageFilter.validationRequest(previewRepairStarted, continuePreviewRepair))
+                   .messagesMatching(validationRequest(previewRepairStarted, continuePreviewRepair))
                    .drop();
 
-            SimpleCondition irRepairStarted = new SimpleCondition();
-            SimpleCondition continueIrRepair = new SimpleCondition();
+            Condition irRepairStarted = newOneTimeCondition();
+            Condition continueIrRepair = newOneTimeCondition();
             // this blocks the IR from committing, so we can reenable the preview
             cluster.filters()
                    .outbound()
-                   .verbs(Verb.FINALIZE_PROPOSE_MSG.id)
+                   .verbs(FINALIZE_PROPOSE_MSG.id)
                    .from(1).to(2)
-                   .messagesMatching(DelayFirstRepairTypeMessageFilter.finalizePropose(irRepairStarted, continueIrRepair))
+                   .messagesMatching(finalizePropose(irRepairStarted, continueIrRepair))
                    .drop();
 
             Future<RepairResult> previewResult = cluster.get(1).asyncCallsOnInstance(repair(options(true, false))).call();
@@ -268,15 +276,15 @@ public class PreviewRepairTest extends TestBaseImpl
             cluster.forEach((node) -> node.flush(KEYSPACE));
 
             // pause preview repair validation messages on node2 until node1 has finished
-            SimpleCondition previewRepairStarted = new SimpleCondition();
-            SimpleCondition continuePreviewRepair = new SimpleCondition();
-            DelayFirstRepairTypeMessageFilter filter = DelayFirstRepairTypeMessageFilter.validationRequest(previewRepairStarted, continuePreviewRepair);
-            cluster.filters().outbound().verbs(Verb.VALIDATION_REQ.id).from(1).to(2).messagesMatching(filter).drop();
+            Condition previewRepairStarted = newOneTimeCondition();
+            Condition continuePreviewRepair = newOneTimeCondition();
+            DelayFirstRepairTypeMessageFilter filter = validationRequest(previewRepairStarted, continuePreviewRepair);
+            cluster.filters().outbound().verbs(VALIDATION_REQ.id).from(1).to(2).messagesMatching(filter).drop();
 
             // get local ranges to repair two separate ranges:
             List<String> localRanges = cluster.get(1).callOnInstance(() -> {
                 List<String> res = new ArrayList<>();
-                for (Range<Token> r : StorageService.instance.getLocalReplicas(KEYSPACE).ranges())
+                for (Range<Token> r : instance.getLocalReplicas(KEYSPACE).ranges())
                     res.add(r.left.getTokenValue()+ ":"+ r.right.getTokenValue());
                 return res;
             });
@@ -319,8 +327,8 @@ public class PreviewRepairTest extends TestBaseImpl
             cluster.forEach((node) -> node.flush(KEYSPACE));
 
             // pause inc repair validation messages on node2 until node1 has finished
-            SimpleCondition incRepairStarted = new SimpleCondition();
-            SimpleCondition continueIncRepair = new SimpleCondition();
+            Condition incRepairStarted = newOneTimeCondition();
+            Condition continueIncRepair = newOneTimeCondition();
 
             DelayFirstRepairTypeMessageFilter filter = DelayFirstRepairTypeMessageFilter.validationRequest(incRepairStarted, continueIncRepair);
             cluster.filters().outbound().verbs(Verb.VALIDATION_REQ.id).from(1).to(2).messagesMatching(filter).drop();
@@ -444,13 +452,13 @@ public class PreviewRepairTest extends TestBaseImpl
         }));
     }
 
-    static abstract class DelayFirstRepairMessageFilter implements IMessageFilters.Matcher
+    static abstract class DelayFirstRepairMessageFilter implements Matcher
     {
-        private final SimpleCondition pause;
-        private final SimpleCondition resume;
+        private final Condition pause;
+        private final Condition resume;
         private final AtomicBoolean waitForRepair = new AtomicBoolean(true);
 
-        protected DelayFirstRepairMessageFilter(SimpleCondition pause, SimpleCondition resume)
+        protected DelayFirstRepairMessageFilter(Condition pause, Condition resume)
         {
             this.pause = pause;
             this.resume = resume;
@@ -462,7 +470,7 @@ public class PreviewRepairTest extends TestBaseImpl
         {
             try
             {
-                Message<?> msg = Instance.deserializeMessage(message);
+                Message<?> msg = deserializeMessage(message);
                 RepairMessage repairMessage = (RepairMessage) msg.payload;
                 // only the first message should be delayed:
                 if (matchesMessage(repairMessage) && waitForRepair.compareAndSet(true, false))
@@ -483,18 +491,18 @@ public class PreviewRepairTest extends TestBaseImpl
     {
         private final Class<? extends RepairMessage> type;
 
-        public DelayFirstRepairTypeMessageFilter(SimpleCondition pause, SimpleCondition resume, Class<? extends RepairMessage> type)
+        public DelayFirstRepairTypeMessageFilter(Condition pause, Condition resume, Class<? extends RepairMessage> type)
         {
             super(pause, resume);
             this.type = type;
         }
 
-        public static DelayFirstRepairTypeMessageFilter validationRequest(SimpleCondition pause, SimpleCondition resume)
+        public static DelayFirstRepairTypeMessageFilter validationRequest(Condition pause, Condition resume)
         {
             return new DelayFirstRepairTypeMessageFilter(pause, resume, ValidationRequest.class);
         }
 
-        public static DelayFirstRepairTypeMessageFilter finalizePropose(SimpleCondition pause, SimpleCondition resume)
+        public static DelayFirstRepairTypeMessageFilter finalizePropose(Condition pause, Condition resume)
         {
             return new DelayFirstRepairTypeMessageFilter(pause, resume, FinalizePropose.class);
         }
@@ -522,25 +530,25 @@ public class PreviewRepairTest extends TestBaseImpl
     private static IIsolatedExecutor.SerializableCallable<RepairResult> repair(Map<String, String> options)
     {
         return () -> {
-            SimpleCondition await = new SimpleCondition();
+            Condition await = newOneTimeCondition();
             AtomicBoolean success = new AtomicBoolean(true);
             AtomicBoolean wasInconsistent = new AtomicBoolean(false);
-            StorageService.instance.repair(KEYSPACE, options, ImmutableList.of((tag, event) -> {
-                if (event.getType() == ProgressEventType.ERROR)
+            instance.repair(KEYSPACE, options, of((tag, event) -> {
+                if (event.getType() == ERROR)
                 {
                     success.set(false);
                     await.signalAll();
                 }
-                else if (event.getType() == ProgressEventType.NOTIFICATION && event.getMessage().contains("Repaired data is inconsistent"))
+                else if (event.getType() == NOTIFICATION && event.getMessage().contains("Repaired data is inconsistent"))
                 {
                     wasInconsistent.set(true);
                 }
-                else if (event.getType() == ProgressEventType.COMPLETE)
+                else if (event.getType() == COMPLETE)
                     await.signalAll();
             }));
             try
             {
-                await.await(1, TimeUnit.MINUTES);
+                await.await(1, MINUTES);
             }
             catch (InterruptedException e)
             {
diff --git a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairTest.java
index c68320e..2718609 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairTest.java
@@ -26,6 +26,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -36,7 +37,6 @@ import net.bytebuddy.implementation.bind.annotation.SuperCall;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
@@ -52,14 +52,14 @@ import org.apache.cassandra.distributed.shared.NetworkTopology;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.locator.ReplicaPlan;
-import org.apache.cassandra.net.Verb;
 import org.apache.cassandra.service.PendingRangeCalculatorService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.service.reads.repair.BlockingReadRepair;
 import org.apache.cassandra.service.reads.repair.ReadRepairStrategy;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
 
 import static net.bytebuddy.matcher.ElementMatchers.named;
+
+import static org.apache.cassandra.db.Keyspace.open;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
 import static org.apache.cassandra.distributed.shared.AssertUtils.assertEquals;
@@ -68,6 +68,7 @@ import static org.apache.cassandra.distributed.shared.AssertUtils.row;
 import static org.apache.cassandra.net.Verb.READ_REPAIR_REQ;
 import static org.apache.cassandra.net.Verb.READ_REPAIR_RSP;
 import static org.apache.cassandra.net.Verb.READ_REQ;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
 import static org.junit.Assert.fail;
 
 public class ReadRepairTest extends TestBaseImpl
@@ -365,7 +366,7 @@ public class ReadRepairTest extends TestBaseImpl
                                  "    PRIMARY KEY (key, column1)\n" +
                                  ") WITH CLUSTERING ORDER BY (column1 ASC)");
 
-            cluster.forEach(i -> i.runOnInstance(() -> Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").disableAutoCompaction()));
+            cluster.forEach(i -> i.runOnInstance(() -> open(KEYSPACE).getColumnFamilyStore("tbl").disableAutoCompaction()));
 
             for (int i = 1; i <= 2; i++)
             {
@@ -378,9 +379,9 @@ public class ReadRepairTest extends TestBaseImpl
             cluster.get(3).flush(KEYSPACE);
 
             // pause the read until we have bootstrapped a new node below
-            SimpleCondition continueRead = new SimpleCondition();
-            SimpleCondition readStarted = new SimpleCondition();
-            cluster.filters().outbound().from(3).to(1,2).verbs(Verb.READ_REQ.id).messagesMatching((i, i1, iMessage) -> {
+            Condition continueRead = newOneTimeCondition();
+            Condition readStarted = newOneTimeCondition();
+            cluster.filters().outbound().from(3).to(1,2).verbs(READ_REQ.id).messagesMatching((i, i1, iMessage) -> {
                 try
                 {
                     readStarted.signalAll();
@@ -394,7 +395,7 @@ public class ReadRepairTest extends TestBaseImpl
             }).drop();
             Future<Object[][]> read = es.submit(() -> cluster.coordinator(3)
                                                           .execute("SELECT * FROM distributed_test_keyspace.tbl WHERE key=? and column1 >= ? and column1 <= ?",
-                                                                   ConsistencyLevel.ALL, key, 20, 40));
+                                                                   ALL, key, 20, 40));
             readStarted.await();
             IInstanceConfig config = cluster.newInstanceConfig();
             config.set("auto_bootstrap", true);
diff --git a/test/distributed/org/apache/cassandra/distributed/test/RepairBoundaryTest.java b/test/distributed/org/apache/cassandra/distributed/test/RepairBoundaryTest.java
index d269bef..d7cd0ec 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/RepairBoundaryTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/RepairBoundaryTest.java
@@ -23,7 +23,7 @@ import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
 
-import com.google.common.collect.ImmutableList;
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -31,16 +31,17 @@ import org.junit.Test;
 import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.IInvokableInstance;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
-import org.apache.cassandra.utils.progress.ProgressEventType;
 
+import static com.google.common.collect.ImmutableList.of;
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.cassandra.dht.Murmur3Partitioner.*;
 import static org.apache.cassandra.dht.Murmur3Partitioner.LongToken.keyForToken;
 import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
 import static org.apache.cassandra.distributed.api.Feature.NETWORK;
 import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.service.StorageService.instance;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
+import static org.apache.cassandra.utils.progress.ProgressEventType.COMPLETE;
 
 public class RepairBoundaryTest extends TestBaseImpl
 {
@@ -142,9 +143,9 @@ public class RepairBoundaryTest extends TestBaseImpl
                 Map<String, String> options = new HashMap<>();
                 options.put("ranges", "999:1000");
                 options.put("incremental", "false");
-                SimpleCondition await = new SimpleCondition();
-                StorageService.instance.repair(KEYSPACE, options, ImmutableList.of((tag, event) -> {
-                    if (event.getType() == ProgressEventType.COMPLETE)
+                Condition await = newOneTimeCondition();
+                instance.repair(KEYSPACE, options, of((tag, event) -> {
+                    if (event.getType() == COMPLETE)
                         await.signalAll();
                 })).right.get();
                 await.await(1L, MINUTES);
diff --git a/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java
index b127a74..555bfb4 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java
@@ -25,8 +25,8 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
 
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import org.apache.cassandra.utils.concurrent.Condition;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -37,14 +37,16 @@ import org.apache.cassandra.distributed.api.ICluster;
 import org.apache.cassandra.distributed.api.IInstanceConfig;
 import org.apache.cassandra.distributed.api.IInvokableInstance;
 import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
-import org.apache.cassandra.utils.progress.ProgressEventType;
 
+import static com.google.common.collect.ImmutableList.of;
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
 import static org.apache.cassandra.distributed.api.Feature.NETWORK;
 import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
 import static org.apache.cassandra.distributed.test.ExecUtil.rethrow;
+import static org.apache.cassandra.service.StorageService.instance;
+import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
+import static org.apache.cassandra.utils.progress.ProgressEventType.COMPLETE;
 
 public class RepairTest extends TestBaseImpl
 {
@@ -95,9 +97,9 @@ public class RepairTest extends TestBaseImpl
     static void repair(ICluster<IInvokableInstance> cluster, String keyspace, Map<String, String> options)
     {
         cluster.get(1).runOnInstance(rethrow(() -> {
-            SimpleCondition await = new SimpleCondition();
-            StorageService.instance.repair(keyspace, options, ImmutableList.of((tag, event) -> {
-                if (event.getType() == ProgressEventType.COMPLETE)
+            Condition await = newOneTimeCondition();
+            instance.repair(keyspace, options, of((tag, event) -> {
+                if (event.getType() == COMPLETE)
                     await.signalAll();
             })).right.get();
             await.await(1L, MINUTES);
diff --git a/test/distributed/org/apache/cassandra/distributed/test/StreamingTest.java b/test/distributed/org/apache/cassandra/distributed/test/StreamingTest.java
index b68d268..8fff004 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/StreamingTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/StreamingTest.java
@@ -34,7 +34,6 @@ import com.google.common.annotations.VisibleForTesting;
 import org.junit.Assert;
 import org.junit.Test;
 
-import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.distributed.api.IInvokableInstance;
 import org.apache.cassandra.locator.InetAddressAndPort;
diff --git a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
index 3de764a..399f3a9 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
@@ -29,7 +29,6 @@ import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
@@ -40,7 +39,6 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.CompactionParams;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.service.ActiveRepairService;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertFalse;
diff --git a/test/long/org/apache/cassandra/dht/tokenallocator/RandomReplicationAwareTokenAllocatorTest.java b/test/long/org/apache/cassandra/dht/tokenallocator/RandomReplicationAwareTokenAllocatorTest.java
index 6a2d59e..bb1a2c8 100644
--- a/test/long/org/apache/cassandra/dht/tokenallocator/RandomReplicationAwareTokenAllocatorTest.java
+++ b/test/long/org/apache/cassandra/dht/tokenallocator/RandomReplicationAwareTokenAllocatorTest.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.dht.tokenallocator;
 
 import org.junit.Test;
 
-import org.apache.cassandra.Util;
 import org.apache.cassandra.dht.RandomPartitioner;
 
 public class RandomReplicationAwareTokenAllocatorTest extends AbstractReplicationAwareTokenAllocatorTest
diff --git a/test/microbench/org/apache/cassandra/test/microbench/BTreeSearchIteratorBench.java b/test/microbench/org/apache/cassandra/test/microbench/BTreeSearchIteratorBench.java
index 400b297..d7cf8dc 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/BTreeSearchIteratorBench.java
+++ b/test/microbench/org/apache/cassandra/test/microbench/BTreeSearchIteratorBench.java
@@ -36,7 +36,6 @@ import org.openjdk.jmh.annotations.Fork;
 import org.openjdk.jmh.annotations.Level;
 import org.openjdk.jmh.annotations.Measurement;
 import org.openjdk.jmh.annotations.Mode;
-import org.openjdk.jmh.annotations.OperationsPerInvocation;
 import org.openjdk.jmh.annotations.OutputTimeUnit;
... 2100 lines suppressed ...

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


[cassandra] 04/06: [CASSANDRA-16928] CEP-10 Phase 1: InetAddressAndPort extends InetSocketAddress

Posted by be...@apache.org.
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 aae7e8b07c84476de893b473a13cdc6d9e260573
Author: Benedict Elliott Smith <be...@apache.org>
AuthorDate: Fri Apr 16 12:01:25 2021 +0100

    [CASSANDRA-16928] CEP-10 Phase 1: InetAddressAndPort extends InetSocketAddress
    
    patch by Benedict; reviewed by Sam Tunnicliffe, Caleb Rackliffe and Aleksei Zotov for CASSANDRA-16928
---
 .../org/apache/cassandra/audit/AuditLogEntry.java  |   6 +-
 .../org/apache/cassandra/db/SystemKeyspace.java    |  37 +++----
 .../db/virtual/InternodeInboundTable.java          |   2 +-
 .../db/virtual/InternodeOutboundTable.java         |   2 +-
 .../dht/tokenallocator/OfflineTokenAllocator.java  |   2 +-
 .../cassandra/locator/DynamicEndpointSnitch.java   |   2 +-
 .../cassandra/locator/Ec2MultiRegionSnitch.java    |   2 +-
 .../apache/cassandra/locator/IEndpointSnitch.java  |   6 ++
 .../cassandra/locator/InetAddressAndPort.java      | 113 ++++++++++++---------
 .../cassandra/locator/RackInferringSnitch.java     |   4 +-
 .../cassandra/net/InboundConnectionInitiator.java  |   2 +-
 .../cassandra/net/InboundConnectionSettings.java   |   8 +-
 .../cassandra/net/OutboundConnectionInitiator.java |   4 +-
 .../cassandra/net/OutboundConnectionSettings.java  |   2 +-
 .../org/apache/cassandra/net/SocketFactory.java    |   2 +-
 .../repair/SystemDistributedKeyspace.java          |   2 +-
 .../repair/consistent/LocalSessionInfo.java        |   2 +-
 .../cassandra/repair/consistent/LocalSessions.java |   6 +-
 .../cassandra/schema/SchemaAnnouncementEvent.java  |   4 +-
 .../apache/cassandra/service/StorageService.java   |  29 +++---
 .../cassandra/service/TruncateResponseHandler.java |   2 +-
 .../service/reads/repair/ReadRepairEvent.java      |   4 +-
 .../management/ProgressInfoCompositeData.java      |   4 +-
 .../SessionCompleteEventCompositeData.java         |   4 +-
 .../management/SessionInfoCompositeData.java       |   8 +-
 .../cassandra/tools/nodetool/HostStatWithPort.java |   4 +-
 .../apache/cassandra/tracing/TraceKeyspace.java    |   8 +-
 src/java/org/apache/cassandra/transport/Event.java |  10 +-
 .../org/apache/cassandra/transport/Server.java     |   2 +-
 .../cassandra/transport/messages/ErrorMessage.java |   4 +-
 .../org/apache/cassandra/utils/FBUtilities.java    |   2 +-
 src/java/org/apache/cassandra/utils/Mx4jTool.java  |   2 +-
 src/java/org/apache/cassandra/utils/UUIDGen.java   |   2 +-
 .../distributed/impl/DistributedTestSnitch.java    |   2 +-
 .../cassandra/distributed/impl/Instance.java       |   4 +-
 .../cassandra/distributed/test/StreamingTest.java  |   8 +-
 .../cassandra/audit/AuditLoggerAuthTest.java       |   4 +-
 .../cassandra/dht/RangeFetchMapCalculatorTest.java |   2 +-
 .../org/apache/cassandra/gms/GossiperTest.java     |   6 +-
 .../locator/InetAddressAndPortSerializerTest.java  |   4 +-
 .../cassandra/locator/InetAddressAndPortTest.java  |  16 +--
 .../apache/cassandra/net/ForwardingInfoTest.java   |   4 +-
 .../apache/cassandra/net/MessagingServiceTest.java |  12 +--
 .../org/apache/cassandra/repair/RepairJobTest.java |   8 +-
 .../service/WriteResponseHandlerTest.java          |   2 +-
 .../service/WriteResponseHandlerTransientTest.java |   2 +-
 .../cassandra/transport/CQLUserAuditTest.java      |   6 +-
 47 files changed, 200 insertions(+), 173 deletions(-)

diff --git a/src/java/org/apache/cassandra/audit/AuditLogEntry.java b/src/java/org/apache/cassandra/audit/AuditLogEntry.java
index 3a015c5..02db076 100644
--- a/src/java/org/apache/cassandra/audit/AuditLogEntry.java
+++ b/src/java/org/apache/cassandra/audit/AuditLogEntry.java
@@ -76,10 +76,10 @@ public class AuditLogEntry
         StringBuilder builder = new StringBuilder(100);
         builder.append("user:").append(user)
                .append("|host:").append(host)
-               .append("|source:").append(source.address);
-        if (source.port > 0)
+               .append("|source:").append(source.getAddress());
+        if (source.getPort() > 0)
         {
-            builder.append("|port:").append(source.port);
+            builder.append("|port:").append(source.getPort());
         }
 
         builder.append("|timestamp:").append(timestamp)
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index a7cbe40..7e8c2b5 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db;
 import java.io.IOError;
 import java.io.IOException;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.time.Instant;
 import java.util.*;
@@ -36,7 +37,7 @@ 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.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -716,9 +717,9 @@ public final class SystemKeyspace
             return;
 
         String req = "INSERT INTO system.%s (peer, tokens) VALUES (?, ?)";
-        executeInternal(String.format(req, LEGACY_PEERS), ep.address, tokensAsSet(tokens));
+        executeInternal(String.format(req, LEGACY_PEERS), ep.getAddress(), tokensAsSet(tokens));
         req = "INSERT INTO system.%s (peer, peer_port, tokens) VALUES (?, ?, ?)";
-        executeInternal(String.format(req, PEERS_V2), ep.address, ep.port, tokensAsSet(tokens));
+        executeInternal(String.format(req, PEERS_V2), ep.getAddress(), ep.getPort(), tokensAsSet(tokens));
     }
 
     public static synchronized boolean updatePreferredIP(InetAddressAndPort ep, InetAddressAndPort preferred_ip)
@@ -727,9 +728,9 @@ public final class SystemKeyspace
             return false;
 
         String req = "INSERT INTO system.%s (peer, preferred_ip) VALUES (?, ?)";
-        executeInternal(String.format(req, LEGACY_PEERS), ep.address, preferred_ip.address);
+        executeInternal(String.format(req, LEGACY_PEERS), ep.getAddress(), preferred_ip.getAddress());
         req = "INSERT INTO system.%s (peer, peer_port, preferred_ip, preferred_port) VALUES (?, ?, ?, ?)";
-        executeInternal(String.format(req, PEERS_V2), ep.address, ep.port, preferred_ip.address, preferred_ip.port);
+        executeInternal(String.format(req, PEERS_V2), ep.getAddress(), ep.getPort(), preferred_ip.getAddress(), preferred_ip.getPort());
         forceBlockingFlush(LEGACY_PEERS, PEERS_V2);
         return true;
     }
@@ -740,14 +741,14 @@ public final class SystemKeyspace
             return;
 
         String req = "INSERT INTO system.%s (peer, %s) VALUES (?, ?)";
-        executeInternal(String.format(req, LEGACY_PEERS, columnName), ep.address, value);
+        executeInternal(String.format(req, LEGACY_PEERS, columnName), ep.getAddress(), value);
         //This column doesn't match across the two tables
         if (columnName.equals("rpc_address"))
         {
             columnName = "native_address";
         }
         req = "INSERT INTO system.%s (peer, peer_port, %s) VALUES (?, ?, ?)";
-        executeInternal(String.format(req, PEERS_V2, columnName), ep.address, ep.port, value);
+        executeInternal(String.format(req, PEERS_V2, columnName), ep.getAddress(), ep.getPort(), value);
     }
 
     public static synchronized void updatePeerNativeAddress(InetAddressAndPort ep, InetAddressAndPort address)
@@ -756,9 +757,9 @@ public final class SystemKeyspace
             return;
 
         String req = "INSERT INTO system.%s (peer, rpc_address) VALUES (?, ?)";
-        executeInternal(String.format(req, LEGACY_PEERS), ep.address, address.address);
+        executeInternal(String.format(req, LEGACY_PEERS), ep.getAddress(), address.getAddress());
         req = "INSERT INTO system.%s (peer, peer_port, native_address, native_port) VALUES (?, ?, ?, ?)";
-        executeInternal(String.format(req, PEERS_V2), ep.address, ep.port, address.address, address.port);
+        executeInternal(String.format(req, PEERS_V2), ep.getAddress(), ep.getPort(), address.getAddress(), address.getPort());
     }
 
 
@@ -766,9 +767,9 @@ public final class SystemKeyspace
     {
         // with 30 day TTL
         String req = "UPDATE system.%s USING TTL 2592000 SET hints_dropped[ ? ] = ? WHERE peer = ?";
-        executeInternal(String.format(req, LEGACY_PEER_EVENTS), timePeriod, value, ep.address);
+        executeInternal(String.format(req, LEGACY_PEER_EVENTS), timePeriod, value, ep.getAddress());
         req = "UPDATE system.%s USING TTL 2592000 SET hints_dropped[ ? ] = ? WHERE peer = ? AND peer_port = ?";
-        executeInternal(String.format(req, PEER_EVENTS_V2), timePeriod, value, ep.address, ep.port);
+        executeInternal(String.format(req, PEER_EVENTS_V2), timePeriod, value, ep.getAddress(), ep.getPort());
     }
 
     public static synchronized void updateSchemaVersion(UUID version)
@@ -800,12 +801,12 @@ public final class SystemKeyspace
     /**
      * Remove stored tokens being used by another node
      */
-    public static synchronized void removeEndpoint(InetAddressAndPort ep)
+    public static synchronized void removeEndpoint(InetSocketAddress ep)
     {
         String req = "DELETE FROM system.%s WHERE peer = ?";
-        executeInternal(String.format(req, LEGACY_PEERS), ep.address);
+        executeInternal(String.format(req, LEGACY_PEERS), ep.getAddress());
         req = String.format("DELETE FROM system.%s WHERE peer = ? AND peer_port = ?", PEERS_V2);
-        executeInternal(req, ep.address, ep.port);
+        executeInternal(req, ep.getAddress(), ep.getPort());
         forceBlockingFlush(LEGACY_PEERS, PEERS_V2);
     }
 
@@ -887,7 +888,7 @@ public final class SystemKeyspace
     public static InetAddressAndPort getPreferredIP(InetAddressAndPort ep)
     {
         String req = "SELECT preferred_ip, preferred_port FROM system.%s WHERE peer=? AND peer_port = ?";
-        UntypedResultSet result = executeInternal(String.format(req, PEERS_V2), ep.address, ep.port);
+        UntypedResultSet result = executeInternal(String.format(req, PEERS_V2), ep.getAddress(), ep.getPort());
         if (!result.isEmpty() && result.one().has("preferred_ip"))
         {
             UntypedResultSet.Row row = result.one();
@@ -934,7 +935,7 @@ public final class SystemKeyspace
                 return CURRENT_VERSION;
             }
             String req = "SELECT release_version FROM system.%s WHERE peer=? AND peer_port=?";
-            UntypedResultSet result = executeInternal(String.format(req, PEERS_V2), ep.address, ep.port);
+            UntypedResultSet result = executeInternal(String.format(req, PEERS_V2), ep.getAddress(), ep.getPort());
             if (result != null && result.one().has("release_version"))
             {
                 return new CassandraVersion(result.one().getString("release_version"));
@@ -1428,9 +1429,9 @@ public final class SystemKeyspace
         {
             rangesToUpdate.add(rangeToBytes(range));
         }
-        executeInternal(format(cql, LEGACY_TRANSFERRED_RANGES), rangesToUpdate, streamOperation.getDescription(), peer.address, keyspace);
+        executeInternal(format(cql, LEGACY_TRANSFERRED_RANGES), rangesToUpdate, streamOperation.getDescription(), peer.getAddress(), keyspace);
         cql = "UPDATE system.%s SET ranges = ranges + ? WHERE operation = ? AND peer = ? AND peer_port = ? AND keyspace_name = ?";
-        executeInternal(String.format(cql, TRANSFERRED_RANGES_V2), rangesToUpdate, streamOperation.getDescription(), peer.address, peer.port, keyspace);
+        executeInternal(String.format(cql, TRANSFERRED_RANGES_V2), rangesToUpdate, streamOperation.getDescription(), peer.getAddress(), peer.getPort(), keyspace);
     }
 
     public static synchronized Map<InetAddressAndPort, Set<Range<Token>>> getTransferredRanges(String description, String keyspace, IPartitioner partitioner)
diff --git a/src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java b/src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java
index b0afe8f..0da5870 100644
--- a/src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java
+++ b/src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java
@@ -114,7 +114,7 @@ public final class InternodeInboundTable extends AbstractVirtualTable
     {
         String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(addressAndPort);
         String rack = DatabaseDescriptor.getEndpointSnitch().getRack(addressAndPort);
-        dataSet.row(addressAndPort.address, addressAndPort.port, dc, rack)
+        dataSet.row(addressAndPort.getAddress(), addressAndPort.getPort(), dc, rack)
                .column(USING_BYTES, handlers.usingCapacity())
                .column(USING_RESERVE_BYTES, handlers.usingEndpointReserveCapacity())
                .column(CORRUPT_FRAMES_RECOVERED, handlers.corruptFramesRecovered())
diff --git a/src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java b/src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java
index 87b3823..687f845 100644
--- a/src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java
+++ b/src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java
@@ -115,7 +115,7 @@ public final class InternodeOutboundTable extends AbstractVirtualTable
         String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(addressAndPort);
         String rack = DatabaseDescriptor.getEndpointSnitch().getRack(addressAndPort);
         long pendingBytes = sum(connections, OutboundConnection::pendingBytes);
-        dataSet.row(addressAndPort.address, addressAndPort.port, dc, rack)
+        dataSet.row(addressAndPort.getAddress(), addressAndPort.getPort(), dc, rack)
                .column(USING_BYTES, pendingBytes)
                .column(USING_RESERVE_BYTES, connections.usingReserveBytes())
                .column(PENDING_COUNT, sum(connections, OutboundConnection::pendingCount))
diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java b/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java
index dc7f407..6a382c0 100644
--- a/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java
+++ b/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java
@@ -98,7 +98,7 @@ public class OfflineTokenAllocator
 
         public int nodeId()
         {
-            return fakeAddressAndPort.port;
+            return fakeAddressAndPort.getPort();
         }
 
         public int rackId()
diff --git a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
index cdb2236..2248248 100644
--- a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
+++ b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
@@ -320,7 +320,7 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements Lat
 
     public Map<InetAddress, Double> getScores()
     {
-        return scores.entrySet().stream().collect(Collectors.toMap(address -> address.getKey().address, Map.Entry::getValue));
+        return scores.entrySet().stream().collect(Collectors.toMap(address -> address.getKey().getAddress(), Map.Entry::getValue));
     }
 
     public Map<String, Double> getScoresWithPort()
diff --git a/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java b/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
index f9de755..45c387d 100644
--- a/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
+++ b/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
@@ -73,7 +73,7 @@ public class Ec2MultiRegionSnitch extends Ec2Snitch
             throw new RuntimeException(e);
         }
         Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_ADDRESS_AND_PORT, StorageService.instance.valueFactory.internalAddressAndPort(address));
-        Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_IP, StorageService.instance.valueFactory.internalIP(address.address));
+        Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_IP, StorageService.instance.valueFactory.internalIP(address.getAddress()));
         Gossiper.instance.register(new ReconnectableSnitchHelper(this, ec2region, true));
     }
 }
diff --git a/src/java/org/apache/cassandra/locator/IEndpointSnitch.java b/src/java/org/apache/cassandra/locator/IEndpointSnitch.java
index 381a642..0120391 100644
--- a/src/java/org/apache/cassandra/locator/IEndpointSnitch.java
+++ b/src/java/org/apache/cassandra/locator/IEndpointSnitch.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.locator;
 
+import java.net.InetSocketAddress;
 import java.util.Set;
 
 import org.apache.cassandra.utils.FBUtilities;
@@ -55,6 +56,11 @@ public interface IEndpointSnitch
         return getDatacenter(FBUtilities.getBroadcastAddressAndPort());
     }
 
+    default String getDatacenter(InetSocketAddress endpoint)
+    {
+        return getDatacenter(InetAddressAndPort.getByAddress(endpoint));
+    }
+
     default public String getDatacenter(Replica replica)
     {
         return getDatacenter(replica.endpoint());
diff --git a/src/java/org/apache/cassandra/locator/InetAddressAndPort.java b/src/java/org/apache/cassandra/locator/InetAddressAndPort.java
index 6e67a23..91c9fd1 100644
--- a/src/java/org/apache/cassandra/locator/InetAddressAndPort.java
+++ b/src/java/org/apache/cassandra/locator/InetAddressAndPort.java
@@ -22,6 +22,7 @@ import java.io.Serializable;
 import java.net.Inet4Address;
 import java.net.Inet6Address;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.util.regex.Pattern;
@@ -51,7 +52,7 @@ import org.apache.cassandra.utils.FastByteOperations;
  *
  */
 @SuppressWarnings("UnstableApiUsage")
-public final class InetAddressAndPort implements Comparable<InetAddressAndPort>, Serializable
+public final class InetAddressAndPort extends InetSocketAddress implements Comparable<InetAddressAndPort>, Serializable
 {
     private static final long serialVersionUID = 0;
 
@@ -61,23 +62,20 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
     //to always override the defaults.
     static volatile int defaultPort = 7000;
 
-    public final InetAddress address;
     public final byte[] addressBytes;
-    public final int port;
 
     private InetAddressAndPort(InetAddress address, byte[] addressBytes, int port)
     {
+        super(address, port);
         Preconditions.checkNotNull(address);
         Preconditions.checkNotNull(addressBytes);
         validatePortRange(port);
-        this.address = address;
-        this.port = port;
         this.addressBytes = addressBytes;
     }
 
     public InetAddressAndPort withPort(int port)
     {
-        return new InetAddressAndPort(address, addressBytes, port);
+        return new InetAddressAndPort(getAddress(), addressBytes, port);
     }
 
     private static void validatePortRange(int port)
@@ -89,26 +87,6 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
     }
 
     @Override
-    public boolean equals(Object o)
-    {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        InetAddressAndPort that = (InetAddressAndPort) o;
-
-        if (port != that.port) return false;
-        return address.equals(that.address);
-    }
-
-    @Override
-    public int hashCode()
-    {
-        int result = address.hashCode();
-        result = 31 * result + port;
-        return result;
-    }
-
-    @Override
     public int compareTo(InetAddressAndPort o)
     {
         int retval = FastByteOperations.compareUnsigned(addressBytes, 0, addressBytes.length, o.addressBytes, 0, o.addressBytes.length);
@@ -117,7 +95,7 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
             return retval;
         }
 
-        return Integer.compare(port, o.port);
+        return Integer.compare(getPort(), o.getPort());
     }
 
     public String getHostAddressAndPort()
@@ -141,31 +119,51 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
 
     public String getHostAddress(boolean withPort)
     {
+        return hostAddress(this, withPort);
+    }
+
+    public static String hostAddressAndPort(InetSocketAddress address)
+    {
+        return hostAddress(address, true);
+    }
+
+    public static String hostAddress(InetSocketAddress address, boolean withPort)
+    {
         if (withPort)
         {
-            return HostAndPort.fromParts(address.getHostAddress(), port).toString();
+            return HostAndPort.fromParts(address.getAddress().getHostAddress(), address.getPort()).toString();
         }
         else
         {
-            return address.getHostAddress();
+            return address.getAddress().getHostAddress();
         }
     }
 
     @Override
     public String toString()
     {
-        return toString(true);
+        return toString(this);
     }
 
     public String toString(boolean withPort)
     {
+        return toString(this, withPort);
+    }
+
+    public static String toString(InetSocketAddress address)
+    {
+        return toString(address, true);
+    }
+
+    public static String toString(InetSocketAddress address, boolean withPort)
+    {
         if (withPort)
         {
-            return toString(address, port);
+            return toString(address.getAddress(), address.getPort());
         }
         else
         {
-            return address.toString();
+            return address.getAddress().toString();
         }
     }
 
@@ -237,6 +235,13 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
         return getByAddressOverrideDefaults(address, null);
     }
 
+    public static InetAddressAndPort getByAddress(InetSocketAddress address)
+    {
+        if (address instanceof InetAddressAndPort)
+            return (InetAddressAndPort) address;
+        return new InetAddressAndPort(address.getAddress(), address.getAddress().getAddress(), address.getPort());
+    }
+
     public static InetAddressAndPort getByAddressOverrideDefaults(InetAddress address, Integer port)
     {
         if (port == null)
@@ -296,18 +301,27 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
 
         public void serialize(InetAddressAndPort endpoint, DataOutputPlus out, int version) throws IOException
         {
-            byte[] buf = endpoint.addressBytes;
+            serialize(endpoint.addressBytes, endpoint.getPort(), out, version);
+        }
 
+        public void serialize(InetSocketAddress endpoint, DataOutputPlus out, int version) throws IOException
+        {
+            byte[] address = endpoint instanceof InetAddressAndPort ? ((InetAddressAndPort) endpoint).addressBytes : endpoint.getAddress().getAddress();
+            serialize(address, endpoint.getPort(), out, version);
+        }
+
+        void serialize(byte[] address, int port, DataOutputPlus out, int version) throws IOException
+        {
             if (version >= MessagingService.VERSION_40)
             {
-                out.writeByte(buf.length + 2);
-                out.write(buf);
-                out.writeShort(endpoint.port);
+                out.writeByte(address.length + 2);
+                out.write(address);
+                out.writeShort(port);
             }
             else
             {
-                out.writeByte(buf.length);
-                out.write(buf);
+                out.writeByte(address.length);
+                out.write(address);
             }
         }
 
@@ -366,19 +380,24 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
 
         public long serializedSize(InetAddressAndPort from, int version)
         {
+            return serializedSize((InetSocketAddress) from, version);
+        }
+
+        public long serializedSize(InetSocketAddress from, int version)
+        {
             //4.0 includes a port number
             if (version >= MessagingService.VERSION_40)
             {
-                if (from.address instanceof Inet4Address)
+                if (from.getAddress() instanceof Inet4Address)
                     return 1 + 4 + 2;
-                assert from.address instanceof Inet6Address;
+                assert from.getAddress() instanceof Inet6Address;
                 return 1 + 16 + 2;
             }
             else
             {
-                if (from.address instanceof Inet4Address)
+                if (from.getAddress() instanceof Inet4Address)
                     return 1 + 4;
-                assert from.address instanceof Inet6Address;
+                assert from.getAddress() instanceof Inet6Address;
                 return 1 + 16;
             }
         }
@@ -400,7 +419,7 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
             {
                 out.writeByte(buf.length + 2);
                 out.write(buf);
-                out.writeShort(endpoint.port);
+                out.writeShort(endpoint.getPort());
             }
             else
             {
@@ -413,16 +432,16 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
             //4.0 includes a port number
             if (version >= MessagingService.VERSION_40)
             {
-                if (from.address instanceof Inet4Address)
+                if (from.getAddress() instanceof Inet4Address)
                     return 1 + 4 + 2;
-                assert from.address instanceof Inet6Address;
+                assert from.getAddress() instanceof Inet6Address;
                 return 1 + 16 + 2;
             }
             else
             {
-                if (from.address instanceof Inet4Address)
+                if (from.getAddress() instanceof Inet4Address)
                     return 4;
-                assert from.address instanceof Inet6Address;
+                assert from.getAddress() instanceof Inet6Address;
                 return 16;
             }
         }
diff --git a/src/java/org/apache/cassandra/locator/RackInferringSnitch.java b/src/java/org/apache/cassandra/locator/RackInferringSnitch.java
index 6ae10cc..3429ad1 100644
--- a/src/java/org/apache/cassandra/locator/RackInferringSnitch.java
+++ b/src/java/org/apache/cassandra/locator/RackInferringSnitch.java
@@ -25,11 +25,11 @@ public class RackInferringSnitch extends AbstractNetworkTopologySnitch
 {
     public String getRack(InetAddressAndPort endpoint)
     {
-        return Integer.toString(endpoint.address.getAddress()[2] & 0xFF, 10);
+        return Integer.toString(endpoint.getAddress().getAddress()[2] & 0xFF, 10);
     }
 
     public String getDatacenter(InetAddressAndPort endpoint)
     {
-        return Integer.toString(endpoint.address.getAddress()[1] & 0xFF, 10);
+        return Integer.toString(endpoint.getAddress().getAddress()[1] & 0xFF, 10);
     }
 }
diff --git a/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java b/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
index 752dc63..fbf655f 100644
--- a/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
+++ b/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
@@ -156,7 +156,7 @@ public class InboundConnectionInitiator
             bootstrap.childOption(ChannelOption.SO_RCVBUF, socketReceiveBufferSizeInBytes);
 
         InetAddressAndPort bind = initializer.settings.bindAddress;
-        ChannelFuture channelFuture = bootstrap.bind(new InetSocketAddress(bind.address, bind.port));
+        ChannelFuture channelFuture = bootstrap.bind(new InetSocketAddress(bind.getAddress(), bind.getPort()));
 
         if (!channelFuture.awaitUninterruptibly().isSuccess())
         {
diff --git a/src/java/org/apache/cassandra/net/InboundConnectionSettings.java b/src/java/org/apache/cassandra/net/InboundConnectionSettings.java
index 00def4f..2eab9bc 100644
--- a/src/java/org/apache/cassandra/net/InboundConnectionSettings.java
+++ b/src/java/org/apache/cassandra/net/InboundConnectionSettings.java
@@ -73,7 +73,7 @@ public class InboundConnectionSettings
 
     public boolean authenticate(InetAddressAndPort endpoint)
     {
-        return authenticator.authenticate(endpoint.address, endpoint.port);
+        return authenticator.authenticate(endpoint.getAddress(), endpoint.getPort());
     }
 
     public boolean authenticate(InetAddress address, int port)
@@ -84,7 +84,7 @@ public class InboundConnectionSettings
     public String toString()
     {
         return format("address: (%s), nic: %s, encryption: %s",
-                      bindAddress, FBUtilities.getNetworkInterface(bindAddress.address), SocketFactory.encryptionOptionsSummary(encryption));
+                      bindAddress, FBUtilities.getNetworkInterface(bindAddress.getAddress()), SocketFactory.encryptionOptionsSummary(encryption));
     }
 
     public InboundConnectionSettings withAuthenticator(IInternodeAuthenticator authenticator)
@@ -168,9 +168,9 @@ public class InboundConnectionSettings
     public InboundConnectionSettings withDefaults()
     {
         // this is for the socket that can be plain, only ssl, or optional plain/ssl
-        if (bindAddress.port != DatabaseDescriptor.getStoragePort() && bindAddress.port != DatabaseDescriptor.getSSLStoragePort())
+        if (bindAddress.getPort() != DatabaseDescriptor.getStoragePort() && bindAddress.getPort() != DatabaseDescriptor.getSSLStoragePort())
             throw new ConfigurationException(format("Local endpoint port %d doesn't match YAML configured port %d or legacy SSL port %d",
-                                                    bindAddress.port, DatabaseDescriptor.getStoragePort(), DatabaseDescriptor.getSSLStoragePort()));
+                                                    bindAddress.getPort(), DatabaseDescriptor.getStoragePort(), DatabaseDescriptor.getSSLStoragePort()));
 
         IInternodeAuthenticator authenticator = this.authenticator;
         ServerEncryptionOptions encryption = this.encryption;
diff --git a/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java b/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java
index dad8526..3e55694 100644
--- a/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java
+++ b/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java
@@ -188,7 +188,7 @@ public class OutboundConnectionInitiator<SuccessType extends OutboundConnectionI
             bootstrap.option(ChannelOption.SO_SNDBUF, settings.socketSendBufferSizeInBytes);
 
         InetAddressAndPort remoteAddress = settings.connectTo;
-        bootstrap.remoteAddress(new InetSocketAddress(remoteAddress.address, remoteAddress.port));
+        bootstrap.remoteAddress(new InetSocketAddress(remoteAddress.getAddress(), remoteAddress.getPort()));
         return bootstrap;
     }
 
@@ -206,7 +206,7 @@ public class OutboundConnectionInitiator<SuccessType extends OutboundConnectionI
                                                                          ISslContextFactory.SocketType.CLIENT);
                 // for some reason channel.remoteAddress() will return null
                 InetAddressAndPort address = settings.to;
-                InetSocketAddress peer = settings.encryption.require_endpoint_verification ? new InetSocketAddress(address.address, address.port) : null;
+                InetSocketAddress peer = settings.encryption.require_endpoint_verification ? new InetSocketAddress(address.getAddress(), address.getPort()) : null;
                 SslHandler sslHandler = newSslHandler(channel, sslContext, peer);
                 logger.trace("creating outbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName());
                 pipeline.addFirst("ssl", sslHandler);
diff --git a/src/java/org/apache/cassandra/net/OutboundConnectionSettings.java b/src/java/org/apache/cassandra/net/OutboundConnectionSettings.java
index c3c7825..599e717 100644
--- a/src/java/org/apache/cassandra/net/OutboundConnectionSettings.java
+++ b/src/java/org/apache/cassandra/net/OutboundConnectionSettings.java
@@ -159,7 +159,7 @@ public class OutboundConnectionSettings
 
     public boolean authenticate()
     {
-        return authenticator.authenticate(to.address, to.port);
+        return authenticator.authenticate(to.getAddress(), to.getPort());
     }
 
     public boolean withEncryption()
diff --git a/src/java/org/apache/cassandra/net/SocketFactory.java b/src/java/org/apache/cassandra/net/SocketFactory.java
index f97e2fe..dd9e41e 100644
--- a/src/java/org/apache/cassandra/net/SocketFactory.java
+++ b/src/java/org/apache/cassandra/net/SocketFactory.java
@@ -316,7 +316,7 @@ public final class SocketFactory
     static String addressId(InetAddressAndPort address, InetSocketAddress realAddress)
     {
         String str = address.toString();
-        if (!address.address.equals(realAddress.getAddress()) || address.port != realAddress.getPort())
+        if (!address.getAddress().equals(realAddress.getAddress()) || address.getPort() != realAddress.getPort())
             str += '(' + InetAddressAndPort.toString(realAddress.getAddress(), realAddress.getPort()) + ')';
         return str;
     }
diff --git a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
index 7e8d8bc..f0faba1 100644
--- a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
+++ b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
@@ -248,7 +248,7 @@ public final class SystemDistributedKeyspace
                                     range.left.toString(),
                                     range.right.toString(),
                                     coordinator.getHostAddress(false),
-                                    coordinator.port,
+                                    coordinator.getPort(),
                                     Joiner.on("', '").join(participants),
                                     Joiner.on("', '").join(participants_v2),
                                     RepairState.STARTED.toString());
diff --git a/src/java/org/apache/cassandra/repair/consistent/LocalSessionInfo.java b/src/java/org/apache/cassandra/repair/consistent/LocalSessionInfo.java
index f1f927b..fa2835e 100644
--- a/src/java/org/apache/cassandra/repair/consistent/LocalSessionInfo.java
+++ b/src/java/org/apache/cassandra/repair/consistent/LocalSessionInfo.java
@@ -62,7 +62,7 @@ public class LocalSessionInfo
         m.put(STARTED, Integer.toString(session.getStartedAt()));
         m.put(LAST_UPDATE, Integer.toString(session.getLastUpdate()));
         m.put(COORDINATOR, session.coordinator.toString());
-        m.put(PARTICIPANTS, Joiner.on(',').join(Iterables.transform(session.participants.stream().map(peer -> peer.address).collect(Collectors.toList()), InetAddress::getHostAddress)));
+        m.put(PARTICIPANTS, Joiner.on(',').join(Iterables.transform(session.participants.stream().map(peer -> peer.getAddress()).collect(Collectors.toList()), InetAddress::getHostAddress)));
         m.put(PARTICIPANTS_WP, Joiner.on(',').join(Iterables.transform(session.participants, InetAddressAndPort::getHostAddressAndPort)));
         m.put(TABLES, Joiner.on(',').join(Iterables.transform(session.tableIds, LocalSessionInfo::tableString)));
 
diff --git a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java
index 741ad90..fd00c40 100644
--- a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java
+++ b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java
@@ -531,9 +531,9 @@ public class LocalSessions
                                        Date.from(Instant.ofEpochSecond(session.getLastUpdate())),
                                        Date.from(Instant.ofEpochMilli(session.repairedAt)),
                                        session.getState().ordinal(),
-                                       session.coordinator.address,
-                                       session.coordinator.port,
-                                       session.participants.stream().map(participant -> participant.address).collect(Collectors.toSet()),
+                                       session.coordinator.getAddress(),
+                                       session.coordinator.getPort(),
+                                       session.participants.stream().map(participant -> participant.getAddress()).collect(Collectors.toSet()),
                                        session.participants.stream().map(participant -> participant.getHostAddressAndPort()).collect(Collectors.toSet()),
                                        serializeRanges(session.ranges),
                                        tableIdToUuid(session.tableIds));
diff --git a/src/java/org/apache/cassandra/schema/SchemaAnnouncementEvent.java b/src/java/org/apache/cassandra/schema/SchemaAnnouncementEvent.java
index 4e0bd68..ea6b9b6 100644
--- a/src/java/org/apache/cassandra/schema/SchemaAnnouncementEvent.java
+++ b/src/java/org/apache/cassandra/schema/SchemaAnnouncementEvent.java
@@ -78,12 +78,12 @@ final class SchemaAnnouncementEvent extends DiagnosticEvent
         HashMap<String, Serializable> ret = new HashMap<>();
         if (schemaDestinationEndpoints != null)
         {
-            Set<String> eps = schemaDestinationEndpoints.stream().map(InetAddressAndPort::toString).collect(Collectors.toSet());
+            Set<String> eps = schemaDestinationEndpoints.stream().map(Object::toString).collect(Collectors.toSet());
             ret.put("endpointDestinations", new HashSet<>(eps));
         }
         if (schemaEndpointsIgnored != null)
         {
-            Set<String> eps = schemaEndpointsIgnored.stream().map(InetAddressAndPort::toString).collect(Collectors.toSet());
+            Set<String> eps = schemaEndpointsIgnored.stream().map(Object::toString).collect(Collectors.toSet());
             ret.put("endpointIgnored", new HashSet<>(eps));
         }
         if (statement != null)
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 06c685c..1638505 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -23,6 +23,7 @@ import java.io.DataInputStream;
 import java.io.IOError;
 import java.io.IOException;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.nio.file.Paths;
@@ -1772,8 +1773,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             states.add(Pair.create(ApplicationState.STATUS_WITH_PORT, replacing?
                                                             valueFactory.bootReplacingWithPort(DatabaseDescriptor.getReplaceAddress()) :
                                                             valueFactory.bootstrapping(tokens)));
-            states.add(Pair.create(ApplicationState.STATUS, replacing?
-                                                            valueFactory.bootReplacing(DatabaseDescriptor.getReplaceAddress().address) :
+            states.add(Pair.create(ApplicationState.STATUS, replacing ?
+                                                            valueFactory.bootReplacing(DatabaseDescriptor.getReplaceAddress().getAddress()) :
                                                             valueFactory.bootstrapping(tokens)));
             Gossiper.instance.addLocalApplicationStates(states);
             setMode(Mode.JOINING, "sleeping " + RING_DELAY + " ms for pending range setup", true);
@@ -1999,7 +2000,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             }
         }
         else if (Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.RPC_ADDRESS) == null)
-            return endpoint.address.getHostAddress() + ":" + DatabaseDescriptor.getNativeTransportPort();
+            return endpoint.getAddress().getHostAddress() + ":" + DatabaseDescriptor.getNativeTransportPort();
         else
             return Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.RPC_ADDRESS).value + ":" + DatabaseDescriptor.getNativeTransportPort();
     }
@@ -2488,8 +2489,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                     try
                     {
                         InetAddressAndPort address = InetAddressAndPort.getByName(entry.getValue().value);
-                        native_address = address.address;
-                        native_port = address.port;
+                        native_address = address.getAddress();
+                        native_port = address.getPort();
                     }
                     catch (UnknownHostException e)
                     {
@@ -3500,7 +3501,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
         for (Pair<Token, InetAddressAndPort> node : tokenMetadata.getMovingEndpoints())
         {
-            endpoints.add(node.right.address.getHostAddress());
+            endpoints.add(node.right.getAddress().getHostAddress());
         }
 
         return endpoints;
@@ -4354,7 +4355,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         EndpointsForToken replicas = getNaturalReplicasForToken(keyspaceName, cf, key);
         List<InetAddress> inetList = new ArrayList<>(replicas.size());
-        replicas.forEach(r -> inetList.add(r.endpoint().address));
+        replicas.forEach(r -> inetList.add(r.endpoint().getAddress()));
         return inetList;
     }
 
@@ -4368,7 +4369,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         EndpointsForToken replicas = getNaturalReplicasForToken(keyspaceName, key);
         List<InetAddress> inetList = new ArrayList<>(replicas.size());
-        replicas.forEach(r -> inetList.add(r.endpoint().address));
+        replicas.forEach(r -> inetList.add(r.endpoint().getAddress()));
         return inetList;
     }
 
@@ -5221,10 +5222,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         {
             InetAddressAndPort endpoint = tokenMetadata.getEndpoint(entry.getKey());
             Float tokenOwnership = entry.getValue();
-            if (nodeMap.containsKey(endpoint.address))
-                nodeMap.put(endpoint.address, nodeMap.get(endpoint.address) + tokenOwnership);
+            if (nodeMap.containsKey(endpoint.getAddress()))
+                nodeMap.put(endpoint.getAddress(), nodeMap.get(endpoint.getAddress()) + tokenOwnership);
             else
-                nodeMap.put(endpoint.address, tokenOwnership);
+                nodeMap.put(endpoint.getAddress(), tokenOwnership);
         }
         return nodeMap;
     }
@@ -5328,7 +5329,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         LinkedHashMap<InetAddressAndPort, Float> result = getEffectiveOwnership(keyspace);
         LinkedHashMap<InetAddress, Float> asInets = new LinkedHashMap<>();
-        result.entrySet().stream().forEachOrdered(entry -> asInets.put(entry.getKey().address, entry.getValue()));
+        result.entrySet().stream().forEachOrdered(entry -> asInets.put(entry.getKey().getAddress(), entry.getValue()));
         return asInets;
     }
 
@@ -6053,7 +6054,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         Map<UUID, Set<InetAddressAndPort>> outstanding = MigrationCoordinator.instance.outstandingVersions();
         return outstanding.entrySet().stream().collect(Collectors.toMap(e -> e.getKey().toString(),
-                                                                        e -> e.getValue().stream().map(i -> i.address).collect(Collectors.toSet())));
+                                                                        e -> e.getValue().stream().map(InetSocketAddress::getAddress).collect(Collectors.toSet())));
     }
 
     @Override
@@ -6061,7 +6062,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         Map<UUID, Set<InetAddressAndPort>> outstanding = MigrationCoordinator.instance.outstandingVersions();
         return outstanding.entrySet().stream().collect(Collectors.toMap(e -> e.getKey().toString(),
-                                                                        e -> e.getValue().stream().map(InetAddressAndPort::toString).collect(Collectors.toSet())));
+                                                                        e -> e.getValue().stream().map(Object::toString).collect(Collectors.toSet())));
     }
 
     public boolean autoOptimiseIncRepairStreams()
diff --git a/src/java/org/apache/cassandra/service/TruncateResponseHandler.java b/src/java/org/apache/cassandra/service/TruncateResponseHandler.java
index 9976376..984ba5a 100644
--- a/src/java/org/apache/cassandra/service/TruncateResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/TruncateResponseHandler.java
@@ -94,7 +94,7 @@ public class TruncateResponseHandler implements RequestCallback<TruncateResponse
     public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason)
     {
         // If the truncation hasn't succeeded on some replica, abort and indicate this back to the client.
-        truncateFailingReplica = from.address;
+        truncateFailingReplica = from.getAddress();
         condition.signalAll();
     }
 
diff --git a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairEvent.java b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairEvent.java
index 65c0215..a30efa1 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairEvent.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairEvent.java
@@ -89,7 +89,7 @@ final class ReadRepairEvent extends DiagnosticEvent
         ret.put("consistency", consistency.name());
         ret.put("speculativeRetry", speculativeRetry.name());
 
-        Set<String> eps = destinations.stream().map(InetAddressAndPort::toString).collect(Collectors.toSet());
+        Set<String> eps = destinations.stream().map(Object::toString).collect(Collectors.toSet());
         ret.put("endpointDestinations", new HashSet<>(eps));
 
         if (digestsByEndpoint != null)
@@ -106,7 +106,7 @@ final class ReadRepairEvent extends DiagnosticEvent
         }
         if (allEndpoints != null)
         {
-            eps = allEndpoints.stream().map(InetAddressAndPort::toString).collect(Collectors.toSet());
+            eps = allEndpoints.stream().map(Object::toString).collect(Collectors.toSet());
             ret.put("allEndpoints", new HashSet<>(eps));
         }
         return ret;
diff --git a/src/java/org/apache/cassandra/streaming/management/ProgressInfoCompositeData.java b/src/java/org/apache/cassandra/streaming/management/ProgressInfoCompositeData.java
index a1fa19f..a1f7d86 100644
--- a/src/java/org/apache/cassandra/streaming/management/ProgressInfoCompositeData.java
+++ b/src/java/org/apache/cassandra/streaming/management/ProgressInfoCompositeData.java
@@ -76,8 +76,8 @@ public class ProgressInfoCompositeData
     {
         Map<String, Object> valueMap = new HashMap<>();
         valueMap.put(ITEM_NAMES[0], planId.toString());
-        valueMap.put(ITEM_NAMES[1], progressInfo.peer.address.getHostAddress());
-        valueMap.put(ITEM_NAMES[2], progressInfo.peer.port);
+        valueMap.put(ITEM_NAMES[1], progressInfo.peer.getAddress().getHostAddress());
+        valueMap.put(ITEM_NAMES[2], progressInfo.peer.getPort());
         valueMap.put(ITEM_NAMES[3], progressInfo.sessionIndex);
         valueMap.put(ITEM_NAMES[4], progressInfo.fileName);
         valueMap.put(ITEM_NAMES[5], progressInfo.direction.name());
diff --git a/src/java/org/apache/cassandra/streaming/management/SessionCompleteEventCompositeData.java b/src/java/org/apache/cassandra/streaming/management/SessionCompleteEventCompositeData.java
index 1c0d8c5..665b4cd 100644
--- a/src/java/org/apache/cassandra/streaming/management/SessionCompleteEventCompositeData.java
+++ b/src/java/org/apache/cassandra/streaming/management/SessionCompleteEventCompositeData.java
@@ -61,8 +61,8 @@ public class SessionCompleteEventCompositeData
     {
         Map<String, Object> valueMap = new HashMap<>();
         valueMap.put(ITEM_NAMES[0], event.planId.toString());
-        valueMap.put(ITEM_NAMES[1], event.peer.address.getHostAddress());
-        valueMap.put(ITEM_NAMES[2], event.peer.port);
+        valueMap.put(ITEM_NAMES[1], event.peer.getAddress().getHostAddress());
+        valueMap.put(ITEM_NAMES[2], event.peer.getPort());
         valueMap.put(ITEM_NAMES[3], event.success);
         try
         {
diff --git a/src/java/org/apache/cassandra/streaming/management/SessionInfoCompositeData.java b/src/java/org/apache/cassandra/streaming/management/SessionInfoCompositeData.java
index d20eaf5..5cadac5 100644
--- a/src/java/org/apache/cassandra/streaming/management/SessionInfoCompositeData.java
+++ b/src/java/org/apache/cassandra/streaming/management/SessionInfoCompositeData.java
@@ -90,10 +90,10 @@ public class SessionInfoCompositeData
     {
         Map<String, Object> valueMap = new HashMap<>();
         valueMap.put(ITEM_NAMES[0], planId.toString());
-        valueMap.put(ITEM_NAMES[1], sessionInfo.peer.address.getHostAddress());
-        valueMap.put(ITEM_NAMES[2], sessionInfo.peer.port);
-        valueMap.put(ITEM_NAMES[3], sessionInfo.connecting.address.getHostAddress());
-        valueMap.put(ITEM_NAMES[4], sessionInfo.connecting.port);
+        valueMap.put(ITEM_NAMES[1], sessionInfo.peer.getAddress().getHostAddress());
+        valueMap.put(ITEM_NAMES[2], sessionInfo.peer.getPort());
+        valueMap.put(ITEM_NAMES[3], sessionInfo.connecting.getAddress().getHostAddress());
+        valueMap.put(ITEM_NAMES[4], sessionInfo.connecting.getPort());
         Function<StreamSummary, CompositeData> fromStreamSummary = new Function<StreamSummary, CompositeData>()
         {
             public CompositeData apply(StreamSummary input)
diff --git a/src/java/org/apache/cassandra/tools/nodetool/HostStatWithPort.java b/src/java/org/apache/cassandra/tools/nodetool/HostStatWithPort.java
index 9cff725..b07a6e3 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/HostStatWithPort.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/HostStatWithPort.java
@@ -26,7 +26,7 @@ public class HostStatWithPort extends HostStat
 
     public HostStatWithPort(String token, InetAddressAndPort endpoint, boolean resolveIp, Float owns)
     {
-        super(token, endpoint.address, resolveIp, owns);
+        super(token, endpoint.getAddress(), resolveIp, owns);
         this.endpointWithPort = endpoint;
     }
 
@@ -41,7 +41,7 @@ public class HostStatWithPort extends HostStat
             return super.ipOrDns();
 
         return resolveIp ?
-               endpointWithPort.address.getHostName() + ':' + endpointWithPort.port :
+               endpointWithPort.getAddress().getHostName() + ':' + endpointWithPort.getPort() :
                endpointWithPort.getHostAddressAndPort();
     }
 }
diff --git a/src/java/org/apache/cassandra/tracing/TraceKeyspace.java b/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
index c2e74d8..cd3fa8a 100644
--- a/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
+++ b/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
@@ -117,9 +117,9 @@ public final class TraceKeyspace
         Row.SimpleBuilder rb = builder.row();
         rb.ttl(ttl)
           .add("client", client)
-          .add("coordinator", FBUtilities.getBroadcastAddressAndPort().address);
+          .add("coordinator", FBUtilities.getBroadcastAddressAndPort().getAddress());
         if (!Gossiper.instance.hasMajorVersion3Nodes())
-            rb.add("coordinator_port", FBUtilities.getBroadcastAddressAndPort().port);
+            rb.add("coordinator_port", FBUtilities.getBroadcastAddressAndPort().getPort());
         rb.add("request", request)
           .add("started_at", new Date(startedAt))
           .add("command", command)
@@ -144,9 +144,9 @@ public final class TraceKeyspace
                                               .ttl(ttl);
 
         rowBuilder.add("activity", message)
-                  .add("source", FBUtilities.getBroadcastAddressAndPort().address);
+                  .add("source", FBUtilities.getBroadcastAddressAndPort().getAddress());
         if (!Gossiper.instance.hasMajorVersion3Nodes())
-            rowBuilder.add("source_port", FBUtilities.getBroadcastAddressAndPort().port);
+            rowBuilder.add("source_port", FBUtilities.getBroadcastAddressAndPort().getPort());
         rowBuilder.add("thread", threadName);
 
         if (elapsed >= 0)
diff --git a/src/java/org/apache/cassandra/transport/Event.java b/src/java/org/apache/cassandra/transport/Event.java
index 9515551..5e8e201 100644
--- a/src/java/org/apache/cassandra/transport/Event.java
+++ b/src/java/org/apache/cassandra/transport/Event.java
@@ -115,17 +115,17 @@ public abstract class Event
 
         public static TopologyChange newNode(InetAddressAndPort address)
         {
-            return new TopologyChange(Change.NEW_NODE, new InetSocketAddress(address.address, address.port));
+            return new TopologyChange(Change.NEW_NODE, new InetSocketAddress(address.getAddress(), address.getPort()));
         }
 
         public static TopologyChange removedNode(InetAddressAndPort address)
         {
-            return new TopologyChange(Change.REMOVED_NODE, new InetSocketAddress(address.address, address.port));
+            return new TopologyChange(Change.REMOVED_NODE, new InetSocketAddress(address.getAddress(), address.getPort()));
         }
 
         public static TopologyChange movedNode(InetAddressAndPort address)
         {
-            return new TopologyChange(Change.MOVED_NODE, new InetSocketAddress(address.address, address.port));
+            return new TopologyChange(Change.MOVED_NODE, new InetSocketAddress(address.getAddress(), address.getPort()));
         }
 
         // Assumes the type has already been deserialized
@@ -186,12 +186,12 @@ public abstract class Event
 
         public static StatusChange nodeUp(InetAddressAndPort address)
         {
-            return new StatusChange(Status.UP, new InetSocketAddress(address.address, address.port));
+            return new StatusChange(Status.UP, new InetSocketAddress(address.getAddress(), address.getPort()));
         }
 
         public static StatusChange nodeDown(InetAddressAndPort address)
         {
-            return new StatusChange(Status.DOWN, new InetSocketAddress(address.address, address.port));
+            return new StatusChange(Status.DOWN, new InetSocketAddress(address.getAddress(), address.getPort()));
         }
 
         // Assumes the type has already been deserialized
diff --git a/src/java/org/apache/cassandra/transport/Server.java b/src/java/org/apache/cassandra/transport/Server.java
index 5c9e575..bf3f3e4 100644
--- a/src/java/org/apache/cassandra/transport/Server.java
+++ b/src/java/org/apache/cassandra/transport/Server.java
@@ -381,7 +381,7 @@ public class Server implements CassandraDaemon.Server
                 // That should not happen, so log an error, but return the
                 // endpoint address since there's a good change this is right
                 logger.error("Problem retrieving RPC address for {}", endpoint, e);
-                return InetAddressAndPort.getByAddressOverrideDefaults(endpoint.address, DatabaseDescriptor.getNativeTransportPort());
+                return InetAddressAndPort.getByAddressOverrideDefaults(endpoint.getAddress(), DatabaseDescriptor.getNativeTransportPort());
             }
         }
 
diff --git a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
index 6890584..5d29d3a 100644
--- a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
@@ -222,7 +222,7 @@ public class ErrorMessage extends Message.Response
                         {
                             for (Map.Entry<InetAddressAndPort, RequestFailureReason> entry : rfe.failureReasonByEndpoint.entrySet())
                             {
-                                CBUtil.writeInetAddr(entry.getKey().address, dest);
+                                CBUtil.writeInetAddr(entry.getKey().getAddress(), dest);
                                 dest.writeShort(entry.getValue().code);
                             }
                         }
@@ -302,7 +302,7 @@ public class ErrorMessage extends Message.Response
                         {
                             for (Map.Entry<InetAddressAndPort, RequestFailureReason> entry : rfe.failureReasonByEndpoint.entrySet())
                             {
-                                size += CBUtil.sizeOfInetAddr(entry.getKey().address);
+                                size += CBUtil.sizeOfInetAddr(entry.getKey().getAddress());
                                 size += 2; // RequestFailureReason code
                             }
                         }
diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java
index e8e8066..3905a77 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -241,7 +241,7 @@ public class FBUtilities
      */
     public static void setBroadcastInetAddressAndPort(InetAddressAndPort addr)
     {
-        broadcastInetAddress = addr.address;
+        broadcastInetAddress = addr.getAddress();
         broadcastInetAddressAndPort = addr;
     }
 
diff --git a/src/java/org/apache/cassandra/utils/Mx4jTool.java b/src/java/org/apache/cassandra/utils/Mx4jTool.java
index e54c0bf..e0e998f 100644
--- a/src/java/org/apache/cassandra/utils/Mx4jTool.java
+++ b/src/java/org/apache/cassandra/utils/Mx4jTool.java
@@ -80,7 +80,7 @@ public class Mx4jTool
     {
         String sAddress = MX4JADDRESS.getString();
         if (StringUtils.isEmpty(sAddress))
-            sAddress = FBUtilities.getBroadcastAddressAndPort().address.getHostAddress();
+            sAddress = FBUtilities.getBroadcastAddressAndPort().getAddress().getHostAddress();
         return sAddress;
     }
 
diff --git a/src/java/org/apache/cassandra/utils/UUIDGen.java b/src/java/org/apache/cassandra/utils/UUIDGen.java
index 7cb8459..1e727d4 100644
--- a/src/java/org/apache/cassandra/utils/UUIDGen.java
+++ b/src/java/org/apache/cassandra/utils/UUIDGen.java
@@ -398,7 +398,7 @@ public class UUIDGen
         for(InetAddressAndPort addr : data)
         {
             hasher.putBytes(addr.addressBytes);
-            hasher.putInt(addr.port);
+            hasher.putInt(addr.getPort());
         }
 
         // Identify the process on the load: we use both the PID and class loader hash.
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/DistributedTestSnitch.java b/test/distributed/org/apache/cassandra/distributed/impl/DistributedTestSnitch.java
index 0dfaa7e..6a892c4 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/DistributedTestSnitch.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/DistributedTestSnitch.java
@@ -56,7 +56,7 @@ public class DistributedTestSnitch extends AbstractNetworkTopologySnitch
         InetSocketAddress m = cache.get(addressAndPort);
         if (m == null)
         {
-            m = NetworkTopology.addressAndPort(addressAndPort.address, addressAndPort.port);
+            m = NetworkTopology.addressAndPort(addressAndPort.getAddress(), addressAndPort.getPort());
             cache.put(addressAndPort, m);
         }
         return m;
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
index 730773d..2d441f2 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
@@ -619,8 +619,8 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
                     CassandraDaemon.getInstanceForTesting().start();
                 }
 
-                if (!FBUtilities.getBroadcastAddressAndPort().address.equals(broadcastAddress().getAddress()) ||
-                    FBUtilities.getBroadcastAddressAndPort().port != broadcastAddress().getPort())
+                if (!FBUtilities.getBroadcastAddressAndPort().getAddress().equals(broadcastAddress().getAddress()) ||
+                    FBUtilities.getBroadcastAddressAndPort().getPort() != broadcastAddress().getPort())
                     throw new IllegalStateException(String.format("%s != %s", FBUtilities.getBroadcastAddressAndPort(), broadcastAddress()));
 
                 ActiveRepairService.instance.start();
diff --git a/test/distributed/org/apache/cassandra/distributed/test/StreamingTest.java b/test/distributed/org/apache/cassandra/distributed/test/StreamingTest.java
index 8fff004..e42f87c 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/StreamingTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/StreamingTest.java
@@ -149,7 +149,7 @@ public class StreamingTest extends TestBaseImpl
         @Override
         public void recordState(InetAddressAndPort from, StreamSession.State state)
         {
-            Queue<Integer> states = stateTransitions.get(from.address);
+            Queue<Integer> states = stateTransitions.get(from.getAddress());
             if (states.peek() == null)
                 Assert.fail("Unexpected state " + state);
 
@@ -163,7 +163,7 @@ public class StreamingTest extends TestBaseImpl
             if (message == StreamMessage.Type.KEEP_ALIVE)
                 return;
 
-            Queue<Integer> messages = messageSink.get(from.address);
+            Queue<Integer> messages = messageSink.get(from.getAddress());
             if (messages.peek() == null)
                 Assert.fail("Unexpected message " + message);
 
@@ -174,10 +174,10 @@ public class StreamingTest extends TestBaseImpl
         @Override
         public void onClose(InetAddressAndPort from)
         {
-            Queue<Integer> states = stateTransitions.get(from.address);
+            Queue<Integer> states = stateTransitions.get(from.getAddress());
             Assert.assertTrue("Missing states: " + states, states.isEmpty());
 
-            Queue<Integer> messages = messageSink.get(from.address);
+            Queue<Integer> messages = messageSink.get(from.getAddress());
             Assert.assertTrue("Missing messages: " + messages, messages.isEmpty());
         }
     }
diff --git a/test/unit/org/apache/cassandra/audit/AuditLoggerAuthTest.java b/test/unit/org/apache/cassandra/audit/AuditLoggerAuthTest.java
index 820b15e..0e73a0e 100644
--- a/test/unit/org/apache/cassandra/audit/AuditLoggerAuthTest.java
+++ b/test/unit/org/apache/cassandra/audit/AuditLoggerAuthTest.java
@@ -287,8 +287,8 @@ public class AuditLoggerAuthTest
 
     private static void assertSource(AuditLogEntry logEntry, String username)
     {
-        assertEquals(InetAddressAndPort.getLoopbackAddress().address, logEntry.getSource().address);
-        assertTrue(logEntry.getSource().port > 0);
+        assertEquals(InetAddressAndPort.getLoopbackAddress().getAddress(), logEntry.getSource().getAddress());
+        assertTrue(logEntry.getSource().getPort() > 0);
         if (logEntry.getType() != AuditLogEntryType.LOGIN_ERROR)
             assertEquals(username, logEntry.getUser());
     }
diff --git a/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java b/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java
index 4595957..bb02f40 100644
--- a/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java
+++ b/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java
@@ -69,7 +69,7 @@ public class RangeFetchMapCalculatorTest
 
             private int getIPLastPart(InetAddressAndPort endpoint)
             {
-                String str = endpoint.address.toString();
+                String str = endpoint.getAddress().toString();
                 int index = str.lastIndexOf(".");
                 return Integer.parseInt(str.substring(index + 1).trim());
             }
diff --git a/test/unit/org/apache/cassandra/gms/GossiperTest.java b/test/unit/org/apache/cassandra/gms/GossiperTest.java
index b0d5daa..68841ea 100644
--- a/test/unit/org/apache/cassandra/gms/GossiperTest.java
+++ b/test/unit/org/apache/cassandra/gms/GossiperTest.java
@@ -274,12 +274,12 @@ public class GossiperTest
         {
             gossiper.seeds.add(addr);
             nextSeeds.add(addr);
-            addr = InetAddressAndPort.getByAddress(InetAddresses.increment(addr.address));
+            addr = InetAddressAndPort.getByAddress(InetAddresses.increment(addr.getAddress()));
         }
         Assert.assertEquals(nextSize, gossiper.seeds.size());
 
         // Add another unique address to the list
-        addr = InetAddressAndPort.getByAddress(InetAddresses.increment(addr.address));
+        addr = InetAddressAndPort.getByAddress(InetAddresses.increment(addr.getAddress()));
         nextSeeds.add(addr);
         nextSize++;
         DatabaseDescriptor.setSeedProvider(new TestSeedProvider(nextSeeds));
@@ -318,7 +318,7 @@ public class GossiperTest
         for (int i = 0; i < disjointSize; i ++)
         {
             disjointSeeds.add(addr);
-            addr = InetAddressAndPort.getByAddress(InetAddresses.increment(addr.address));
+            addr = InetAddressAndPort.getByAddress(InetAddresses.increment(addr.getAddress()));
         }
         DatabaseDescriptor.setSeedProvider(new TestSeedProvider(disjointSeeds));
         loadedList = gossiper.reloadSeeds();
diff --git a/test/unit/org/apache/cassandra/locator/InetAddressAndPortSerializerTest.java b/test/unit/org/apache/cassandra/locator/InetAddressAndPortSerializerTest.java
index c6ea3d7..78ecfb5 100644
--- a/test/unit/org/apache/cassandra/locator/InetAddressAndPortSerializerTest.java
+++ b/test/unit/org/apache/cassandra/locator/InetAddressAndPortSerializerTest.java
@@ -65,8 +65,8 @@ public class InetAddressAndPortSerializerTest
         }
         else
         {
-            assertEquals(address.address, roundtripped.address);
-            assertEquals(InetAddressAndPort.getDefaultPort(), roundtripped.port);
+            assertEquals(address.getAddress(), roundtripped.getAddress());
+            assertEquals(InetAddressAndPort.getDefaultPort(), roundtripped.getPort());
         }
     }
 }
diff --git a/test/unit/org/apache/cassandra/locator/InetAddressAndPortTest.java b/test/unit/org/apache/cassandra/locator/InetAddressAndPortTest.java
index 78b9a6f..c8aec44 100644
--- a/test/unit/org/apache/cassandra/locator/InetAddressAndPortTest.java
+++ b/test/unit/org/apache/cassandra/locator/InetAddressAndPortTest.java
@@ -47,13 +47,13 @@ public class InetAddressAndPortTest
 
         //Test default port
         InetAddressAndPort address = InetAddressAndPort.getByName("127.0.0.1");
-        assertEquals(InetAddress.getByName("127.0.0.1"), address.address);
-        assertEquals(InetAddressAndPort.defaultPort, address.port);
+        assertEquals(InetAddress.getByName("127.0.0.1"), address.getAddress());
+        assertEquals(InetAddressAndPort.defaultPort, address.getPort());
 
         //Test overriding default port
         address = InetAddressAndPort.getByName("127.0.0.1:42");
-        assertEquals(InetAddress.getByName("127.0.0.1"), address.address);
-        assertEquals(42, address.port);
+        assertEquals(InetAddress.getByName("127.0.0.1"), address.getAddress());
+        assertEquals(42, address.getPort());
     }
 
     @Test
@@ -69,13 +69,13 @@ public class InetAddressAndPortTest
 
         //Test default port
         InetAddressAndPort address = InetAddressAndPort.getByName("2001:0db8:0000:0000:0000:ff00:0042:8329");
-        assertEquals(InetAddress.getByName("2001:0db8:0000:0000:0000:ff00:0042:8329"), address.address);
-        assertEquals(InetAddressAndPort.defaultPort, address.port);
+        assertEquals(InetAddress.getByName("2001:0db8:0000:0000:0000:ff00:0042:8329"), address.getAddress());
+        assertEquals(InetAddressAndPort.defaultPort, address.getPort());
 
         //Test overriding default port
         address = InetAddressAndPort.getByName("[2001:0db8:0000:0000:0000:ff00:0042:8329]:42");
-        assertEquals(InetAddress.getByName("2001:0db8:0000:0000:0000:ff00:0042:8329"), address.address);
-        assertEquals(42, address.port);
+        assertEquals(InetAddress.getByName("2001:0db8:0000:0000:0000:ff00:0042:8329"), address.getAddress());
+        assertEquals(42, address.getPort());
     }
 
     @Test
diff --git a/test/unit/org/apache/cassandra/net/ForwardingInfoTest.java b/test/unit/org/apache/cassandra/net/ForwardingInfoTest.java
index 16dec9f..6e0d11a 100644
--- a/test/unit/org/apache/cassandra/net/ForwardingInfoTest.java
+++ b/test/unit/org/apache/cassandra/net/ForwardingInfoTest.java
@@ -91,9 +91,9 @@ public class ForwardingInfoTest
             {
                 InetAddressAndPort original = addresses.get(ii);
                 InetAddressAndPort roundtripped = iterator.next();
-                assertEquals(original.address, roundtripped.address);
+                assertEquals(original.getAddress(), roundtripped.getAddress());
                 //3.0 can't send port numbers so you get the defaults
-                assertEquals(65532, roundtripped.port);
+                assertEquals(65532, roundtripped.getPort());
             }
         }
     }
diff --git a/test/unit/org/apache/cassandra/net/MessagingServiceTest.java b/test/unit/org/apache/cassandra/net/MessagingServiceTest.java
index 8870a4e..a82315b 100644
--- a/test/unit/org/apache/cassandra/net/MessagingServiceTest.java
+++ b/test/unit/org/apache/cassandra/net/MessagingServiceTest.java
@@ -100,7 +100,7 @@ public class MessagingServiceTest
         DatabaseDescriptor.setInternodeAuthenticator(originalAuthenticator);
         DatabaseDescriptor.setInternodeMessagingEncyptionOptions(originalServerEncryptionOptions);
         DatabaseDescriptor.setShouldListenOnBroadcastAddress(false);
-        DatabaseDescriptor.setListenAddress(originalListenAddress.address);
+        DatabaseDescriptor.setListenAddress(originalListenAddress.getAddress());
         FBUtilities.reset();
     }
 
@@ -329,7 +329,7 @@ public class MessagingServiceTest
         if (listenOnBroadcastAddr)
         {
             DatabaseDescriptor.setShouldListenOnBroadcastAddress(true);
-            listenAddress = InetAddresses.increment(FBUtilities.getBroadcastAddressAndPort().address);
+            listenAddress = InetAddresses.increment(FBUtilities.getBroadcastAddressAndPort().getAddress());
             DatabaseDescriptor.setListenAddress(listenAddress);
             FBUtilities.reset();
         }
@@ -348,9 +348,9 @@ public class MessagingServiceTest
                 expect.add(InetAddressAndPort.getByAddressOverrideDefaults(listenAddress, DatabaseDescriptor.getSSLStoragePort()));
             if (listenOnBroadcastAddr)
             {
-                expect.add(InetAddressAndPort.getByAddressOverrideDefaults(FBUtilities.getBroadcastAddressAndPort().address, DatabaseDescriptor.getStoragePort()));
+                expect.add(InetAddressAndPort.getByAddressOverrideDefaults(FBUtilities.getBroadcastAddressAndPort().getAddress(), DatabaseDescriptor.getStoragePort()));
                 if (settings.encryption.enable_legacy_ssl_storage_port)
-                    expect.add(InetAddressAndPort.getByAddressOverrideDefaults(FBUtilities.getBroadcastAddressAndPort().address, DatabaseDescriptor.getSSLStoragePort()));
+                    expect.add(InetAddressAndPort.getByAddressOverrideDefaults(FBUtilities.getBroadcastAddressAndPort().getAddress(), DatabaseDescriptor.getSSLStoragePort()));
             }
 
             Assert.assertEquals(expect.size(), connections.sockets().size());
@@ -361,8 +361,8 @@ public class MessagingServiceTest
                 Assert.assertEquals(serverEncryptionOptions.isEnabled(), socket.settings.encryption.isEnabled());
                 Assert.assertEquals(serverEncryptionOptions.isOptional(), socket.settings.encryption.isOptional());
                 if (!serverEncryptionOptions.isEnabled())
-                    assertNotEquals(legacySslPort, socket.settings.bindAddress.port);
-                if (legacySslPort == socket.settings.bindAddress.port)
+                    assertNotEquals(legacySslPort, socket.settings.bindAddress.getPort());
+                if (legacySslPort == socket.settings.bindAddress.getPort())
                     Assert.assertFalse(socket.settings.encryption.isOptional());
                 Assert.assertTrue(socket.settings.bindAddress.toString(), expect.remove(socket.settings.bindAddress));
             }
diff --git a/test/unit/org/apache/cassandra/repair/RepairJobTest.java b/test/unit/org/apache/cassandra/repair/RepairJobTest.java
index 573934b..cfe7297 100644
--- a/test/unit/org/apache/cassandra/repair/RepairJobTest.java
+++ b/test/unit/org/apache/cassandra/repair/RepairJobTest.java
@@ -184,7 +184,7 @@ public class RepairJobTest
         this.sessionJobDesc = new RepairJobDesc(session.parentRepairSession, session.getId(),
                                                 session.keyspace, CF, session.ranges());
 
-        FBUtilities.setBroadcastInetAddress(addr1.address);
+        FBUtilities.setBroadcastInetAddress(addr1.getAddress());
     }
 
     @After
@@ -561,7 +561,7 @@ public class RepairJobTest
         for (InetAddressAndPort local : new InetAddressAndPort[]{ addr1, addr2, addr3 })
         {
             FBUtilities.reset();
-            FBUtilities.setBroadcastInetAddress(local.address);
+            FBUtilities.setBroadcastInetAddress(local.getAddress());
             testLocalSyncWithTransient(local, false);
         }
     }
@@ -572,7 +572,7 @@ public class RepairJobTest
         for (InetAddressAndPort local : new InetAddressAndPort[]{ addr1, addr2, addr3 })
         {
             FBUtilities.reset();
-            FBUtilities.setBroadcastInetAddress(local.address);
+            FBUtilities.setBroadcastInetAddress(local.getAddress());
             testLocalSyncWithTransient(local, true);
         }
     }
@@ -628,7 +628,7 @@ public class RepairJobTest
 
     private static void testLocalAndRemoteTransient(boolean pullRepair)
     {
-        FBUtilities.setBroadcastInetAddress(addr4.address);
+        FBUtilities.setBroadcastInetAddress(addr4.getAddress());
         List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, RANGE_1, "one", RANGE_2, "one", RANGE_3, "one"),
                                                          treeResponse(addr2, RANGE_1, "two", RANGE_2, "two", RANGE_3, "two"),
                                                          treeResponse(addr3, RANGE_1, "three", RANGE_2, "three", RANGE_3, "three"),
diff --git a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java
index 837cf1b..5285eb0 100644
--- a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java
+++ b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java
@@ -94,7 +94,7 @@ public class WriteResponseHandlerTest
 
             public String getDatacenter(InetAddressAndPort endpoint)
             {
-                byte[] address = endpoint.address.getAddress();
+                byte[] address = endpoint.getAddress().getAddress();
                 if (address[1] == 1)
                     return "datacenter1";
                 else
diff --git a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTransientTest.java b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTransientTest.java
index 273adf9..2d3a236 100644
--- a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTransientTest.java
+++ b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTransientTest.java
@@ -109,7 +109,7 @@ public class WriteResponseHandlerTransientTest
 
             public String getDatacenter(InetAddressAndPort endpoint)
             {
-                byte[] address = endpoint.address.getAddress();
+                byte[] address = endpoint.getAddress().getAddress();
                 if (address[1] == 1)
                     return DC1;
                 else
diff --git a/test/unit/org/apache/cassandra/transport/CQLUserAuditTest.java b/test/unit/org/apache/cassandra/transport/CQLUserAuditTest.java
index e7a5694..1c4e41d 100644
--- a/test/unit/org/apache/cassandra/transport/CQLUserAuditTest.java
+++ b/test/unit/org/apache/cassandra/transport/CQLUserAuditTest.java
@@ -230,9 +230,9 @@ public class CQLUserAuditTest
         AuditEvent event = auditEvents.poll(100, TimeUnit.MILLISECONDS);
         assertEquals(expectedAuthType, event.getType());
         assertTrue(!authFailed || event.getType() == AuditLogEntryType.LOGIN_ERROR);
-        assertEquals(InetAddressAndPort.getLoopbackAddress().address,
-                     event.getEntry().getSource().address);
-        assertTrue(event.getEntry().getSource().port > 0);
+        assertEquals(InetAddressAndPort.getLoopbackAddress().getAddress(),
+                     event.getEntry().getSource().getAddress());
+        assertTrue(event.getEntry().getSource().getPort() > 0);
         if (event.getType() != AuditLogEntryType.LOGIN_ERROR)
             assertEquals(username, event.toMap().get("user"));
 

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