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

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

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 5c3a05a5787f7ab0b46e3a6401d64f376f6c9d9f
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
    
    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;
 import org.openjdk.jmh.annotations.Param;
 import org.openjdk.jmh.annotations.Scope;
... 2098 lines suppressed ...

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