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/11/30 16:06:58 UTC

[cassandra] branch trunk updated (f47e4b2 -> 209c35a)

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

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


    from f47e4b2  Flaky GrantAndRevokeTest
     new 31bea0b  [CEP-10] Cluster and Code Simulations: Minor fixes
     new eae581a  [CEP-10] Cluster and Code Simulations: Minor improvements
     new 209c35a  [CEP-10] Phase 4: Cluster and Code Simulation

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


Summary of changes:
 build.xml                                          | 167 +++-
 ide/idea-iml-file.xml                              |   3 +
 .../cassandra/auth/CassandraRoleManager.java       |   2 +-
 .../apache/cassandra/cache/AutoSavingCache.java    |   1 -
 .../cassandra/concurrent/ImmediateExecutor.java    |   4 +
 .../cassandra/concurrent/InfiniteLoopExecutor.java |  17 +
 .../cassandra/concurrent/NamedThreadFactory.java   |   2 +-
 .../concurrent/ScheduledExecutorPlus.java          |  24 +
 .../ScheduledThreadPoolExecutorPlus.java           |  27 +
 .../cassandra/concurrent/SyncFutureTask.java       |   5 +-
 .../config/CassandraRelevantProperties.java        |   2 +
 .../cassandra/config/DatabaseDescriptor.java       |   4 +
 .../cql3/statements/BatchUpdatesCollector.java     |   2 +-
 .../org/apache/cassandra/db/ColumnFamilyStore.java |   2 +-
 src/java/org/apache/cassandra/db/Directories.java  |   1 -
 src/java/org/apache/cassandra/db/Keyspace.java     |   2 +-
 src/java/org/apache/cassandra/db/Mutation.java     |   2 +-
 src/java/org/apache/cassandra/db/ReadCommand.java  |   2 +-
 .../cassandra/db/ReadExecutionController.java      |   2 +-
 src/java/org/apache/cassandra/db/ReadResponse.java |   1 -
 .../org/apache/cassandra/db/SystemKeyspace.java    |   1 -
 .../cassandra/db/WindowsFailedSnapshotTracker.java |   2 -
 .../db/commitlog/AbstractCommitLogService.java     |   3 +-
 .../commitlog/CommitLogSegmentManagerStandard.java |   1 -
 .../apache/cassandra/db/compaction/Scrubber.java   |   1 -
 .../apache/cassandra/db/compaction/Verifier.java   |  11 +-
 .../compaction/writers/CompactionAwareWriter.java  |   1 -
 .../apache/cassandra/db/lifecycle/LogRecord.java   |   1 -
 .../cassandra/db/monitoring/MonitorableImpl.java   |   2 +-
 .../cassandra/db/monitoring/MonitoringTask.java    |   2 +-
 .../org/apache/cassandra/gms/EndpointState.java    |   1 -
 .../org/apache/cassandra/gms/FailureDetector.java  |   4 +-
 .../org/apache/cassandra/gms/GossipDigest.java     |   1 -
 .../org/apache/cassandra/gms/GossipDigestAck2.java |   1 -
 .../org/apache/cassandra/gms/GossipDigestSyn.java  |   1 -
 .../org/apache/cassandra/gms/HeartBeatState.java   |   1 -
 .../org/apache/cassandra/gms/TokenSerializer.java  |   3 -
 .../org/apache/cassandra/gms/VersionedValue.java   |   1 -
 .../cassandra/hints/ChecksummedDataInput.java      |   1 -
 .../apache/cassandra/hints/HintsDispatcher.java    |   2 +-
 .../apache/cassandra/index/sasi/TermIterator.java  |   4 -
 .../index/sasi/disk/OnDiskIndexBuilder.java        |   1 -
 .../io/compress/CompressedSequentialWriter.java    |   1 -
 .../cassandra/io/compress/CompressionMetadata.java |   3 -
 .../apache/cassandra/io/sstable/Descriptor.java    |   2 -
 .../io/sstable/SSTableIdentityIterator.java        |   1 -
 .../cassandra/io/sstable/format/SSTableReader.java |  30 +-
 .../io/sstable/format/SSTableReaderBuilder.java    |   2 -
 .../io/sstable/format/big/BigTableWriter.java      |   2 +-
 .../io/sstable/metadata/MetadataSerializer.java    |   1 -
 .../io/util/BufferedDataOutputStreamPlus.java      |   3 +
 .../org/apache/cassandra/io/util/FileHandle.java   |  12 +-
 .../org/apache/cassandra/io/util/PathUtils.java    |   2 -
 .../apache/cassandra/locator/TokenMetadata.java    |  76 +-
 src/java/org/apache/cassandra/metrics/Sampler.java |   2 +-
 .../cassandra/net/AbstractMessageHandler.java      |   2 +-
 .../cassandra/net/InboundMessageHandler.java       |   2 +-
 .../cassandra/net/InboundMessageHandlers.java      |   2 +-
 src/java/org/apache/cassandra/net/Message.java     |   2 +-
 .../apache/cassandra/net/OutboundConnection.java   |   2 +-
 .../org/apache/cassandra/net/RequestCallbacks.java |   2 +-
 .../apache/cassandra/net/ResponseVerbHandler.java  |   2 +-
 src/java/org/apache/cassandra/net/Verb.java        |   2 +-
 .../repair/consistent/SyncStatSummary.java         |   1 -
 .../apache/cassandra/schema/MigrationManager.java  |   4 +
 .../apache/cassandra/schema/SchemaKeyspace.java    |   3 +
 .../schema/SystemDistributedKeyspace.java          |   5 +-
 .../cassandra/service/ActiveRepairService.java     |   3 +
 .../apache/cassandra/service/StartupChecks.java    |   2 -
 .../org/apache/cassandra/service/StorageProxy.java |  28 +-
 .../apache/cassandra/service/StorageService.java   |   1 -
 .../cassandra/service/paxos/BallotGenerator.java   |  75 ++
 .../org/apache/cassandra/service/paxos/Commit.java |  34 +
 .../apache/cassandra/service/paxos/PaxosState.java |   6 +-
 .../cassandra/service/paxos/ProposeCallback.java   |   3 +-
 .../service/snapshot/SnapshotManifest.java         |  14 +-
 .../cassandra/streaming/StreamResultFuture.java    |   5 +-
 .../apache/cassandra/streaming/StreamSession.java  |   9 +
 .../cassandra/streaming/StreamTransferTask.java    |  38 +-
 .../async/StreamingMultiplexedChannel.java         |   1 +
 .../cassandra/tools/BulkLoadConnectionFactory.java |   1 -
 .../cassandra/transport/CQLMessageHandler.java     |   2 +-
 .../cassandra/utils/BloomFilterSerializer.java     |   2 -
 src/java/org/apache/cassandra/utils/Clock.java     |  31 +-
 .../org/apache/cassandra/utils/FBUtilities.java    |   2 +-
 .../org/apache/cassandra/utils/MonotonicClock.java |  19 +-
 .../Nemesis.java}                                  |  27 +-
 src/java/org/apache/cassandra/utils/Simulate.java  |  56 ++
 .../org/apache/cassandra/utils/Throwables.java     |   2 +
 .../cassandra/utils/binlog/ExternalArchiver.java   |   1 -
 .../cassandra/utils/concurrent/AbstractFuture.java |  26 +-
 .../cassandra/utils/concurrent/AsyncFuture.java    |   2 +-
 .../cassandra/utils/concurrent/Awaitable.java      | 135 +--
 .../apache/cassandra/utils/concurrent/Future.java  |  11 +
 .../cassandra/utils/concurrent/ListenerList.java   |  18 +-
 .../org/apache/cassandra/utils/concurrent/Ref.java |  29 +-
 .../cassandra/utils/concurrent/Semaphore.java      | 200 +----
 .../cassandra/utils/concurrent/SyncFuture.java     |   4 +-
 .../apache/cassandra/utils/concurrent/Threads.java |   5 +
 .../apache/cassandra/utils/memory/BufferPool.java  |  19 +-
 .../apache/cassandra/utils/memory/HeapPool.java    |   4 +
 .../apache/cassandra/utils/obs/OffHeapBitSet.java  |   1 -
 test/burn/org/apache/cassandra/net/Connection.java |   2 +-
 .../apache/cassandra/net/ConnectionBurnTest.java   |   4 +-
 .../org/apache/cassandra/net/MessageGenerator.java |   2 +-
 test/burn/org/apache/cassandra/net/Verifier.java   |   2 +-
 .../cassandra/utils/memory/LongBufferPoolTest.java |   4 +-
 test/conf/logback-simulator.xml                    |  55 ++
 .../org/apache/cassandra/distributed/Cluster.java  |   4 +-
 .../cassandra/distributed/UpgradeableCluster.java  |   4 +-
 .../cassandra/distributed/action/GossipHelper.java |   5 +
 .../distributed/impl/AbstractCluster.java          |  28 +-
 .../cassandra/distributed/impl/Coordinator.java    |   2 +-
 .../cassandra/distributed/impl/Instance.java       |  13 +-
 .../cassandra/distributed/impl/InstanceConfig.java |   2 +-
 .../cassandra/distributed/impl/InstanceKiller.java |   1 -
 .../distributed/impl/IsolatedExecutor.java         |   3 +-
 .../apache/cassandra/distributed/impl/Query.java   |   4 +-
 .../distributed/impl/UnsafeGossipHelper.java       |  14 +-
 .../cassandra/distributed/upgrade/GroupByTest.java |   1 -
 .../distributed/upgrade/UpgradeTestBase.java       |   1 -
 .../io/compress/CompressorPerformance.java         |   1 -
 .../test/microbench/BatchStatementBench.java       |   1 -
 .../cassandra/simulator/asm/ChanceSupplier.java    |  12 +-
 .../cassandra/simulator/asm/ClassTransformer.java  | 210 +++++
 .../org/apache/cassandra/simulator/asm/Flag.java   |  12 +-
 .../simulator/asm/GlobalMethodTransformer.java     | 156 ++++
 .../apache/cassandra/simulator/asm/Hashcode.java   |  44 +
 .../cassandra/simulator/asm/InterceptAgent.java    | 334 ++++++++
 .../cassandra/simulator/asm/InterceptClasses.java  | 251 ++++++
 .../cassandra/simulator/asm/MethodLogger.java      | 259 ++++++
 .../cassandra/simulator/asm/MethodWriterSink.java  |  13 +-
 .../asm/MonitorEnterExitParkTransformer.java       | 123 +++
 .../simulator/asm/MonitorMethodTransformer.java    | 338 ++++++++
 .../cassandra/simulator/asm/NemesisFieldKind.java  |  50 ++
 .../cassandra/simulator/asm/NemesisGenerator.java  |  73 ++
 .../simulator/asm/NemesisTransformer.java          | 134 +++
 .../simulator/asm/ShadowingTransformer.java        | 278 ++++++
 .../simulator/asm/TransformationKind.java          |  15 +-
 .../org/apache/cassandra/simulator/asm/Utils.java  | 265 ++++++
 .../cassandra/simulator/asm/package-info.java      |  17 +-
 .../systems/InterceptorOfSystemMethods.java        | 373 +++++++++
 .../org/apache/cassandra/simulator/Action.java     | 929 +++++++++++++++++++++
 .../org/apache/cassandra/simulator/ActionList.java | 142 ++++
 .../apache/cassandra/simulator/ActionListener.java | 201 +++++
 .../org/apache/cassandra/simulator/ActionPlan.java |  91 ++
 .../apache/cassandra/simulator/ActionSchedule.java | 390 +++++++++
 .../org/apache/cassandra/simulator/Actions.java    | 158 ++++
 .../cassandra/simulator/ClusterSimulation.java     | 748 +++++++++++++++++
 .../main/org/apache/cassandra/simulator/Debug.java | 356 ++++++++
 .../cassandra/simulator/FutureActionScheduler.java |  57 ++
 .../org/apache/cassandra/simulator/OrderOn.java    | 180 ++++
 .../org/apache/cassandra/simulator/OrderOns.java   | 160 ++++
 .../org/apache/cassandra/simulator/Ordered.java    | 331 ++++++++
 .../org/apache/cassandra/simulator/OrderedOn.java  |  10 +-
 .../apache/cassandra/simulator/RandomSource.java   | 320 +++++++
 .../simulator/RunnableActionScheduler.java         | 141 ++++
 .../org/apache/cassandra/simulator/Simulation.java |  13 +-
 .../cassandra/simulator/SimulationRunner.java      | 456 ++++++++++
 .../apache/cassandra/simulator/SimulatorUtils.java |  48 ++
 .../simulator/asm/NemesisFieldSelectors.java       |  67 ++
 .../cassandra/simulator/cluster/ClusterAction.java |  17 +-
 .../simulator/cluster/ClusterActionListener.java   |  23 +-
 .../simulator/cluster/ClusterActions.java          | 267 ++++++
 .../simulator/cluster/ClusterReliableAction.java   |  16 +-
 .../cluster/ClusterReliableQueryAction.java        |  38 +
 .../simulator/cluster/ClusterUnsafeAction.java     |  22 +-
 .../simulator/cluster/KeyspaceActions.java         | 391 +++++++++
 .../cassandra/simulator/cluster/NodeLookup.java    |  32 +-
 .../cassandra/simulator/cluster/NodesByDc.java     | 172 ++++
 .../simulator/cluster/OnClusterChangeRf.java       |  66 ++
 .../simulator/cluster/OnClusterChangeTopology.java |  70 ++
 .../simulator/cluster/OnClusterFullRepair.java     |  66 ++
 .../cassandra/simulator/cluster/OnClusterJoin.java |  50 ++
 .../simulator/cluster/OnClusterLeave.java          |  60 ++
 .../simulator/cluster/OnClusterMarkDown.java       |  21 +-
 .../simulator/cluster/OnClusterRepairRanges.java   |  40 +
 .../simulator/cluster/OnClusterReplace.java        | 115 +++
 .../cluster/OnClusterSetPaxosVariant.java          |  44 +
 .../cluster/OnClusterSyncPendingRanges.java        |  15 +-
 .../simulator/cluster/OnClusterUpdateGossip.java   |  64 ++
 .../simulator/cluster/OnInstanceBootstrap.java     |  55 ++
 .../cluster/OnInstanceFlushAndCleanup.java         |  61 ++
 .../simulator/cluster/OnInstanceGossipWith.java    |  51 ++
 .../simulator/cluster/OnInstanceGossipWithAll.java |  22 +-
 .../simulator/cluster/OnInstanceMarkShutdown.java  |  15 +-
 .../simulator/cluster/OnInstanceRepair.java        | 105 +++
 .../cluster/OnInstanceSendLocalGossipState.java    |  51 ++
 .../simulator/cluster/OnInstanceSendShutdown.java  |  50 ++
 .../cluster/OnInstanceSendShutdownToAll.java       |  17 +-
 .../cluster/OnInstanceSetBootstrapReplacing.java   |  15 +-
 .../cluster/OnInstanceSetBootstrapping.java        |  15 +-
 .../simulator/cluster/OnInstanceSetLeaving.java    |  15 +-
 .../simulator/cluster/OnInstanceSetLeft.java       |  15 +-
 .../simulator/cluster/OnInstanceSetNormal.java     |  20 +-
 .../cluster/OnInstanceSetPaxosVariant.java         |  56 ++
 .../cluster/OnInstanceSyncPendingRanges.java       |  42 +
 .../cluster/OnInstanceSyncSchemaForBootstrap.java  |  18 +-
 .../cassandra/simulator/cluster/Topology.java      |  94 +++
 .../simulator/cluster/TopologyListener.java        |  12 +-
 .../apache/cassandra/simulator/cluster/Utils.java  |  84 ++
 .../apache/cassandra/simulator/debug/Capture.java  |  21 +-
 .../cassandra/simulator/debug/Reconcile.java       | 469 +++++++++++
 .../apache/cassandra/simulator/debug/Record.java   | 442 ++++++++++
 .../cassandra/simulator/debug/SelfReconcile.java   | 286 +++++++
 .../simulator/debug/SelfReconcilingRandom.java     | 136 +++
 .../apache/cassandra/simulator/package-info.java   |  84 ++
 .../apache/cassandra/simulator/paxos/Ballots.java  | 251 ++++++
 .../cassandra/simulator/paxos/HistoryChecker.java  | 350 ++++++++
 .../simulator/paxos/HistoryViolation.java          |  13 +-
 .../cassandra/simulator/paxos/Observation.java     |  31 +-
 .../paxos/PairOfSequencesPaxosSimulation.java      | 349 ++++++++
 .../simulator/paxos/PaxosClusterSimulation.java    |  99 +++
 .../simulator/paxos/PaxosRepairValidator.java      | 100 +++
 .../cassandra/simulator/paxos/PaxosSimulation.java | 237 ++++++
 .../simulator/paxos/PaxosSimulationRunner.java     | 143 ++++
 .../paxos/PaxosTopologyChangeVerifier.java         | 116 +++
 .../cassandra/simulator/systems/Failures.java      |  70 ++
 .../simulator/systems/InterceptedExecution.java    | 281 +++++++
 .../systems/InterceptedIdentityHashMap.java        | 158 ++++
 .../simulator/systems/InterceptedWait.java         | 341 ++++++++
 .../simulator/systems/InterceptibleThread.java     | 502 +++++++++++
 .../systems/InterceptibleThreadFactory.java        |  87 ++
 .../simulator/systems/InterceptingAwaitable.java   | 296 +++++++
 .../simulator/systems/InterceptingExecutor.java    | 848 +++++++++++++++++++
 .../systems/InterceptingExecutorFactory.java       | 392 +++++++++
 .../systems/InterceptingGlobalMethods.java         |  90 ++
 .../simulator/systems/InterceptingMonitors.java    | 759 +++++++++++++++++
 .../simulator/systems/InterceptingWaitQueue.java   |  99 +++
 .../systems/InterceptorOfConsequences.java         |  15 +-
 .../simulator/systems/InterceptorOfExecution.java  |  19 +-
 .../systems/InterceptorOfGlobalMethods.java        | 381 +++++++++
 .../simulator/systems/InterceptorOfWaits.java      |  22 +-
 .../cassandra/simulator/systems/NetworkConfig.java |  55 ++
 .../simulator/systems/NonInterceptible.java        |  78 ++
 .../systems/NotInterceptedSyncCondition.java       |  70 ++
 .../simulator/systems/NotifyThreadPaused.java      |  73 ++
 .../simulator/systems/PerClassLoader.java          |  11 +-
 .../simulator/systems/SchedulerConfig.java         |  21 +-
 .../simulator/systems/SimulatedAction.java         | 432 ++++++++++
 .../simulator/systems/SimulatedActionCallable.java |  54 ++
 .../simulator/systems/SimulatedActionConsumer.java |  57 ++
 .../simulator/systems/SimulatedActionTask.java     | 118 +++
 .../simulator/systems/SimulatedBallots.java        |  98 +++
 .../simulator/systems/SimulatedExecution.java      | 221 +++++
 .../systems/SimulatedFailureDetector.java          | 126 +++
 .../systems/SimulatedFutureActionScheduler.java    | 195 +++++
 .../systems/SimulatedMessageDelivery.java          |  42 +
 .../simulator/systems/SimulatedQuery.java          |  53 ++
 .../simulator/systems/SimulatedSnitch.java         | 126 +++
 .../simulator/systems/SimulatedSystems.java        | 106 +++
 .../cassandra/simulator/systems/SimulatedTime.java | 317 +++++++
 .../simulator/systems/SimulatedWaits.java          | 156 ++++
 .../cassandra/simulator/utils/ChanceRange.java     |  50 ++
 .../cassandra/simulator/utils/CompactLists.java    | 160 ++++
 .../simulator/utils/CountingCollection.java        |  38 +-
 .../apache/cassandra/simulator/utils/IntRange.java |  74 ++
 .../simulator/utils/IntrusiveLinkedList.java       |  90 ++
 .../simulator/utils/IntrusiveLinkedListNode.java   |  27 +-
 .../cassandra/simulator/utils/KindOfSequence.java  | 429 ++++++++++
 .../cassandra/simulator/utils/LongRange.java       |  96 +++
 .../cassandra/simulator/utils/SafeCollections.java |  29 +-
 .../test/ClassWithSynchronizedMethods.java         |  71 ++
 .../test/MonitorMethodTransformerTest.java         | 110 +++
 .../cassandra/simulator/test/SimulationTest.java   |  85 ++
 .../simulator/test/SimulationTestBase.java         | 283 +++++++
 .../cassandra/AbstractSerializationsTester.java    |   2 -
 .../unit/org/apache/cassandra/ServerTestUtils.java |   1 -
 .../apache/cassandra/audit/BinAuditLoggerTest.java |   1 -
 .../concurrent/AbstractExecutorPlusTest.java       |   8 +-
 .../org/apache/cassandra/db/NativeCellTest.java    |   1 -
 .../cassandra/db/RangeTombstoneListTest.java       |   1 -
 .../org/apache/cassandra/db/ReadMessageTest.java   |   1 -
 .../org/apache/cassandra/db/RowIndexEntryTest.java |   1 -
 .../db/commitlog/AbstractCommitLogServiceTest.java |   1 -
 .../db/monitoring/MonitoringTaskTest.java          |   2 +-
 .../apache/cassandra/gms/SerializationsTest.java   |   1 -
 .../apache/cassandra/hints/DTestSerializer.java    |   1 -
 .../compress/CompressedRandomAccessReaderTest.java |   1 -
 .../compress/CompressedSequentialWriterTest.java   |   1 -
 .../io/sstable/SSTableCorruptionDetectionTest.java |   1 -
 .../cassandra/io/sstable/SSTableRewriterTest.java  |   1 -
 .../org/apache/cassandra/net/ConnectionTest.java   |   2 +-
 .../unit/org/apache/cassandra/net/MessageTest.java |   2 +-
 .../cassandra/net/OutboundMessageQueueTest.java    |   2 +-
 .../cassandra/net/ProxyHandlerConnectionsTest.java |   2 +-
 .../apache/cassandra/service/PaxosStateTest.java   | 128 ---
 .../streaming/StreamTransferTaskTest.java          |   2 +-
 .../async/StreamingInboundHandlerTest.java         |   4 -
 .../tools/cassandrastress/CassandrastressTest.java |   1 -
 .../apache/cassandra/utils/BloomFilterTest.java    |   1 -
 .../apache/cassandra/utils/MonotonicClockTest.java |   2 +-
 .../apache/cassandra/utils/SerializationsTest.java |   3 -
 .../utils/concurrent/ImmediateFutureTest.java      |   1 -
 .../cassandra/utils/concurrent/SemaphoreTest.java  |   8 +-
 .../utils/memory/MemtableCleanerThreadTest.java    |   1 -
 296 files changed, 22265 insertions(+), 967 deletions(-)
 create mode 100644 src/java/org/apache/cassandra/service/paxos/BallotGenerator.java
 copy src/java/org/apache/cassandra/{concurrent/ScheduledExecutorPlus.java => utils/Nemesis.java} (50%)
 create mode 100644 src/java/org/apache/cassandra/utils/Simulate.java
 create mode 100644 test/conf/logback-simulator.xml
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/asm/org/apache/cassandra/simulator/asm/ChanceSupplier.java (71%)
 create mode 100644 test/simulator/asm/org/apache/cassandra/simulator/asm/ClassTransformer.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/asm/org/apache/cassandra/simulator/asm/Flag.java (71%)
 create mode 100644 test/simulator/asm/org/apache/cassandra/simulator/asm/GlobalMethodTransformer.java
 create mode 100644 test/simulator/asm/org/apache/cassandra/simulator/asm/Hashcode.java
 create mode 100644 test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptAgent.java
 create mode 100644 test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptClasses.java
 create mode 100644 test/simulator/asm/org/apache/cassandra/simulator/asm/MethodLogger.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/asm/org/apache/cassandra/simulator/asm/MethodWriterSink.java (71%)
 create mode 100644 test/simulator/asm/org/apache/cassandra/simulator/asm/MonitorEnterExitParkTransformer.java
 create mode 100644 test/simulator/asm/org/apache/cassandra/simulator/asm/MonitorMethodTransformer.java
 create mode 100644 test/simulator/asm/org/apache/cassandra/simulator/asm/NemesisFieldKind.java
 create mode 100644 test/simulator/asm/org/apache/cassandra/simulator/asm/NemesisGenerator.java
 create mode 100644 test/simulator/asm/org/apache/cassandra/simulator/asm/NemesisTransformer.java
 create mode 100644 test/simulator/asm/org/apache/cassandra/simulator/asm/ShadowingTransformer.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/asm/org/apache/cassandra/simulator/asm/TransformationKind.java (68%)
 create mode 100644 test/simulator/asm/org/apache/cassandra/simulator/asm/Utils.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/asm/org/apache/cassandra/simulator/asm/package-info.java (71%)
 create mode 100644 test/simulator/bootstrap/org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/Action.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/ActionList.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/ActionListener.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/ActionPlan.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/ActionSchedule.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/Actions.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/Debug.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/FutureActionScheduler.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/OrderOn.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/OrderOns.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/Ordered.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/OrderedOn.java (77%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/RandomSource.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/RunnableActionScheduler.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/Simulation.java (71%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/SimulationRunner.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/SimulatorUtils.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/asm/NemesisFieldSelectors.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterAction.java (61%)
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterActionListener.java (60%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterActions.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterReliableAction.java (62%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterReliableQueryAction.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterUnsafeAction.java (54%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/KeyspaceActions.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/NodeLookup.java (56%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/NodesByDc.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterChangeRf.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterChangeTopology.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterFullRepair.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterJoin.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterLeave.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterMarkDown.java (55%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterRepairRanges.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterReplace.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterSetPaxosVariant.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterSyncPendingRanges.java (71%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterUpdateGossip.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceBootstrap.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceFlushAndCleanup.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceGossipWith.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceGossipWithAll.java (53%)
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceMarkShutdown.java (67%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceRepair.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceSendLocalGossipState.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceSendShutdown.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceSendShutdownToAll.java (60%)
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceSetBootstrapReplacing.java (59%)
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceSetBootstrapping.java (66%)
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceSetLeaving.java (68%)
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceSetLeft.java (69%)
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceSetNormal.java (56%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceSetPaxosVariant.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceSyncPendingRanges.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceSyncSchemaForBootstrap.java (53%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/Topology.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/cluster/TopologyListener.java (71%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/cluster/Utils.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/debug/Capture.java (68%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/debug/Reconcile.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/debug/Record.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/debug/SelfReconcile.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/debug/SelfReconcilingRandom.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/package-info.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/paxos/Ballots.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/paxos/HistoryChecker.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/paxos/HistoryViolation.java (77%)
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/paxos/Observation.java (54%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/paxos/PairOfSequencesPaxosSimulation.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/paxos/PaxosClusterSimulation.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/paxos/PaxosRepairValidator.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/paxos/PaxosSimulation.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/paxos/PaxosSimulationRunner.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/paxos/PaxosTopologyChangeVerifier.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/Failures.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/InterceptedExecution.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/InterceptedIdentityHashMap.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/InterceptedWait.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/InterceptibleThread.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/InterceptibleThreadFactory.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/InterceptingAwaitable.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/InterceptingExecutor.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/InterceptingExecutorFactory.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/InterceptingGlobalMethods.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/InterceptingMonitors.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/InterceptingWaitQueue.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/systems/InterceptorOfConsequences.java (57%)
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/systems/InterceptorOfExecution.java (53%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/InterceptorOfGlobalMethods.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/systems/InterceptorOfWaits.java (50%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/NetworkConfig.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/NonInterceptible.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/NotInterceptedSyncCondition.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/NotifyThreadPaused.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/systems/PerClassLoader.java (71%)
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/systems/SchedulerConfig.java (60%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedAction.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedActionCallable.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedActionConsumer.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedActionTask.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedBallots.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedExecution.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedFailureDetector.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedFutureActionScheduler.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedMessageDelivery.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedQuery.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedSnitch.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedSystems.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedTime.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedWaits.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/utils/ChanceRange.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/utils/CompactLists.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/utils/CountingCollection.java (56%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/utils/IntRange.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/utils/IntrusiveLinkedList.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/utils/IntrusiveLinkedListNode.java (65%)
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/utils/KindOfSequence.java
 create mode 100644 test/simulator/main/org/apache/cassandra/simulator/utils/LongRange.java
 copy src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java => test/simulator/main/org/apache/cassandra/simulator/utils/SafeCollections.java (57%)
 create mode 100644 test/simulator/test/org/apache/cassandra/simulator/test/ClassWithSynchronizedMethods.java
 create mode 100644 test/simulator/test/org/apache/cassandra/simulator/test/MonitorMethodTransformerTest.java
 create mode 100644 test/simulator/test/org/apache/cassandra/simulator/test/SimulationTest.java
 create mode 100644 test/simulator/test/org/apache/cassandra/simulator/test/SimulationTestBase.java
 delete mode 100644 test/unit/org/apache/cassandra/service/PaxosStateTest.java

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


[cassandra] 03/03: [CEP-10] Phase 4: Cluster and Code Simulation

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 209c35aa62855e2834e629e8ecb38ebb329aa6ce
Author: Benedict Elliott Smith <be...@apache.org>
AuthorDate: Wed Sep 1 15:35:22 2021 +0100

    [CEP-10] Phase 4: Cluster and Code Simulation
    
    patch by Benedict; reviewed by Sam Tunnicliffe for CASSANDRA-17008
    
    Co-authored-by: Benedict Elliott Smith <be...@apache.org>
    Co-authored-by: Sam Tunnicliffe <sa...@apache.org>
    Co-authored-by: Alex Petrov <ol...@gmail.com>
---
 build.xml                                          | 167 +++-
 ide/idea-iml-file.xml                              |   3 +
 .../cassandra/concurrent/ImmediateExecutor.java    |   4 +
 .../cassandra/concurrent/InfiniteLoopExecutor.java |  17 +
 .../config/CassandraRelevantProperties.java        |   1 +
 .../org/apache/cassandra/utils/Throwables.java     |   2 +
 test/burn/org/apache/cassandra/net/Connection.java |   2 +-
 .../apache/cassandra/net/ConnectionBurnTest.java   |   4 +-
 .../org/apache/cassandra/net/MessageGenerator.java |   2 +-
 test/burn/org/apache/cassandra/net/Verifier.java   |   2 +-
 test/conf/logback-simulator.xml                    |  55 ++
 .../org/apache/cassandra/distributed/Cluster.java  |   4 +-
 .../cassandra/distributed/UpgradeableCluster.java  |   4 +-
 .../cassandra/distributed/action/GossipHelper.java |   5 +
 .../distributed/impl/AbstractCluster.java          |  28 +-
 .../cassandra/distributed/impl/Instance.java       |  13 +-
 .../cassandra/distributed/impl/InstanceConfig.java |   2 +-
 .../cassandra/distributed/impl/InstanceKiller.java |   1 -
 .../distributed/impl/IsolatedExecutor.java         |   3 +-
 .../distributed/impl/UnsafeGossipHelper.java       |  14 +-
 .../cassandra/distributed/upgrade/GroupByTest.java |   1 -
 .../distributed/upgrade/UpgradeTestBase.java       |   1 -
 .../io/compress/CompressorPerformance.java         |   1 -
 .../test/microbench/BatchStatementBench.java       |   1 -
 .../cassandra/simulator/asm/ChanceSupplier.java    |  24 +
 .../cassandra/simulator/asm/ClassTransformer.java  | 210 +++++
 .../org/apache/cassandra/simulator/asm/Flag.java   |  24 +
 .../simulator/asm/GlobalMethodTransformer.java     | 156 ++++
 .../apache/cassandra/simulator/asm/Hashcode.java   |  44 +
 .../cassandra/simulator/asm/InterceptAgent.java    | 334 ++++++++
 .../cassandra/simulator/asm/InterceptClasses.java  | 251 ++++++
 .../cassandra/simulator/asm/MethodLogger.java      | 259 ++++++
 .../cassandra/simulator/asm/MethodWriterSink.java  |  29 +
 .../asm/MonitorEnterExitParkTransformer.java       | 123 +++
 .../simulator/asm/MonitorMethodTransformer.java    | 338 ++++++++
 .../cassandra/simulator/asm/NemesisFieldKind.java  |  50 ++
 .../cassandra/simulator/asm/NemesisGenerator.java  |  73 ++
 .../simulator/asm/NemesisTransformer.java          | 134 +++
 .../simulator/asm/ShadowingTransformer.java        | 278 ++++++
 .../simulator/asm/TransformationKind.java          |  29 +
 .../org/apache/cassandra/simulator/asm/Utils.java  | 265 ++++++
 .../cassandra/simulator/asm/package-info.java      |  25 +
 .../systems/InterceptorOfSystemMethods.java        | 373 +++++++++
 .../org/apache/cassandra/simulator/Action.java     | 929 +++++++++++++++++++++
 .../org/apache/cassandra/simulator/ActionList.java | 142 ++++
 .../apache/cassandra/simulator/ActionListener.java | 201 +++++
 .../org/apache/cassandra/simulator/ActionPlan.java |  91 ++
 .../apache/cassandra/simulator/ActionSchedule.java | 390 +++++++++
 .../org/apache/cassandra/simulator/Actions.java    | 158 ++++
 .../cassandra/simulator/ClusterSimulation.java     | 748 +++++++++++++++++
 .../main/org/apache/cassandra/simulator/Debug.java | 356 ++++++++
 .../cassandra/simulator/FutureActionScheduler.java |  57 ++
 .../org/apache/cassandra/simulator/OrderOn.java    | 180 ++++
 .../org/apache/cassandra/simulator/OrderOns.java   | 160 ++++
 .../org/apache/cassandra/simulator/Ordered.java    | 331 ++++++++
 .../org/apache/cassandra/simulator/OrderedOn.java  |  32 +
 .../apache/cassandra/simulator/RandomSource.java   | 320 +++++++
 .../simulator/RunnableActionScheduler.java         | 141 ++++
 .../org/apache/cassandra/simulator/Simulation.java |  27 +
 .../cassandra/simulator/SimulationRunner.java      | 456 ++++++++++
 .../apache/cassandra/simulator/SimulatorUtils.java |  48 ++
 .../simulator/asm/NemesisFieldSelectors.java       |  67 ++
 .../cassandra/simulator/cluster/ClusterAction.java |  29 +
 .../simulator/cluster/ClusterActionListener.java   |  37 +
 .../simulator/cluster/ClusterActions.java          | 267 ++++++
 .../simulator/cluster/ClusterReliableAction.java   |  30 +
 .../cluster/ClusterReliableQueryAction.java        |  38 +
 .../simulator/cluster/ClusterUnsafeAction.java     |  34 +
 .../simulator/cluster/KeyspaceActions.java         | 391 +++++++++
 .../cassandra/simulator/cluster/NodeLookup.java    |  48 ++
 .../cassandra/simulator/cluster/NodesByDc.java     | 172 ++++
 .../simulator/cluster/OnClusterChangeRf.java       |  66 ++
 .../simulator/cluster/OnClusterChangeTopology.java |  70 ++
 .../simulator/cluster/OnClusterFullRepair.java     |  66 ++
 .../cassandra/simulator/cluster/OnClusterJoin.java |  50 ++
 .../simulator/cluster/OnClusterLeave.java          |  60 ++
 .../simulator/cluster/OnClusterMarkDown.java       |  37 +
 .../simulator/cluster/OnClusterRepairRanges.java   |  40 +
 .../simulator/cluster/OnClusterReplace.java        | 115 +++
 .../cluster/OnClusterSetPaxosVariant.java          |  44 +
 .../cluster/OnClusterSyncPendingRanges.java        |  29 +
 .../simulator/cluster/OnClusterUpdateGossip.java   |  64 ++
 .../simulator/cluster/OnInstanceBootstrap.java     |  55 ++
 .../cluster/OnInstanceFlushAndCleanup.java         |  61 ++
 .../simulator/cluster/OnInstanceGossipWith.java    |  51 ++
 .../simulator/cluster/OnInstanceGossipWithAll.java |  38 +
 .../simulator/cluster/OnInstanceMarkShutdown.java  |  29 +
 .../simulator/cluster/OnInstanceRepair.java        | 105 +++
 .../cluster/OnInstanceSendLocalGossipState.java    |  51 ++
 .../simulator/cluster/OnInstanceSendShutdown.java  |  50 ++
 .../cluster/OnInstanceSendShutdownToAll.java       |  33 +
 .../cluster/OnInstanceSetBootstrapReplacing.java   |  31 +
 .../cluster/OnInstanceSetBootstrapping.java        |  29 +
 .../simulator/cluster/OnInstanceSetLeaving.java    |  29 +
 .../simulator/cluster/OnInstanceSetLeft.java       |  29 +
 .../simulator/cluster/OnInstanceSetNormal.java     |  36 +
 .../cluster/OnInstanceSetPaxosVariant.java         |  56 ++
 .../cluster/OnInstanceSyncPendingRanges.java       |  42 +
 .../cluster/OnInstanceSyncSchemaForBootstrap.java  |  34 +
 .../cassandra/simulator/cluster/Topology.java      |  94 +++
 .../simulator/cluster/TopologyListener.java        |  24 +
 .../apache/cassandra/simulator/cluster/Utils.java  |  84 ++
 .../apache/cassandra/simulator/debug/Capture.java  |  33 +
 .../cassandra/simulator/debug/Reconcile.java       | 469 +++++++++++
 .../apache/cassandra/simulator/debug/Record.java   | 442 ++++++++++
 .../cassandra/simulator/debug/SelfReconcile.java   | 286 +++++++
 .../simulator/debug/SelfReconcilingRandom.java     | 136 +++
 .../apache/cassandra/simulator/package-info.java   |  84 ++
 .../apache/cassandra/simulator/paxos/Ballots.java  | 251 ++++++
 .../cassandra/simulator/paxos/HistoryChecker.java  | 350 ++++++++
 .../simulator/paxos/HistoryViolation.java          |  35 +
 .../cassandra/simulator/paxos/Observation.java     |  45 +
 .../paxos/PairOfSequencesPaxosSimulation.java      | 349 ++++++++
 .../simulator/paxos/PaxosClusterSimulation.java    |  99 +++
 .../simulator/paxos/PaxosRepairValidator.java      | 100 +++
 .../cassandra/simulator/paxos/PaxosSimulation.java | 237 ++++++
 .../simulator/paxos/PaxosSimulationRunner.java     | 143 ++++
 .../paxos/PaxosTopologyChangeVerifier.java         | 116 +++
 .../cassandra/simulator/systems/Failures.java      |  70 ++
 .../simulator/systems/InterceptedExecution.java    | 281 +++++++
 .../systems/InterceptedIdentityHashMap.java        | 158 ++++
 .../simulator/systems/InterceptedWait.java         | 341 ++++++++
 .../simulator/systems/InterceptibleThread.java     | 502 +++++++++++
 .../systems/InterceptibleThreadFactory.java        |  87 ++
 .../simulator/systems/InterceptingAwaitable.java   | 296 +++++++
 .../simulator/systems/InterceptingExecutor.java    | 848 +++++++++++++++++++
 .../systems/InterceptingExecutorFactory.java       | 392 +++++++++
 .../systems/InterceptingGlobalMethods.java         |  90 ++
 .../simulator/systems/InterceptingMonitors.java    | 759 +++++++++++++++++
 .../simulator/systems/InterceptingWaitQueue.java   |  99 +++
 .../systems/InterceptorOfConsequences.java         |  37 +
 .../simulator/systems/InterceptorOfExecution.java  |  43 +
 .../systems/InterceptorOfGlobalMethods.java        | 381 +++++++++
 .../simulator/systems/InterceptorOfWaits.java      |  46 +
 .../cassandra/simulator/systems/NetworkConfig.java |  55 ++
 .../simulator/systems/NonInterceptible.java        |  78 ++
 .../systems/NotInterceptedSyncCondition.java       |  70 ++
 .../simulator/systems/NotifyThreadPaused.java      |  73 ++
 .../simulator/systems/PerClassLoader.java          |  23 +
 .../simulator/systems/SchedulerConfig.java         |  35 +
 .../simulator/systems/SimulatedAction.java         | 432 ++++++++++
 .../simulator/systems/SimulatedActionCallable.java |  54 ++
 .../simulator/systems/SimulatedActionConsumer.java |  57 ++
 .../simulator/systems/SimulatedActionTask.java     | 118 +++
 .../simulator/systems/SimulatedBallots.java        |  98 +++
 .../simulator/systems/SimulatedExecution.java      | 221 +++++
 .../systems/SimulatedFailureDetector.java          | 126 +++
 .../systems/SimulatedFutureActionScheduler.java    | 195 +++++
 .../systems/SimulatedMessageDelivery.java          |  42 +
 .../simulator/systems/SimulatedQuery.java          |  53 ++
 .../simulator/systems/SimulatedSnitch.java         | 126 +++
 .../simulator/systems/SimulatedSystems.java        | 106 +++
 .../cassandra/simulator/systems/SimulatedTime.java | 317 +++++++
 .../simulator/systems/SimulatedWaits.java          | 156 ++++
 .../cassandra/simulator/utils/ChanceRange.java     |  50 ++
 .../cassandra/simulator/utils/CompactLists.java    | 160 ++++
 .../simulator/utils/CountingCollection.java        |  54 ++
 .../apache/cassandra/simulator/utils/IntRange.java |  74 ++
 .../simulator/utils/IntrusiveLinkedList.java       |  90 ++
 .../simulator/utils/IntrusiveLinkedListNode.java   |  41 +
 .../cassandra/simulator/utils/KindOfSequence.java  | 429 ++++++++++
 .../cassandra/simulator/utils/LongRange.java       |  96 +++
 .../cassandra/simulator/utils/SafeCollections.java |  45 +
 .../test/ClassWithSynchronizedMethods.java         |  71 ++
 .../test/MonitorMethodTransformerTest.java         | 110 +++
 .../cassandra/simulator/test/SimulationTest.java   |  85 ++
 .../simulator/test/SimulationTestBase.java         | 283 +++++++
 .../cassandra/AbstractSerializationsTester.java    |   2 -
 .../unit/org/apache/cassandra/ServerTestUtils.java |   1 -
 .../apache/cassandra/audit/BinAuditLoggerTest.java |   1 -
 .../org/apache/cassandra/db/NativeCellTest.java    |   1 -
 .../cassandra/db/RangeTombstoneListTest.java       |   1 -
 .../org/apache/cassandra/db/ReadMessageTest.java   |   1 -
 .../org/apache/cassandra/db/RowIndexEntryTest.java |   1 -
 .../db/commitlog/AbstractCommitLogServiceTest.java |   1 -
 .../db/monitoring/MonitoringTaskTest.java          |   2 +-
 .../apache/cassandra/gms/SerializationsTest.java   |   1 -
 .../apache/cassandra/hints/DTestSerializer.java    |   1 -
 .../compress/CompressedRandomAccessReaderTest.java |   1 -
 .../compress/CompressedSequentialWriterTest.java   |   1 -
 .../io/sstable/SSTableCorruptionDetectionTest.java |   1 -
 .../cassandra/io/sstable/SSTableRewriterTest.java  |   1 -
 .../org/apache/cassandra/net/ConnectionTest.java   |   2 +-
 .../unit/org/apache/cassandra/net/MessageTest.java |   2 +-
 .../cassandra/net/OutboundMessageQueueTest.java    |   2 +-
 .../cassandra/net/ProxyHandlerConnectionsTest.java |   2 +-
 .../apache/cassandra/service/PaxosStateTest.java   | 128 ---
 .../streaming/StreamTransferTaskTest.java          |   1 -
 .../async/StreamingInboundHandlerTest.java         |   4 -
 .../tools/cassandrastress/CassandrastressTest.java |   1 -
 .../apache/cassandra/utils/BloomFilterTest.java    |   1 -
 .../apache/cassandra/utils/MonotonicClockTest.java |   2 +-
 .../apache/cassandra/utils/SerializationsTest.java |   3 -
 .../utils/concurrent/ImmediateFutureTest.java      |   1 -
 .../utils/memory/MemtableCleanerThreadTest.java    |   1 -
 195 files changed, 22402 insertions(+), 231 deletions(-)

diff --git a/build.xml b/build.xml
index 2c521cb..54fa104 100644
--- a/build.xml
+++ b/build.xml
@@ -62,6 +62,9 @@
     <property name="test.microbench.src" value="${test.dir}/microbench"/>
     <property name="test.distributed.src" value="${test.dir}/distributed"/>
     <property name="test.compression.algo" value="LZ4"/>
+    <property name="test.simulator.src" value="${test.dir}/simulator/main"/>
+    <property name="test.simulator-asm.src" value="${test.dir}/simulator/asm"/>
+    <property name="test.simulator-bootstrap.src" value="${test.dir}/simulator/bootstrap"/>
     <property name="test.driver.connection_timeout_ms" value="5000"/>
     <property name="test.driver.read_timeout_ms" value="12000"/>
     <property name="dist.dir" value="${build.dir}/dist"/>
@@ -118,7 +121,7 @@
     <property name="jamm.version" value="0.3.2"/>
     <property name="ecj.version" value="4.6.1"/>
     <property name="ohc.version" value="0.5.1"/>
-    <property name="asm.version" value="7.1"/>
+    <property name="asm.version" value="9.1"/>
     <property name="allocation-instrumenter.version" value="3.1.0"/>
     <property name="bytebuddy.version" value="1.10.10"/>
     <property name="jflex.version" value="1.8.2"/>
@@ -500,6 +503,7 @@
             <exclusion groupId="org.checkerframework" artifactId="checker-qual" />
             <exclusion groupId="com.google.errorprone" artifactId="error_prone_annotations" />
           </dependency>
+          <dependency groupId="com.google.jimfs" artifactId="jimfs" version="1.1"/>
           <dependency groupId="org.hdrhistogram" artifactId="HdrHistogram" version="2.1.9"/>
           <dependency groupId="commons-cli" artifactId="commons-cli" version="1.1"/>
           <dependency groupId="commons-codec" artifactId="commons-codec" version="1.9"/>
@@ -554,7 +558,9 @@
           <dependency groupId="net.java.dev.jna" artifactId="jna" version="5.6.0"/>
 
           <dependency groupId="org.jacoco" artifactId="org.jacoco.agent" version="${jacoco.version}" scope="test"/>
-          <dependency groupId="org.jacoco" artifactId="org.jacoco.ant" version="${jacoco.version}" scope="test"/>
+          <dependency groupId="org.jacoco" artifactId="org.jacoco.ant" version="${jacoco.version}" scope="test">
+            <exclusion groupId="org.ow2.asm" artifactId="asm"/>
+          </dependency>
 
           <dependency groupId="org.jboss.byteman" artifactId="byteman-install" version="${byteman.version}" scope="provided"/>
           <dependency groupId="org.jboss.byteman" artifactId="byteman" version="${byteman.version}" scope="provided"/>
@@ -635,6 +641,7 @@
           <dependency groupId="org.ow2.asm" artifactId="asm" version="${asm.version}"/>
           <dependency groupId="org.ow2.asm" artifactId="asm-tree" version="${asm.version}" scope="test"/>
           <dependency groupId="org.ow2.asm" artifactId="asm-commons" version="${asm.version}" scope="test"/>
+          <dependency groupId="org.ow2.asm" artifactId="asm-util" version="${asm.version}" scope="test"/>
           <dependency groupId="org.gridkit.jvmtool" artifactId="sjk-cli" version="0.14"/>
           <dependency groupId="org.gridkit.jvmtool" artifactId="sjk-core" version="0.14">
             <exclusion groupId="org.gridkit.jvmtool" artifactId="sjk-hflame"/>
@@ -721,6 +728,11 @@
         <dependency groupId="junit" artifactId="junit" scope="test"/>
         <dependency groupId="commons-io" artifactId="commons-io" scope="test"/>
         <dependency groupId="org.mockito" artifactId="mockito-core" scope="test"/>
+        <dependency groupId="org.ow2.asm" artifactId="asm" version="${asm.version}"/>
+        <dependency groupId="org.ow2.asm" artifactId="asm-tree" version="${asm.version}" scope="test"/>
+        <dependency groupId="org.ow2.asm" artifactId="asm-commons" version="${asm.version}" scope="test"/>
+        <dependency groupId="org.ow2.asm" artifactId="asm-util" version="${asm.version}" scope="test"/>
+        <dependency groupId="com.google.jimfs" artifactId="jimfs" version="1.1" scope="test"/>
         <dependency groupId="com.puppycrawl.tools" artifactId="checkstyle" scope="test"/>
         <dependency groupId="org.quicktheories" artifactId="quicktheories" scope="test"/>
         <dependency groupId="org.reflections" artifactId="reflections" scope="test"/>
@@ -875,6 +887,7 @@
         The build target builds all the .class files
     -->
     <target name="build" depends="resolver-retrieve-build,build-project,checkstyle" description="Compile Cassandra classes"/>
+    <target name="_build_unsafe" depends="resolver-retrieve-build,build-project" description="Compile Cassandra classes without checks"/>
     <target name="codecoverage" depends="jacoco-run,jacoco-report" description="Create code coverage report"/>
 
     <target name="_build_java">
@@ -1005,6 +1018,59 @@
         </testmacro>
     </target>
 
+    <!--
+        simulator asm build file
+        -->
+    <property name="simulator-asm.build.src" value="${test.simulator-asm.src}" />
+    <property name="simulator-asm.build.classes" value="${build.classes}/simulator-asm" />
+    <property name="simulator-asm.manifest" value="${simulator-asm.build.classes}/MANIFEST.MF" />
+
+    <property name="simulator-bootstrap.build.src" value="${test.simulator-bootstrap.src}" />
+    <property name="simulator-bootstrap.build.classes" value="${build.classes}/simulator-bootstrap" />
+    <property name="simulator-bootstrap.manifest" value="${simulator-bootstrap.build.classes}/MANIFEST.MF" />
+
+    <target name="simulator-asm-build" depends="_build_unsafe" description="build simulator-asm">
+        <antcall target="_simulator-asm_build"/>
+    </target>
+
+    <target name="simulator-bootstrap-build" depends="_build_unsafe" description="build simulator-bootstrap">
+        <antcall target="_simulator-bootstrap_build"/>
+    </target>
+
+    <target name="_simulator-asm_build">
+    	<mkdir dir="${simulator-asm.build.classes}" />
+        <javac compiler="modern" debug="true" debuglevel="${debuglevel}"
+               source="${source.version}" target="${target.version}"
+               encoding="utf-8" destdir="${simulator-asm.build.classes}" includeantruntime="true">
+            <src path="${simulator-asm.build.src}" />
+            <classpath>
+                <fileset dir="${test.lib}">
+                     <include name="**/asm-*${asm.version}.jar" /> 
+                </fileset>
+                <fileset dir="${build.lib}">
+                     <include name="**/asm-*${asm.version}.jar" />
+                </fileset>
+            </classpath>
+        </javac>
+    </target>
+
+    <target name="_simulator-bootstrap_build">
+    	<mkdir dir="${simulator-bootstrap.build.classes}" />
+        <javac compiler="modern" debug="true" debuglevel="${debuglevel}"
+               source="${source.version}" target="${target.version}"
+               encoding="utf-8" destdir="${simulator-bootstrap.build.classes}" includeantruntime="true">
+            <src path="${simulator-bootstrap.build.src}" />
+            <classpath>
+                <fileset dir="${test.lib}">
+                     <include name="**/asm-*${asm.version}.jar" />
+                </fileset>
+                <fileset dir="${build.lib}">
+                     <include name="**/asm-*${asm.version}.jar" />
+                </fileset>
+            </classpath>
+        </javac>
+    </target>
+
 	<target name="_write-poms" depends="maven-declare-dependencies">
 	    <artifact:writepom pomRefId="parent-pom" file="${build.dir}/${final.name}-parent.pom"/>
 	    <artifact:writepom pomRefId="all-pom" file="${build.dir}/${final.name}.pom"/>
@@ -1041,29 +1107,76 @@
         </manifest>
       </jar>
     </target>
+
+    <target name="stress-jar"
+            depends="stress-build"
+            description="Assemble Cassandra JAR files">
+        <!-- Stress jar -->
+        <manifest file="${stress.manifest}">
+            <attribute name="Built-By" value="Pavel Yaskevich"/>
+            <attribute name="Main-Class" value="org.apache.cassandra.stress.Stress"/>
+        </manifest>
+        <mkdir dir="${stress.build.classes}/META-INF" />
+        <mkdir dir="${build.dir}/tools/lib/" />
+        <jar destfile="${build.dir}/tools/lib/stress.jar" manifest="${stress.manifest}">
+            <fileset dir="${stress.build.classes}"/>
+        </jar>
+    </target>
+
+    <target name="fqltool-jar"
+            depends="fqltool-build"
+            description="Assemble Cassandra JAR files">
+
+        <!-- fqltool jar -->
+        <manifest file="${fqltool.manifest}">
+            <attribute name="Built-By" value="Marcus Eriksson"/>
+            <attribute name="Main-Class" value="org.apache.cassandra.fqltool.FullQueryLogTool"/>
+        </manifest>
+        <mkdir dir="${fqltool.build.classes}/META-INF" />
+        <mkdir dir="${build.dir}/tools/lib/" />
+        <jar destfile="${build.dir}/tools/lib/fqltool.jar" manifest="${stress.manifest}">
+            <fileset dir="${fqltool.build.classes}"/>
+        </jar>
+    </target>
+
+    <target name="simulator-jars"
+            depends="simulator-asm-build,simulator-bootstrap-build"
+            description="Assemble Cassandra JAR files">
+
+        <!-- simulator asm jar -->
+        <manifest file="${simulator-asm.manifest}">
+            <attribute name="Built-By" value="Benedict Elliott Smith"/>
+            <attribute name="Premain-Class" value="org.apache.cassandra.simulator.asm.InterceptAgent"/>
+            <attribute name="Agent-Class" value="org.apache.cassandra.simulator.asm.InterceptAgent"/>
+            <attribute name="Can-Redefine-Classes" value="true"/>
+            <attribute name="Can-Retransform-Classes" value="true"/>
+        </manifest>
+        <mkdir dir="${simulator-asm.build.classes}/META-INF" />
+        <mkdir dir="${build.dir}/test/lib/" />
+        <jar destfile="${build.dir}/test/lib/simulator-asm.jar" manifest="${simulator-asm.manifest}">
+            <fileset dir="${simulator-asm.build.classes}"/>
+            <fileset dir="${test.lib}">
+                <include name="**/asm-*${asm.version}.jar" />
+            </fileset>
+        </jar>
+
+        <!-- simulator bootstrap jar -->
+        <manifest file="${simulator-bootstrap.manifest}">
+            <attribute name="Built-By" value="Benedict Elliott Smith"/>
+        </manifest>
+        <mkdir dir="${simulator-bootstrap.build.classes}/META-INF" />
+        <mkdir dir="${build.dir}/test/lib/" />
+        <jar destfile="${build.dir}/test/lib/simulator-bootstrap.jar" manifest="${simulator-bootstrap.manifest}">
+            <fileset dir="${simulator-bootstrap.build.classes}"/>
+            <fileset dir="${test.lib}">
+                <include name="**/asm-*${asm.version}.jar" />
+            </fileset>
+        </jar>
+    </target>
+
     <target name="jar"
-            depends="_main-jar,build-test,stress-build,fqltool-build,write-poms"
+            depends="_main-jar,build-test,stress-jar,fqltool-jar,simulator-jars,write-poms"
             description="Assemble Cassandra JAR files">
-      <!-- Stress jar -->
-      <manifest file="${stress.manifest}">
-        <attribute name="Built-By" value="Pavel Yaskevich"/>
-        <attribute name="Main-Class" value="org.apache.cassandra.stress.Stress"/>
-      </manifest>
-      <mkdir dir="${stress.build.classes}/META-INF" />
-      <mkdir dir="${build.dir}/tools/lib/" />
-      <jar destfile="${build.dir}/tools/lib/stress.jar" manifest="${stress.manifest}">
-        <fileset dir="${stress.build.classes}"/>
-      </jar>
-      <!-- fqltool jar -->
-      <manifest file="${fqltool.manifest}">
-        <attribute name="Built-By" value="Marcus Eriksson"/>
-        <attribute name="Main-Class" value="org.apache.cassandra.fqltool.FullQueryLogTool"/>
-      </manifest>
-      <mkdir dir="${fqltool.build.classes}/META-INF" />
-      <mkdir dir="${build.dir}/tools/lib/" />
-      <jar destfile="${build.dir}/tools/lib/fqltool.jar" manifest="${stress.manifest}">
-        <fileset dir="${fqltool.build.classes}"/>
-      </jar>
     </target>
 
     <!--
@@ -1278,6 +1391,9 @@
      <src path="${test.memory.src}"/>
      <src path="${test.microbench.src}"/>
      <src path="${test.distributed.src}"/>
+     <src path="${test.simulator.src}"/>
+     <src path="${test.simulator-asm.src}"/>
+     <src path="${test.simulator-bootstrap.src}"/>
     </javac>
 
     <!-- Non-java resources needed by the test suite -->
@@ -1857,6 +1973,7 @@
       <jar jarfile="${build.dir}/dtest-${base.version}.jar">
           <zipgroupfileset dir="${build.lib}" includes="*.jar" excludes="META-INF/*.SF"/>
           <zipgroupfileset dir="${build.dir.lib}/jars" includes="javassist-*.jar,reflections-*.jar" excludes="META-INF/*.SF"/>
+          <zipgroupfileset dir="${build.dir.lib}/jars" includes="jimfs-1.1.jar" excludes="META-INF/*.SF"/>
           <fileset dir="${build.classes.main}"/>
           <fileset dir="${test.classes}"/>
           <fileset dir="${test.conf}" />
@@ -1944,7 +2061,7 @@
               <pathelement location="${test.classes}"/>
               <pathelement location="${test.conf}"/>
               <fileset dir="${test.lib}">
-                  <include name="**/*.jar" />
+                  asm-<include name="**/*.jar" />
               </fileset>
           </classpath>
       </java>
@@ -2015,6 +2132,8 @@
   <classpathentry kind="src" output="build/test/classes" path="test/unit"/>
   <classpathentry kind="src" output="build/test/classes" path="test/long"/>
   <classpathentry kind="src" output="build/test/classes" path="test/distributed"/>
+  <classpathentry kind="src" output="build/test/classes" path="test/simulator/asm"/>
+  <classpathentry kind="src" output="build/test/classes" path="test/simulator/main"/>
   <classpathentry kind="src" output="build/test/classes" path="test/resources" />
   <classpathentry kind="src" path="tools/stress/src"/>
   <classpathentry kind="src" path="tools/fqltool/src"/>
diff --git a/ide/idea-iml-file.xml b/ide/idea-iml-file.xml
index 59fb99e..c0b5584 100644
--- a/ide/idea-iml-file.xml
+++ b/ide/idea-iml-file.xml
@@ -36,6 +36,9 @@
             <sourceFolder url="file://$MODULE_DIR$/test/microbench" isTestSource="true" />
             <sourceFolder url="file://$MODULE_DIR$/test/burn" isTestSource="true" />
             <sourceFolder url="file://$MODULE_DIR$/test/distributed" isTestSource="true" />
+            <sourceFolder url="file://$MODULE_DIR$/test/simulator/asm" isTestSource="true" />
+            <sourceFolder url="file://$MODULE_DIR$/test/simulator/bootstrap" isTestSource="true" />
+            <sourceFolder url="file://$MODULE_DIR$/test/simulator/main" isTestSource="true" />
             <sourceFolder url="file://$MODULE_DIR$/test/resources" type="java-test-resource" />
             <sourceFolder url="file://$MODULE_DIR$/test/conf" type="java-test-resource" />
             <excludeFolder url="file://$MODULE_DIR$/.idea" />
diff --git a/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java b/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java
index 14777ff..5edc5be 100644
--- a/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java
@@ -19,6 +19,7 @@
 package org.apache.cassandra.concurrent;
 
 import org.apache.cassandra.utils.Closeable;
+import org.apache.cassandra.utils.Shared;
 import org.apache.cassandra.utils.WithResources;
 import org.apache.cassandra.utils.concurrent.Future;
 import org.apache.cassandra.utils.concurrent.ImmediateFuture;
@@ -28,6 +29,9 @@ import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
+
+@Shared(scope = SIMULATION) // shared to support instanceof check in SimulatedAction
 public class ImmediateExecutor implements LocalAwareExecutorPlus
 {
     public static final ImmediateExecutor INSTANCE = new ImmediateExecutor();
diff --git a/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java b/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java
index 6ef439a..e1d25e5 100644
--- a/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import java.util.function.BiFunction;
 import java.util.function.Consumer;
 
+import org.apache.cassandra.utils.Shared;
 import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.InternalState.TERMINATED;
@@ -41,9 +42,16 @@ public class InfiniteLoopExecutor implements Interruptible
 {
     private static final Logger logger = LoggerFactory.getLogger(InfiniteLoopExecutor.class);
 
+    @Shared(scope = Shared.Scope.SIMULATION)
     public enum InternalState { TERMINATED }
+
+    @Shared(scope = Shared.Scope.SIMULATION)
     public enum SimulatorSafe { SAFE, UNSAFE }
+
+    @Shared(scope = Shared.Scope.SIMULATION)
     public enum Daemon        { DAEMON, NON_DAEMON }
+
+    @Shared(scope = Shared.Scope.SIMULATION)
     public enum Interrupts    { SYNCHRONIZED, UNSYNCHRONIZED }
 
     private static final AtomicReferenceFieldUpdater<InfiniteLoopExecutor, Object> stateUpdater = AtomicReferenceFieldUpdater.newUpdater(InfiniteLoopExecutor.class, Object.class, "state");
@@ -71,6 +79,15 @@ public class InfiniteLoopExecutor implements Interruptible
                                 : Thread::interrupt;
     }
 
+    public InfiniteLoopExecutor(BiFunction<String, Runnable, Thread> threadStarter, String name, Task task, Interrupts interrupts)
+    {
+        this.task = task;
+        this.thread = threadStarter.apply(name, this::loop);
+        this.interruptHandler = interrupts == SYNCHRONIZED
+                                ? interruptHandler(task)
+                                : Thread::interrupt;
+    }
+
     private static Consumer<Thread> interruptHandler(final Object monitor)
     {
         return thread -> {
diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
index 807516c..5e2f0a8 100644
--- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
+++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
@@ -229,6 +229,7 @@ public enum CassandraRelevantProperties
     TEST_SIMULATOR_PRINT_ASM("cassandra.test.simulator.print_asm", "none"),
     TEST_SIMULATOR_PRINT_ASM_TYPES("cassandra.test.simulator.print_asm_types", ""),
     TEST_SIMULATOR_LIVENESS_CHECK("cassandra.test.simulator.livenesscheck", "true"),
+    TEST_SIMULATOR_DEBUG("cassandra.test.simulator.debug", "true"),
 
     // determinism properties for testing
     DETERMINISM_SSTABLE_COMPRESSION_DEFAULT("cassandra.sstable_compression_default", "true"),
diff --git a/src/java/org/apache/cassandra/utils/Throwables.java b/src/java/org/apache/cassandra/utils/Throwables.java
index 7012132..8337a56 100644
--- a/src/java/org/apache/cassandra/utils/Throwables.java
+++ b/src/java/org/apache/cassandra/utils/Throwables.java
@@ -61,6 +61,8 @@ public final class Throwables
     {
         if (existingFail == null)
             return newFail;
+        if (newFail == null)
+            return existingFail;
         existingFail.addSuppressed(newFail);
         return existingFail;
     }
diff --git a/test/burn/org/apache/cassandra/net/Connection.java b/test/burn/org/apache/cassandra/net/Connection.java
index c74c0ae..de5df6b 100644
--- a/test/burn/org/apache/cassandra/net/Connection.java
+++ b/test/burn/org/apache/cassandra/net/Connection.java
@@ -35,7 +35,7 @@ import org.apache.cassandra.net.Verifier.Destiny;
 import static org.apache.cassandra.net.MessagingService.VERSION_40;
 import static org.apache.cassandra.net.MessagingService.current_version;
 import static org.apache.cassandra.utils.ExecutorUtils.runWithThreadName;
-import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime;
 
 public class Connection implements InboundMessageCallbacks, OutboundMessageCallbacks, OutboundDebugCallbacks
 {
diff --git a/test/burn/org/apache/cassandra/net/ConnectionBurnTest.java b/test/burn/org/apache/cassandra/net/ConnectionBurnTest.java
index 9681224..6b0f5f5 100644
--- a/test/burn/org/apache/cassandra/net/ConnectionBurnTest.java
+++ b/test/burn/org/apache/cassandra/net/ConnectionBurnTest.java
@@ -66,8 +66,8 @@ import static java.lang.Math.min;
 import static org.apache.cassandra.net.MessagingService.current_version;
 import static org.apache.cassandra.net.ConnectionType.LARGE_MESSAGES;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
-import static org.apache.cassandra.utils.MonotonicClock.approxTime;
-import static org.apache.cassandra.utils.MonotonicClock.preciseTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.preciseTime;
 
 public class ConnectionBurnTest
 {
diff --git a/test/burn/org/apache/cassandra/net/MessageGenerator.java b/test/burn/org/apache/cassandra/net/MessageGenerator.java
index 3c03689..43ea16e 100644
--- a/test/burn/org/apache/cassandra/net/MessageGenerator.java
+++ b/test/burn/org/apache/cassandra/net/MessageGenerator.java
@@ -31,7 +31,7 @@ import org.apache.cassandra.utils.vint.VIntCoding;
 import sun.misc.Unsafe;
 
 import static org.apache.cassandra.net.MessagingService.VERSION_40;
-import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime;
 
 abstract class MessageGenerator
 {
diff --git a/test/burn/org/apache/cassandra/net/Verifier.java b/test/burn/org/apache/cassandra/net/Verifier.java
index 2776dcb..cd6bdc3 100644
--- a/test/burn/org/apache/cassandra/net/Verifier.java
+++ b/test/burn/org/apache/cassandra/net/Verifier.java
@@ -63,7 +63,7 @@ import static org.apache.cassandra.net.Verifier.EventType.SENT_FRAME;
 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.MonotonicClock.Global.approxTime;
 import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue;
 
 /**
diff --git a/test/conf/logback-simulator.xml b/test/conf/logback-simulator.xml
new file mode 100644
index 0000000..a7e286f
--- /dev/null
+++ b/test/conf/logback-simulator.xml
@@ -0,0 +1,55 @@
+<!--
+  ~ 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.
+  -->
+
+<configuration debug="false" scan="true" scanPeriod="60 seconds">
+  <define name="cluster_id" class="org.apache.cassandra.distributed.impl.ClusterIDDefiner" />
+  <define name="instance_id" class="org.apache.cassandra.distributed.impl.InstanceIDDefiner" />
+
+  <!-- Shutdown hook ensures that async appender flushes -->
+  <shutdownHook class="ch.qos.logback.core.hook.DelayingShutdownHook"/>
+
+  <appender name="INSTANCEFILE" class="ch.qos.logback.core.FileAppender">
+    <file>./build/test/logs/${cassandra.testtag}/${suitename}/${cluster_id}/${instance_id}/system.log</file>
+    <encoder>
+      <pattern>%-5level [%thread] ${instance_id} %date{ISO8601} %msg%n</pattern>
+    </encoder>
+    <immediateFlush>true</immediateFlush>
+  </appender>
+
+  <appender name="STDOUT" target="System.out" class="ch.qos.logback.core.ConsoleAppender">
+    <encoder>
+      <pattern>%-5level [%thread] ${instance_id} %date{ISO8601} %F:%L - %msg%n</pattern>
+    </encoder>
+    <filter class="ch.qos.logback.classic.filter.ThresholdFilter">
+      <level>WARN</level>
+    </filter>
+  </appender>
+
+  <logger name="org.apache.hadoop" level="WARN"/>
+  <logger name="org.apache.cassandra.utils.SigarLibrary" level="ERROR"/>
+  <logger name="org.apache.cassandra.utils.FBUtilities" level="ERROR"/>
+  <logger name="org.apache.cassandra.config.DatabaseDescriptor" level="ERROR"/>
+  <logger name="org.apache.cassandra.service.StartupChecks" level="ERROR"/>
+  <logger name="io.netty.handler.ssl.SslHandler" level="WARN"/>
+
+  <root level="INFO">
+    <appender-ref ref="INSTANCEFILE" />
+    <appender-ref ref="STDOUT" />
+  </root>
+</configuration>
+
diff --git a/test/distributed/org/apache/cassandra/distributed/Cluster.java b/test/distributed/org/apache/cassandra/distributed/Cluster.java
index 116dcd6..4e16905 100644
--- a/test/distributed/org/apache/cassandra/distributed/Cluster.java
+++ b/test/distributed/org/apache/cassandra/distributed/Cluster.java
@@ -38,9 +38,9 @@ public class Cluster extends AbstractCluster<IInvokableInstance>
         super(builder);
     }
 
-    protected IInvokableInstance newInstanceWrapper(int generation, Versions.Version version, IInstanceConfig config)
+    protected IInvokableInstance newInstanceWrapper(Versions.Version version, IInstanceConfig config)
     {
-        return new Wrapper(generation, version, config);
+        return new Wrapper(version, config);
     }
 
     public static Builder build()
diff --git a/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java b/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java
index 7a4d2bb..b7bd1d6 100644
--- a/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java
+++ b/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java
@@ -40,10 +40,10 @@ public class UpgradeableCluster extends AbstractCluster<IUpgradeableInstance> im
         super(builder);
     }
 
-    protected IUpgradeableInstance newInstanceWrapper(int generation, Versions.Version version, IInstanceConfig config)
+    protected IUpgradeableInstance newInstanceWrapper(Versions.Version version, IInstanceConfig config)
     {
         config.set(Constants.KEY_DTEST_API_CONFIG_CHECK, false);
-        return new Wrapper(generation, version, config);
+        return new Wrapper(version, config);
     }
 
     public static Builder build()
diff --git a/test/distributed/org/apache/cassandra/distributed/action/GossipHelper.java b/test/distributed/org/apache/cassandra/distributed/action/GossipHelper.java
index 4315834..683713a 100644
--- a/test/distributed/org/apache/cassandra/distributed/action/GossipHelper.java
+++ b/test/distributed/org/apache/cassandra/distributed/action/GossipHelper.java
@@ -54,6 +54,11 @@ import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
 
 public class GossipHelper
 {
+    public static InstanceAction statusToBlank(IInvokableInstance newNode)
+    {
+        return (instance) -> changeGossipState(instance, newNode,Collections.emptyList());
+    }
+
     public static InstanceAction statusToBootstrap(IInvokableInstance newNode)
     {
         return (instance) ->
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
index 1cb7c13..fc70ce1 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
@@ -43,7 +43,6 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.BiPredicate;
 import java.util.function.Consumer;
 import java.util.function.Function;
@@ -139,7 +138,6 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
     // to ensure we have instantiated the main classloader's LoggerFactory (and any LogbackStatusListener)
     // before we instantiate any for a new instance
     private static final Logger logger = LoggerFactory.getLogger(AbstractCluster.class);
-    private static final AtomicInteger GENERATION = new AtomicInteger();
 
     // include byteman so tests can use
     public static final Predicate<String> SHARED_PREDICATE = getSharedClassPredicate(ANY);
@@ -197,7 +195,6 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
 
     protected class Wrapper extends DelegatingInvokableInstance implements IUpgradeableInstance
     {
-        private final int generation;
         private final IInstanceConfig config;
         private volatile IInvokableInstance delegate;
         private volatile Versions.Version version;
@@ -205,6 +202,7 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
         private volatile boolean isShutdown = true;
         @GuardedBy("this")
         private InetSocketAddress broadcastAddress;
+        private int generation = -1;
 
         protected IInvokableInstance delegate()
         {
@@ -216,22 +214,22 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
         protected IInvokableInstance delegateForStartup()
         {
             if (delegate == null)
-                delegate = newInstance(generation);
+                delegate = newInstance();
             return delegate;
         }
 
-        public Wrapper(int generation, Versions.Version version, IInstanceConfig config)
+        public Wrapper(Versions.Version version, IInstanceConfig config)
         {
-            this.generation = generation;
             this.config = config;
             this.version = version;
             // we ensure there is always a non-null delegate, so that the executor may be used while the node is offline
-            this.delegate = newInstance(generation);
+            this.delegate = newInstance();
             this.broadcastAddress = config.broadcastAddress();
         }
 
-        private IInvokableInstance newInstance(int generation)
+        private IInvokableInstance newInstance()
         {
+            ++generation;
             ClassLoader classLoader = new InstanceClassLoader(generation, config.num(), version.classpath, sharedClassLoader, sharedClassPredicate, classTransformer);
             ThreadGroup threadGroup = new ThreadGroup(clusterThreadGroup, "node" + config.num() + (generation > 1 ? "_" + generation : ""));
             if (instanceInitializer != null)
@@ -261,6 +259,9 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
 
         public Executor executorFor(int verb)
         {
+            if (isShutdown)
+                throw new IllegalStateException();
+
             // this method must be lock-free to avoid Simulator deadlock
             return delegate().executorFor(verb);
         }
@@ -461,13 +462,12 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
         this.instanceInitializer = builder.getInstanceInitializer2();
         this.datadirCount = builder.getDatadirCount();
 
-        int generation = GENERATION.incrementAndGet();
         for (int i = 0; i < builder.getNodeCount(); ++i)
         {
             int nodeNum = i + 1;
             InstanceConfig config = createInstanceConfig(nodeNum);
 
-            I instance = newInstanceWrapperInternal(generation, initialVersion, config);
+            I instance = newInstanceWrapperInternal(initialVersion, config);
             instances.add(instance);
             // we use the config().broadcastAddressAndPort() here because we have not initialised the Instance
             I prev = instanceMap.put(instance.config().broadcastAddress(), instance);
@@ -507,17 +507,17 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I
     }
 
 
-    protected abstract I newInstanceWrapper(int generation, Versions.Version version, IInstanceConfig config);
+    protected abstract I newInstanceWrapper(Versions.Version version, IInstanceConfig config);
 
-    protected I newInstanceWrapperInternal(int generation, Versions.Version version, IInstanceConfig config)
+    protected I newInstanceWrapperInternal(Versions.Version version, IInstanceConfig config)
     {
         config.validate();
-        return newInstanceWrapper(generation, version, config);
+        return newInstanceWrapper(version, config);
     }
 
     public I bootstrap(IInstanceConfig config)
     {
-        I instance = newInstanceWrapperInternal(0, initialVersion, config);
+        I instance = newInstanceWrapperInternal(initialVersion, config);
         instances.add(instance);
         I prev = instanceMap.put(config.broadcastAddress(), instance);
 
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
index 18c1573..6ea12fa 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
@@ -141,6 +141,7 @@ import org.apache.cassandra.utils.memory.BufferPools;
 
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
+import static org.apache.cassandra.distributed.api.Feature.BLANK_GOSSIP;
 import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
 import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
 import static org.apache.cassandra.distributed.api.Feature.NETWORK;
@@ -631,12 +632,12 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
                 }
                 else
                 {
-                    cluster.stream().forEach(peer -> {
-                        if (cluster instanceof Cluster)
-                            GossipHelper.statusToNormal((IInvokableInstance) peer).accept(this);
-                        else
-                            GossipHelper.unsafeStatusToNormal(this, (IInstance) peer);
-                    });
+                    if (config.has(BLANK_GOSSIP))
+                        cluster.stream().forEach(peer -> GossipHelper.statusToBlank((IInvokableInstance) peer).accept(this));
+                    else if (cluster instanceof Cluster)
+                        cluster.stream().forEach(peer -> GossipHelper.statusToNormal((IInvokableInstance) peer).accept(this));
+                    else
+                        cluster.stream().forEach(peer -> GossipHelper.unsafeStatusToNormal(this, (IInstance) peer));
 
                     StorageService.instance.setUpDistributedSystemKeyspaces();
                     StorageService.instance.setNormalModeUnsafe();
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
index c7ae775..7574318 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
@@ -73,7 +73,7 @@ public class InstanceConfig implements IInstanceConfig
     {
         this.num = num;
         this.networkTopology = networkTopology;
-        this.hostId = java.util.UUID.randomUUID();
+        this.hostId = new UUID(0x4000L, (1L << 63) | num); // deterministic hostId for simulator
         this    .set("num_tokens", 1)
                 .set("broadcast_address", broadcast_address)
                 .set("listen_address", listen_address)
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/InstanceKiller.java b/test/distributed/org/apache/cassandra/distributed/impl/InstanceKiller.java
index 99fc75d..e7ca49b 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/InstanceKiller.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/InstanceKiller.java
@@ -43,7 +43,6 @@ public class InstanceKiller extends JVMStabilityInspector.Killer
         // the bad part is that System.exit kills the JVM, so all code which calls kill won't hit the
         // next line; yet in in-JVM dtests System.exit is not desirable, so need to rely on a runtime exception
         // as a means to try to stop execution
-        // TODO (now): this is only used for one dtest, and can cause infinite loops with Simulator in e.g. AbstractCommitLogSegmentManager (failing its first assert, invoking the handler, throwing this exception, restarting)
         throw new InstanceShutdown();
     }
 
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java
index 38151be..c9c0087 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java
@@ -46,6 +46,7 @@ import org.slf4j.LoggerFactory;
 import ch.qos.logback.classic.LoggerContext;
 import io.netty.util.concurrent.FastThreadLocal;
 import org.apache.cassandra.concurrent.ExecutorFactory;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.distributed.api.IIsolatedExecutor;
 import org.apache.cassandra.utils.ExecutorUtils;
 import org.apache.cassandra.utils.Throwables;
@@ -59,7 +60,7 @@ public class IsolatedExecutor implements IIsolatedExecutor
     final ClassLoader classLoader;
     private final DynamicFunction<Serializable> transfer;
 
-    IsolatedExecutor(String name, ClassLoader classLoader, ExecutorFactory executorFactory)
+    public IsolatedExecutor(String name, ClassLoader classLoader, ExecutorFactory executorFactory)
     {
         this(name, classLoader, executorFactory.pooled("isolatedExecutor", Integer.MAX_VALUE));
     }
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/UnsafeGossipHelper.java b/test/distributed/org/apache/cassandra/distributed/impl/UnsafeGossipHelper.java
index fd166f5..9680b2d 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/UnsafeGossipHelper.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/UnsafeGossipHelper.java
@@ -86,16 +86,10 @@ public class UnsafeGossipHelper
                 Token token;
                 if (FBUtilities.getBroadcastAddressAndPort().equals(addressAndPort))
                 {
-                    if (tokenString == null)
-                    {
-                        token = Iterables.getOnlyElement(SystemKeyspace.getSavedTokens());
-                    }
-                    else
-                    {
-                        token = DatabaseDescriptor.getPartitioner().getTokenFactory().fromString(tokenString);
-                        SystemKeyspace.setLocalHostId(hostId);
-                        SystemKeyspace.updateTokens(singleton(token));
-                    }
+                    String str = tokenString == null ? Iterables.getOnlyElement(DatabaseDescriptor.getInitialTokens()) : tokenString;
+                    token = DatabaseDescriptor.getPartitioner().getTokenFactory().fromString(str);
+                    SystemKeyspace.setLocalHostId(hostId);
+                    SystemKeyspace.updateTokens(singleton(token));
                 }
                 else
                 {
diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/GroupByTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/GroupByTest.java
index 2e67497..634c886 100644
--- a/test/distributed/org/apache/cassandra/distributed/upgrade/GroupByTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/upgrade/GroupByTest.java
@@ -24,7 +24,6 @@ import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.shared.Versions;
 
 import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
-import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
 import static org.apache.cassandra.distributed.api.Feature.NETWORK;
 import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
 import static org.apache.cassandra.distributed.shared.AssertUtils.row;
diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java b/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java
index 6fb5805..768c6cc 100644
--- a/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java
+++ b/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java
@@ -37,7 +37,6 @@ import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.distributed.UpgradeableCluster;
 import org.apache.cassandra.distributed.api.ICluster;
 import org.apache.cassandra.distributed.api.IInstanceConfig;
-import org.apache.cassandra.distributed.impl.Instance;
 import org.apache.cassandra.distributed.shared.DistributedTestBase;
 import org.apache.cassandra.distributed.shared.ThrowingRunnable;
 import org.apache.cassandra.distributed.shared.Versions;
diff --git a/test/long/org/apache/cassandra/io/compress/CompressorPerformance.java b/test/long/org/apache/cassandra/io/compress/CompressorPerformance.java
index 9499401..9fbf24d 100644
--- a/test/long/org/apache/cassandra/io/compress/CompressorPerformance.java
+++ b/test/long/org/apache/cassandra/io/compress/CompressorPerformance.java
@@ -22,7 +22,6 @@ package org.apache.cassandra.io.compress;
 
 import org.apache.cassandra.io.util.FileInputStreamPlus;
 
-import java.io.FileInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collections;
diff --git a/test/microbench/org/apache/cassandra/test/microbench/BatchStatementBench.java b/test/microbench/org/apache/cassandra/test/microbench/BatchStatementBench.java
index 1fff65c..d487e2d 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/BatchStatementBench.java
+++ b/test/microbench/org/apache/cassandra/test/microbench/BatchStatementBench.java
@@ -41,7 +41,6 @@ import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.utils.FBUtilities;
 import org.openjdk.jmh.annotations.Benchmark;
 import org.openjdk.jmh.annotations.BenchmarkMode;
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/ChanceSupplier.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/ChanceSupplier.java
new file mode 100644
index 0000000..620f3e0
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/ChanceSupplier.java
@@ -0,0 +1,24 @@
+/*
+ * 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.simulator.asm;
+
+public interface ChanceSupplier
+{
+    float get();
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/ClassTransformer.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/ClassTransformer.java
new file mode 100644
index 0000000..0b9ee32
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/ClassTransformer.java
@@ -0,0 +1,210 @@
+/*
+ * 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.simulator.asm;
+
+import java.util.EnumSet;
+import java.util.List;
+
+import org.objectweb.asm.AnnotationVisitor;
+import org.objectweb.asm.ClassReader;
+import org.objectweb.asm.ClassVisitor;
+import org.objectweb.asm.ClassWriter;
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.Opcodes;
+import org.objectweb.asm.tree.AbstractInsnNode;
+import org.objectweb.asm.tree.MethodInsnNode;
+import org.objectweb.asm.tree.MethodNode;
+
+import static java.util.Collections.singletonList;
+import static org.apache.cassandra.simulator.asm.Flag.DETERMINISTIC;
+import static org.apache.cassandra.simulator.asm.Flag.GLOBAL_METHODS;
+import static org.apache.cassandra.simulator.asm.Flag.MONITORS;
+import static org.apache.cassandra.simulator.asm.Flag.NEMESIS;
+import static org.apache.cassandra.simulator.asm.Flag.NO_PROXY_METHODS;
+import static org.apache.cassandra.simulator.asm.TransformationKind.HASHCODE;
+import static org.apache.cassandra.simulator.asm.TransformationKind.SYNCHRONIZED;
+import static org.apache.cassandra.simulator.asm.Utils.deterministicToString;
+import static org.apache.cassandra.simulator.asm.Utils.generateTryFinallyProxyCall;
+import static org.objectweb.asm.Opcodes.ACC_PRIVATE;
+import static org.objectweb.asm.Opcodes.ACC_SYNTHETIC;
+import static org.objectweb.asm.Opcodes.INVOKESTATIC;
+
+class ClassTransformer extends ClassVisitor implements MethodWriterSink
+{
+    private static final List<AbstractInsnNode> DETERMINISM_SETUP = singletonList(new MethodInsnNode(INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptibleThread", "enterDeterministicMethod", "()V", false));
+    private static final List<AbstractInsnNode> DETERMINISM_CLEANUP = singletonList(new MethodInsnNode(INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptibleThread", "exitDeterministicMethod", "()V", false));
+
+    private final String className;
+    private final ChanceSupplier monitorDelayChance;
+    private final NemesisGenerator nemesis;
+    private final NemesisFieldKind.Selector nemesisFieldSelector;
+    private final Hashcode insertHashcode;
+    private final MethodLogger methodLogger;
+    private boolean isTransformed;
+    private boolean isCacheablyTransformed = true;
+    private final EnumSet<Flag> flags;
+
+    ClassTransformer(int api, String className, EnumSet<Flag> flags)
+    {
+        this(api, new ClassWriter(0), className, flags, null, null, null, null);
+    }
+
+    ClassTransformer(int api, String className, EnumSet<Flag> flags, ChanceSupplier monitorDelayChance, NemesisGenerator nemesis, NemesisFieldKind.Selector nemesisFieldSelector, Hashcode insertHashcode)
+    {
+        this(api, new ClassWriter(0), className, flags, monitorDelayChance, nemesis, nemesisFieldSelector, insertHashcode);
+    }
+
+    private ClassTransformer(int api, ClassWriter classWriter, String className, EnumSet<Flag> flags, ChanceSupplier monitorDelayChance, NemesisGenerator nemesis, NemesisFieldKind.Selector nemesisFieldSelector, Hashcode insertHashcode)
+    {
+        super(api, classWriter);
+        if (flags.contains(NEMESIS) && (nemesis == null || nemesisFieldSelector == null))
+            throw new IllegalArgumentException();
+        if (flags.contains(MONITORS) && monitorDelayChance == null)
+            throw new IllegalArgumentException();
+        this.className = className;
+        this.flags = flags;
+        this.monitorDelayChance = monitorDelayChance;
+        this.nemesis = nemesis;
+        this.nemesisFieldSelector = nemesisFieldSelector;
+        this.insertHashcode = insertHashcode;
+        this.methodLogger = MethodLogger.log(api, className);
+    }
+
+    @Override
+    public MethodVisitor visitMethod(int access, String name, String descriptor, String signature, String[] exceptions)
+    {
+        EnumSet<Flag> flags = this.flags;
+        if (flags.isEmpty() || ((access & ACC_SYNTHETIC) != 0 && (name.endsWith("$unsync") || name.endsWith("$catch") || name.endsWith("$nemesis"))))
+            return super.visitMethod(access, name, descriptor, signature, exceptions);
+
+        boolean isToString = false;
+        if (access == Opcodes.ACC_PUBLIC && name.equals("toString") && descriptor.equals("()Ljava/lang/String;") && !flags.contains(NO_PROXY_METHODS))
+        {
+            generateTryFinallyProxyCall(super.visitMethod(access, name, descriptor, signature, exceptions), className,
+                                        "toString$original", "()Ljava/lang/String;", access, true, false, DETERMINISM_SETUP, DETERMINISM_CLEANUP);
+            access = ACC_PRIVATE | ACC_SYNTHETIC;
+            name = "toString$original";
+            if (!flags.contains(DETERMINISTIC) || flags.contains(NEMESIS))
+            {
+                flags = EnumSet.copyOf(flags);
+                flags.add(DETERMINISTIC);
+                flags.remove(NEMESIS);
+            }
+            isToString = true;
+        }
+
+        MethodVisitor visitor;
+        if (flags.contains(MONITORS) && (access & Opcodes.ACC_SYNCHRONIZED) != 0)
+        {
+            visitor = new MonitorMethodTransformer(this, className, api, access, name, descriptor, signature, exceptions, monitorDelayChance);
+            witness(SYNCHRONIZED);
+        }
+        else
+        {
+            visitor = super.visitMethod(access, name, descriptor, signature, exceptions);
+            visitor = methodLogger.visitMethod(access, name, descriptor, visitor);
+        }
+
+        if (flags.contains(MONITORS))
+            visitor = new MonitorEnterExitParkTransformer(this, api, visitor, className, monitorDelayChance);
+        if (isToString)
+            visitor = deterministicToString(visitor);
+        if (flags.contains(GLOBAL_METHODS) || flags.contains(Flag.LOCK_SUPPORT) || flags.contains(Flag.DETERMINISTIC))
+            visitor = new GlobalMethodTransformer(flags, this, api, name, visitor);
+        if (flags.contains(NEMESIS))
+            visitor = new NemesisTransformer(this, api, name, visitor, nemesis, nemesisFieldSelector);
+        return visitor;
+    }
+
+    @Override
+    public void visitEnd()
+    {
+        if (insertHashcode != null)
+            writeSyntheticMethod(HASHCODE, insertHashcode);
+        super.visitEnd();
+        methodLogger.visitEndOfClass();
+    }
+
+    public void writeMethod(MethodNode node)
+    {
+        writeMethod(null, node);
+    }
+
+    public void writeSyntheticMethod(TransformationKind kind, MethodNode node)
+    {
+        writeMethod(kind, node);
+    }
+
+    void writeMethod(TransformationKind kind, MethodNode node)
+    {
+        String[] exceptions = node.exceptions == null ? null : node.exceptions.toArray(new String[0]);
+        MethodVisitor visitor = super.visitMethod(node.access, node.name, node.desc, node.signature, exceptions);
+        visitor = methodLogger.visitMethod(node.access, node.name, node.desc, visitor);
+        if (kind != null)
+            witness(kind);
+        node.accept(visitor);
+    }
+
+    @Override
+    public AnnotationVisitor visitAnnotation(String descriptor, boolean visible)
+    {
+        return Utils.checkForSimulationAnnotations(api, descriptor, super.visitAnnotation(descriptor, visible), (flag, add) -> {
+            if (add) flags.add(flag);
+            else flags.remove(flag);
+        });
+    }
+
+    void readAndTransform(byte[] input)
+    {
+        ClassReader reader = new ClassReader(input);
+        reader.accept(this, 0);
+    }
+
+    void witness(TransformationKind kind)
+    {
+        isTransformed = true;
+        switch (kind)
+        {
+            case FIELD_NEMESIS:
+            case SIGNAL_NEMESIS:
+                isCacheablyTransformed = false;
+        }
+        methodLogger.witness(kind);
+    }
+
+    String className()
+    {
+        return className;
+    }
+
+    boolean isTransformed()
+    {
+        return isTransformed;
+    }
+
+    boolean isCacheablyTransformed()
+    {
+        return isCacheablyTransformed;
+    }
+
+    byte[] toBytes()
+    {
+        return ((ClassWriter) cv).toByteArray();
+    }
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/Flag.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/Flag.java
new file mode 100644
index 0000000..00b88ab
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/Flag.java
@@ -0,0 +1,24 @@
+/*
+ * 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.simulator.asm;
+
+public enum Flag
+{
+    GLOBAL_CLOCK, MONITORS, LOCK_SUPPORT, GLOBAL_METHODS, DETERMINISTIC, NO_PROXY_METHODS, NEMESIS
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/GlobalMethodTransformer.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/GlobalMethodTransformer.java
new file mode 100644
index 0000000..868833a
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/GlobalMethodTransformer.java
@@ -0,0 +1,156 @@
+/*
+ * 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.simulator.asm;
+
+import java.util.EnumSet;
+
+import org.objectweb.asm.AnnotationVisitor;
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.Opcodes;
+
+import static org.apache.cassandra.simulator.asm.Flag.GLOBAL_METHODS;
+import static org.apache.cassandra.simulator.asm.TransformationKind.CONCURRENT_HASH_MAP;
+import static org.apache.cassandra.simulator.asm.TransformationKind.GLOBAL_METHOD;
+import static org.apache.cassandra.simulator.asm.TransformationKind.IDENTITY_HASH_MAP;
+
+/**
+ * Intercept factory methods in org.apache.concurrent.utils.concurrent, and redirect them to
+ * {@link org.apache.cassandra.simulator.systems.InterceptorOfGlobalMethods}
+ */
+class GlobalMethodTransformer extends MethodVisitor
+{
+    private final ClassTransformer transformer;
+    private final String methodName;
+    private boolean globalMethods;
+    private boolean globalClock;
+    private boolean lockSupport;
+    private boolean deterministic;
+    boolean hasSeenAnyMethodInsn;
+
+    public GlobalMethodTransformer(EnumSet<Flag> flags, ClassTransformer transformer, int api, String methodName, MethodVisitor parent)
+    {
+        super(api, parent);
+        this.globalMethods = flags.contains(GLOBAL_METHODS);
+        this.globalClock = flags.contains(Flag.GLOBAL_CLOCK);
+        this.lockSupport = flags.contains(Flag.LOCK_SUPPORT);
+        this.deterministic = flags.contains(Flag.DETERMINISTIC);
+        this.transformer = transformer;
+        this.methodName = methodName;
+    }
+
+    @Override
+    public void visitMethodInsn(int opcode, String owner, String name, String descriptor, boolean isInterface)
+    {
+        boolean isFirstMethodInsn = !hasSeenAnyMethodInsn;
+        hasSeenAnyMethodInsn = true;
+
+        if (globalMethods && opcode == Opcodes.INVOKESTATIC && owner.startsWith("org/apache/cassandra/utils/") && (
+               (owner.equals("org/apache/cassandra/utils/concurrent/WaitQueue") && name.equals("newWaitQueue"))
+            || (owner.equals("org/apache/cassandra/utils/concurrent/CountDownLatch") && name.equals("newCountDownLatch"))
+            || (owner.equals("org/apache/cassandra/utils/concurrent/Condition") && name.equals("newOneTimeCondition"))
+            || (owner.equals("org/apache/cassandra/utils/concurrent/BlockingQueues") && name.equals("newBlockingQueue"))
+            || (owner.equals("org/apache/cassandra/utils/concurrent/Semaphore") && (name.equals("newSemaphore") || name.equals("newFairSemaphore")))
+            ))
+        {
+            transformer.witness(GLOBAL_METHOD);
+            super.visitMethodInsn(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfGlobalMethods$Global", name, descriptor, false);
+        }
+        else if (globalMethods && ((opcode == Opcodes.INVOKESTATIC && (
+                   owner.startsWith("org/apache/cassandra/utils/") && (
+                        (owner.equals("org/apache/cassandra/utils/Clock") && name.equals("waitUntil"))
+                     || (owner.equals("org/apache/cassandra/utils/concurrent/Awaitable$SyncAwaitable") && name.equals("waitUntil")))
+                || !deterministic && owner.equals("java/lang/System") && name.equals("identityHashCode")
+                || owner.equals("java/util/UUID") && name.equals("randomUUID")
+                || owner.equals("com/google/common/util/concurrent/Uninterruptibles") && name.equals("sleepUninterruptibly")
+                || owner.equals("sun/misc/Unsafe") && name.equals("getUnsafe")))
+             || (owner.equals("java/util/concurrent/TimeUnit") && name.equals("sleep")))
+        )
+        {
+            transformer.witness(GLOBAL_METHOD);
+            super.visitMethodInsn(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", name, descriptor, false);
+        }
+        else if ((globalMethods || deterministic) && opcode == Opcodes.INVOKESTATIC &&
+            owner.equals("java/util/concurrent/ThreadLocalRandom") && (name.equals("getProbe") || name.equals("advanceProbe") || name.equals("localInit"))
+        )
+        {
+            transformer.witness(GLOBAL_METHOD);
+            // if we're in deterministic mode (i.e. for base ConcurrentHashMap) don't initialise ThreadLocalRandom
+            if (name.equals("getProbe")) super.visitLdcInsn(0);
+            else if (name.equals("advanceProbe")) super.visitMethodInsn(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", name, descriptor, false);
+        }
+        else if (globalMethods && opcode == Opcodes.INVOKESPECIAL && owner.equals("java/util/IdentityHashMap") && name.equals("<init>"))
+        {
+            transformer.witness(IDENTITY_HASH_MAP);
+            super.visitMethodInsn(opcode, "org/apache/cassandra/simulator/systems/InterceptedIdentityHashMap", name, descriptor, false);
+        }
+        else if (globalMethods && opcode == Opcodes.INVOKESPECIAL && owner.equals("java/util/concurrent/ConcurrentHashMap") && name.equals("<init>")
+                 && !(transformer.className().equals("org/apache/cassandra/simulator/systems/InterceptibleConcurrentHashMap") && methodName.equals("<init>") && isFirstMethodInsn))
+        {
+            transformer.witness(CONCURRENT_HASH_MAP);
+            super.visitMethodInsn(opcode, "org/apache/cassandra/simulator/systems/InterceptibleConcurrentHashMap", name, descriptor, false);
+        }
+        else if (lockSupport && opcode == Opcodes.INVOKESTATIC && owner.equals("java/util/concurrent/locks/LockSupport") && (name.startsWith("park") || name.equals("unpark")))
+        {
+            transformer.witness(TransformationKind.LOCK_SUPPORT);
+            super.visitMethodInsn(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", name, descriptor, false);
+        }
+        else if (globalClock && opcode == Opcodes.INVOKESTATIC && name.equals("timestampMicros") && owner.equals("org/apache/cassandra/utils/FBUtilities"))
+        {
+            transformer.witness(GLOBAL_METHOD);
+            super.visitMethodInsn(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/SimulatedTime$Global", "nextGlobalMonotonicMicros", descriptor, false);
+        }
+        else
+        {
+            super.visitMethodInsn(opcode, owner, name, descriptor, isInterface);
+        }
+    }
+
+    @Override
+    public void visitTypeInsn(int opcode, String type)
+    {
+        if (globalMethods && opcode == Opcodes.NEW && type.equals("java/util/IdentityHashMap"))
+        {
+            super.visitTypeInsn(opcode, "org/apache/cassandra/simulator/systems/InterceptedIdentityHashMap");
+        }
+        else if (globalMethods && opcode == Opcodes.NEW && type.equals("java/util/concurrent/ConcurrentHashMap"))
+        {
+            super.visitTypeInsn(opcode, "org/apache/cassandra/simulator/systems/InterceptibleConcurrentHashMap");
+        }
+        else
+        {
+            super.visitTypeInsn(opcode, type);
+        }
+    }
+
+    @Override
+    public AnnotationVisitor visitAnnotation(String descriptor, boolean visible)
+    {
+        return Utils.checkForSimulationAnnotations(api, descriptor, super.visitAnnotation(descriptor, visible), (flag, add) -> {
+            switch (flag)
+            {
+                default: throw new AssertionError();
+                case GLOBAL_METHODS: globalMethods = add; break;
+                case GLOBAL_CLOCK: globalClock = add; break;
+                case LOCK_SUPPORT: lockSupport = add; break;
+                case DETERMINISTIC: deterministic = add; break;
+                case MONITORS: throw new UnsupportedOperationException("Cannot currently toggle MONITORS at the method level");
+            }
+        });
+    }
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/Hashcode.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/Hashcode.java
new file mode 100644
index 0000000..0377854
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/Hashcode.java
@@ -0,0 +1,44 @@
+/*
+ * 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.simulator.asm;
+
+import org.objectweb.asm.Opcodes;
+import org.objectweb.asm.tree.InsnNode;
+import org.objectweb.asm.tree.IntInsnNode;
+import org.objectweb.asm.tree.LabelNode;
+import org.objectweb.asm.tree.MethodInsnNode;
+import org.objectweb.asm.tree.MethodNode;
+
+/**
+ * Generate a new hashCode method in the class that invokes a deterministic hashCode generator
+ */
+class Hashcode extends MethodNode
+{
+    Hashcode(int api)
+    {
+        super(api, Opcodes.ACC_PUBLIC, "hashCode", "()I", null, null);
+        maxLocals = 1;
+        maxStack = 1;
+        instructions.add(new LabelNode());
+        instructions.add(new IntInsnNode(Opcodes.ALOAD, 0));
+        instructions.add(new MethodInsnNode(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", "identityHashCode", "(Ljava/lang/Object;)I", false));
+        instructions.add(new LabelNode());
+        instructions.add(new InsnNode(Opcodes.IRETURN));
+    }
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptAgent.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptAgent.java
new file mode 100644
index 0000000..019ea3d
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptAgent.java
@@ -0,0 +1,334 @@
+/*
+ * 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.simulator.asm;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.instrument.ClassDefinition;
+import java.lang.instrument.ClassFileTransformer;
+import java.lang.instrument.IllegalClassFormatException;
+import java.lang.instrument.Instrumentation;
+import java.lang.instrument.UnmodifiableClassException;
+import java.security.ProtectionDomain;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.function.BiFunction;
+import java.util.regex.Pattern;
+
+import org.objectweb.asm.ClassReader;
+import org.objectweb.asm.ClassVisitor;
+import org.objectweb.asm.ClassWriter;
+import org.objectweb.asm.FieldVisitor;
+import org.objectweb.asm.Label;
+import org.objectweb.asm.MethodVisitor;
+
+import static org.apache.cassandra.simulator.asm.Flag.DETERMINISTIC;
+import static org.apache.cassandra.simulator.asm.Flag.LOCK_SUPPORT;
+import static org.apache.cassandra.simulator.asm.Flag.NO_PROXY_METHODS;
+import static org.apache.cassandra.simulator.asm.InterceptClasses.BYTECODE_VERSION;
+import static org.objectweb.asm.Opcodes.ALOAD;
+import static org.objectweb.asm.Opcodes.GETFIELD;
+import static org.objectweb.asm.Opcodes.GETSTATIC;
+import static org.objectweb.asm.Opcodes.INVOKESPECIAL;
+import static org.objectweb.asm.Opcodes.INVOKESTATIC;
+import static org.objectweb.asm.Opcodes.INVOKEVIRTUAL;
+import static org.objectweb.asm.Opcodes.IRETURN;
+import static org.objectweb.asm.Opcodes.RETURN;
+
+/**
+ * A mechanism for weaving classes loaded by the bootstrap classloader that we cannot override.
+ * The design supports weaving of the internals of these classes, and in future we may want to
+ * weave LockSupport or the internals of other blocking concurrency primitives.
+ *
+ * Ultimately this wasn't necessary for the initial functionality, but we have maintained
+ * the layout so that it will be easier to enable such functionality in future should it be needed.
+ *
+ * To this end, the asm package and simulator-asm.jar is as self-contained set of classes for performing
+ * simulator byteweaving, and simulator-bootstrap.jar contains a self-contained class and interface for
+ * replacing important system methods.
+ */
+public class InterceptAgent
+{
+    public static void premain(final String agentArgs, final Instrumentation instrumentation) throws UnmodifiableClassException, ClassNotFoundException, IOException
+    {
+        setup(agentArgs, instrumentation);
+    }
+
+    public void agentmain(final String agentArgs, final Instrumentation instrumentation) throws UnmodifiableClassException, ClassNotFoundException, IOException
+    {
+        setup(agentArgs, instrumentation);
+    }
+
+    private static void setup(final String agentArgs, final Instrumentation instrumentation) throws UnmodifiableClassException, ClassNotFoundException, IOException
+    {
+        instrumentation.addTransformer(new ClassFileTransformer()
+        {
+            @Override
+            public byte[] transform(ClassLoader loader, String className, Class<?> classBeingRedefined, ProtectionDomain protectionDomain, byte[] bytecode) throws IllegalClassFormatException
+            {
+                if (className == null)
+                    return null;
+
+                if (className.equals("java/lang/Object"))
+                    return transformObject(bytecode);
+
+                if (className.equals("java/lang/Enum"))
+                    return transformEnum(bytecode);
+
+                if (className.equals("java/util/Random"))
+                    return transformRandom(bytecode);
+
+                if (className.equals("java/util/concurrent/ThreadLocalRandom"))
+                    return transformThreadLocalRandom(bytecode);
+
+                if (className.startsWith("java/util/concurrent/ConcurrentHashMap"))
+                    return transformConcurrent(className, bytecode, DETERMINISTIC, NO_PROXY_METHODS);
+
+                if (className.startsWith("java/util/concurrent/locks"))
+                    return transformConcurrent(className, bytecode, LOCK_SUPPORT, NO_PROXY_METHODS);
+
+                return null;
+            }
+        });
+
+        Pattern reloadPattern = Pattern.compile("java\\.(lang\\.Enum|util\\.concurrent\\.(locks\\..*|ConcurrentHashMap)|util\\.(concurrent\\.ThreadLocal)?Random|lang\\.Object)");
+        List<ClassDefinition> redefine = new ArrayList<>();
+        for (Class<?> loadedClass : instrumentation.getAllLoadedClasses())
+        {
+            if (reloadPattern.matcher(loadedClass.getName()).matches())
+                redefine.add(new ClassDefinition(loadedClass, readDefinition(loadedClass)));
+        }
+        if (!redefine.isEmpty())
+            instrumentation.redefineClasses(redefine.toArray(new ClassDefinition[0]));
+    }
+
+    private static byte[] readDefinition(Class<?> clazz) throws IOException
+    {
+        return readDefinition(clazz.getName().replaceAll("\\.", "/"));
+    }
+
+    private static byte[] readDefinition(String className) throws IOException
+    {
+        byte[] bytes = new byte[1024];
+        try (InputStream in = ClassLoader.getSystemResourceAsStream(className + ".class"))
+        {
+            int count = 0;
+            while (true)
+            {
+                int add = in.read(bytes, count, bytes.length - count);
+                if (add < 0)
+                    break;
+                if (add == 0)
+                    bytes = Arrays.copyOf(bytes, bytes.length * 2);
+                count += add;
+            }
+            return Arrays.copyOf(bytes, count);
+        }
+    }
+
+    /**
+     * We don't want Object.toString() to invoke our overridden identityHashCode by virtue of invoking some overridden hashCode()
+     * So we overwrite Object.toString() to replace calls to Object.hashCode() with direct calls to System.identityHashCode()
+     */
+    private static byte[] transformObject(byte[] bytes)
+    {
+        class ObjectVisitor extends ClassVisitor
+        {
+            public ObjectVisitor(int api, ClassVisitor classVisitor)
+            {
+                super(api, classVisitor);
+            }
+
+            @Override
+            public MethodVisitor visitMethod(int access, String name, String descriptor, String signature, String[] exceptions)
+            {
+                if (descriptor.equals("()Ljava/lang/String;") && name.equals("toString"))
+                    return Utils.deterministicToString(super.visitMethod(access, name, descriptor, signature, exceptions));
+                else
+                    return super.visitMethod(access, name, descriptor, signature, exceptions);
+            }
+        }
+        return transform(bytes, ObjectVisitor::new);
+    }
+
+    /**
+     * We want Enum to have a deterministic hashCode() so we simply forward calls to ordinal()
+     */
+    private static byte[] transformEnum(byte[] bytes)
+    {
+        class EnumVisitor extends ClassVisitor
+        {
+            public EnumVisitor(int api, ClassVisitor classVisitor)
+            {
+                super(api, classVisitor);
+            }
+
+            @Override
+            public MethodVisitor visitMethod(int access, String name, String descriptor, String signature, String[] exceptions)
+            {
+                if (descriptor.equals("()I") && name.equals("hashCode"))
+                {
+                    MethodVisitor visitor = super.visitMethod(access, name, descriptor, signature, exceptions);
+                    visitor.visitLabel(new Label());
+                    visitor.visitIntInsn(ALOAD, 0);
+                    visitor.visitFieldInsn(GETFIELD, "java/lang/Enum", "ordinal", "I");
+                    visitor.visitInsn(IRETURN);
+                    visitor.visitLabel(new Label());
+                    visitor.visitMaxs(1, 1);
+                    visitor.visitEnd();
+
+                    return new MethodVisitor(BYTECODE_VERSION) {};
+                }
+                else
+                {
+                    return super.visitMethod(access, name, descriptor, signature, exceptions);
+                }
+            }
+        }
+        return transform(bytes, EnumVisitor::new);
+    }
+
+    /**
+     * We want Random to be initialised deterministically, so we modify the default constructor to fetch
+     * some deterministically generated seed to pass to its seed constructor
+     */
+    private static byte[] transformRandom(byte[] bytes)
+    {
+        class RandomVisitor extends ClassVisitor
+        {
+            public RandomVisitor(int api, ClassVisitor classVisitor)
+            {
+                super(api, classVisitor);
+            }
+
+            @Override
+            public MethodVisitor visitMethod(int access, String name, String descriptor, String signature, String[] exceptions)
+            {
+                if (descriptor.equals("()V") && name.equals("<init>"))
+                {
+                    MethodVisitor visitor = super.visitMethod(access, name, descriptor, signature, exceptions);
+                    visitor.visitLabel(new Label());
+                    visitor.visitIntInsn(ALOAD, 0);
+                    visitor.visitMethodInsn(INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", "randomSeed", "()J", false);
+                    visitor.visitMethodInsn(INVOKESPECIAL, "java/util/Random", "<init>", "(J)V", false);
+                    visitor.visitInsn(RETURN);
+                    visitor.visitLabel(new Label());
+                    visitor.visitMaxs(3, 1);
+                    visitor.visitEnd();
+
+                    return new MethodVisitor(BYTECODE_VERSION) {};
+                }
+                else
+                {
+                    return super.visitMethod(access, name, descriptor, signature, exceptions);
+                }
+            }
+        }
+        return transform(bytes, RandomVisitor::new);
+    }
+
+    /**
+     * We require ThreadLocalRandom to be deterministic, so we modify its initialisation method to invoke a
+     * global deterministic random value generator
+     */
+    private static byte[] transformThreadLocalRandom(byte[] bytes)
+    {
+        class ThreadLocalRandomVisitor extends ClassVisitor
+        {
+            public ThreadLocalRandomVisitor(int api, ClassVisitor classVisitor)
+            {
+                super(api, classVisitor);
+            }
+
+            String unsafeDescriptor;
+            String unsafeFieldName;
+
+            @Override
+            public FieldVisitor visitField(int access, String name, String descriptor, String signature, Object value)
+            {
+                if (descriptor.equals("Lsun/misc/Unsafe;") || descriptor.equals("Ljdk/internal/misc/Unsafe;"))
+                {
+                    unsafeFieldName = name;
+                    unsafeDescriptor = descriptor;
+                }
+                return super.visitField(access, name, descriptor, signature, value);
+            }
+
+            @Override
+            public MethodVisitor visitMethod(int access, String name, String descriptor, String signature, String[] exceptions)
+            {
+                if (descriptor.equals("()V") && name.equals("localInit"))
+                {
+                    if (unsafeFieldName == null)
+                    {
+                        String version = System.getProperty("java.version");
+                        if (version.startsWith("11.")) { unsafeFieldName = "U"; unsafeDescriptor = "Ljdk/internal/misc/Unsafe;"; }
+                        else if (version.startsWith("1.8")) { unsafeFieldName = "UNSAFE"; unsafeDescriptor = "Lsun/misc/Unsafe;"; }
+                        else throw new AssertionError("Unsupported Java Version");
+                    }
+
+                    MethodVisitor visitor = super.visitMethod(access, name, descriptor, signature, exceptions);
+                    visitor.visitLabel(new Label());
+                    visitor.visitIntInsn(ALOAD, 0);
+                    visitor.visitFieldInsn(GETSTATIC, "java/util/concurrent/ThreadLocalRandom", unsafeFieldName, unsafeDescriptor);
+                    visitor.visitMethodInsn(INVOKESTATIC, "java/lang/Thread", "currentThread", "()Ljava/lang/Thread;", false);
+                    visitor.visitFieldInsn(GETSTATIC, "java/util/concurrent/ThreadLocalRandom", "SEED", "J");
+                    visitor.visitMethodInsn(INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", "randomSeed", "()J", false);
+                    visitor.visitMethodInsn(INVOKEVIRTUAL, "sun/misc/Unsafe", "putLong", "(Ljava/lang/Object;JJ)V", false);
+                    visitor.visitFieldInsn(GETSTATIC, "java/util/concurrent/ThreadLocalRandom", unsafeFieldName, unsafeDescriptor);
+                    visitor.visitMethodInsn(INVOKESTATIC, "java/lang/Thread", "currentThread", "()Ljava/lang/Thread;", false);
+                    visitor.visitFieldInsn(GETSTATIC, "java/util/concurrent/ThreadLocalRandom", "PROBE", "J");
+                    visitor.visitLdcInsn(0);
+                    visitor.visitMethodInsn(INVOKEVIRTUAL, "sun/misc/Unsafe", "putInt", "(Ljava/lang/Object;JI)V", false);
+                    visitor.visitInsn(RETURN);
+                    visitor.visitLabel(new Label());
+                    visitor.visitMaxs(6, 1);
+                    visitor.visitEnd();
+
+                    return new MethodVisitor(BYTECODE_VERSION) {};
+                }
+                else
+                {
+                    return super.visitMethod(access, name, descriptor, signature, exceptions);
+                }
+            }
+        }
+        return transform(bytes, ThreadLocalRandomVisitor::new);
+    }
+
+    private static byte[] transform(byte[] bytes, BiFunction<Integer, ClassWriter, ClassVisitor> constructor)
+    {
+        ClassWriter out = new ClassWriter(0);
+        ClassReader in = new ClassReader(bytes);
+        ClassVisitor transform = constructor.apply(BYTECODE_VERSION, out);
+        in.accept(transform, 0);
+        return out.toByteArray();
+    }
+
+    private static byte[] transformConcurrent(String className, byte[] bytes, Flag flag, Flag ... flags)
+    {
+        ClassTransformer transformer = new ClassTransformer(BYTECODE_VERSION, className, EnumSet.of(flag, flags));
+        transformer.readAndTransform(bytes);
+        if (!transformer.isTransformed())
+            return null;
+        return transformer.toBytes();
+    }
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptClasses.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptClasses.java
new file mode 100644
index 0000000..f57fb77
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptClasses.java
@@ -0,0 +1,251 @@
+/*
+ * 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.simulator.asm;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.io.UncheckedIOException;
+import java.lang.reflect.Method;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import java.util.regex.Pattern;
+
+import org.objectweb.asm.Opcodes;
+
+// TODO (completeness): confirm that those classes we weave monitor-access for only extend other classes we also weave monitor access for
+// TODO (completeness): confirm that those classes we weave monitor access for only take monitors on types we also weave monitor access for (and vice versa)
+public class InterceptClasses implements BiFunction<String, byte[], byte[]>
+{
+    public static final int BYTECODE_VERSION = Opcodes.ASM7;
+
+    // TODO (cleanup): use annotations
+    private static final Pattern MONITORS = Pattern.compile( "org[/.]apache[/.]cassandra[/.]utils[/.]concurrent[/.].*" +
+                                                            "|org[/.]apache[/.]cassandra[/.]concurrent[/.].*" +
+                                                            "|org[/.]apache[/.]cassandra[/.]simulator[/.]test.*" +
+                                                            "|org[/.]apache[/.]cassandra[/.]db[/.]ColumnFamilyStore.*" +
+                                                            "|org[/.]apache[/.]cassandra[/.]db[/.]Keyspace.*" +
+                                                            "|org[/.]apache[/.]cassandra[/.]db[/.]SystemKeyspace.*" +
+                                                            "|org[/.]apache[/.]cassandra[/.]streaming[/.].*" +
+                                                            "|org[/.]apache[/.]cassandra[/.]db.streaming[/.].*" +
+                                                            "|org[/.]apache[/.]cassandra[/.]distributed[/.]impl[/.]DirectStreamingConnectionFactory.*" +
+                                                            "|org[/.]apache[/.]cassandra[/.]db[/.]commitlog[/.].*" +
+                                                            "|org[/.]apache[/.]cassandra[/.]service[/.]paxos[/.].*");
+
+    private static final Pattern GLOBAL_METHODS = Pattern.compile("org[/.]apache[/.]cassandra[/.](?!simulator[/.]).*" +
+                                                                  "|org[/.]apache[/.]cassandra[/.]simulator[/.]test[/.].*" +
+                                                                  "|org[/.]apache[/.]cassandra[/.]simulator[/.]cluster[/.].*" +
+                                                                  "|io[/.]netty[/.]util[/.]concurrent[/.]FastThreadLocal"); // intercept IdentityHashMap for execution consistency
+    private static final Pattern NEMESIS = GLOBAL_METHODS;
+    private static final Set<String> WARNED = Collections.newSetFromMap(new ConcurrentHashMap<>());
+
+    static final Cached SENTINEL = new Cached(null);
+    static class Cached
+    {
+        final byte[] cached;
+        private Cached(byte[] cached)
+        {
+            this.cached = cached;
+        }
+    }
+
+    private final Map<String, Cached> cache = new ConcurrentHashMap<>();
+
+    private final int api;
+    private final ChanceSupplier nemesisChance;
+    private final ChanceSupplier monitorDelayChance;
+    private final Hashcode insertHashcode;
+    private final NemesisFieldKind.Selector nemesisFieldSelector;
+
+    public InterceptClasses(ChanceSupplier monitorDelayChance, ChanceSupplier nemesisChance, NemesisFieldKind.Selector nemesisFieldSelector)
+    {
+        this(BYTECODE_VERSION, monitorDelayChance, nemesisChance, nemesisFieldSelector);
+    }
+
+    public InterceptClasses(int api, ChanceSupplier monitorDelayChance, ChanceSupplier nemesisChance, NemesisFieldKind.Selector nemesisFieldSelector)
+    {
+        this.api = api;
+        this.nemesisChance = nemesisChance;
+        this.monitorDelayChance = monitorDelayChance;
+        this.insertHashcode = new Hashcode(api);
+        this.nemesisFieldSelector = nemesisFieldSelector;
+    }
+
+    @Override
+    public synchronized byte[] apply(String name, byte[] bytes)
+    {
+        if (bytes == null)
+            return maybeSynthetic(name);
+
+        Hashcode hashcode = insertHashCode(name);
+
+        name = dotsToSlashes(name);
+        EnumSet<Flag> flags = EnumSet.noneOf(Flag.class);
+        if (MONITORS.matcher(name).matches())
+        {
+            flags.add(Flag.MONITORS);
+        }
+        if (GLOBAL_METHODS.matcher(name).matches())
+        {
+            flags.add(Flag.GLOBAL_METHODS);
+            flags.add(Flag.LOCK_SUPPORT);
+        }
+        if (NEMESIS.matcher(name).matches())
+        {
+            flags.add(Flag.NEMESIS);
+        }
+
+        if (flags.isEmpty() && hashcode == null)
+            return bytes;
+
+        Cached prev = cache.get(name);
+        if (prev != null)
+        {
+            if (prev == SENTINEL)
+                return bytes;
+            return prev.cached;
+        }
+
+        ClassTransformer transformer = new ClassTransformer(api, name, flags, monitorDelayChance, new NemesisGenerator(api, name, nemesisChance), nemesisFieldSelector, hashcode);
+        transformer.readAndTransform(bytes);
+
+        if (!transformer.isTransformed())
+        {
+            cache.put(name, SENTINEL);
+            return bytes;
+        }
+
+        bytes = transformer.toBytes();
+        if (transformer.isCacheablyTransformed())
+            cache.put(name, new Cached(bytes));
+
+        return bytes;
+    }
+
+    static String dotsToSlashes(String className)
+    {
+        return className.replace('.', '/');
+    }
+
+    static String dotsToSlashes(Class<?> clazz)
+    {
+        return dotsToSlashes(clazz.getName());
+    }
+
+    /**
+     * Decide if we should insert our own hashCode() implementation that assigns deterministic hashes, i.e.
+     *   - If it's one of our classes
+     *   - If its parent is not one of our classes (else we'll assign it one anyway)
+     *   - If it does not have its own hashCode() implementation that overrides Object's
+     *   - If it is not Serializable OR it has a serialVersionUID
+     *
+     * Otherwise we either probably do not need it, or may break serialization between classloaders
+     */
+    private Hashcode insertHashCode(String name)
+    {
+        try
+        {
+            if (!name.startsWith("org.apache.cassandra"))
+                return null;
+
+            Class<?> sharedClass = getClass().getClassLoader().loadClass(name);
+            if (sharedClass.isInterface() || sharedClass.isEnum() || sharedClass.isArray() || sharedClass.isSynthetic())
+                return null;
+
+            Class<?> parent = sharedClass.getSuperclass();
+            if (parent.getName().startsWith("org.apache.cassandra"))
+                return null;
+
+            try
+            {
+                Method method = sharedClass.getMethod("hashCode");
+                if (method.getDeclaringClass() != Object.class)
+                    return null;
+            }
+            catch (NoSuchMethodException ignore)
+            {
+            }
+
+            if (!Serializable.class.isAssignableFrom(sharedClass))
+                return insertHashcode;
+
+            try
+            {
+                // if we haven't specified serialVersionUID we break ObjectInputStream transfers between class loaders
+                // (might be easiest to switch to serialization that doesn't require it)
+                sharedClass.getDeclaredField("serialVersionUID");
+                return insertHashcode;
+            }
+            catch (NoSuchFieldException e)
+            {
+                if (!Throwable.class.isAssignableFrom(sharedClass) && WARNED.add(name))
+                    System.err.println("No serialVersionUID on Serializable " + sharedClass);
+                return null;
+            }
+        }
+        catch (ClassNotFoundException e)
+        {
+            System.err.println("Unable to determine if should insert hashCode() for " + name);
+            e.printStackTrace();
+        }
+        return null;
+    }
+
+    static final String shadowRootExternalType = "org.apache.cassandra.simulator.systems.InterceptibleConcurrentHashMap";
+    static final String shadowRootType = "org/apache/cassandra/simulator/systems/InterceptibleConcurrentHashMap";
+    static final String originalRootType = Utils.toInternalName(ConcurrentHashMap.class);
+    static final String shadowOuterTypePrefix = shadowRootType + '$';
+    static final String originalOuterTypePrefix = originalRootType + '$';
+
+    protected byte[] maybeSynthetic(String name)
+    {
+        if (!name.startsWith(shadowRootExternalType))
+            return null;
+
+        try
+        {
+            String originalType, shadowType = Utils.toInternalName(name);
+            if (!shadowType.startsWith(shadowOuterTypePrefix))
+                originalType = originalRootType;
+            else
+                originalType = originalOuterTypePrefix + name.substring(shadowOuterTypePrefix.length());
+
+            EnumSet<Flag> flags = EnumSet.of(Flag.GLOBAL_METHODS, Flag.MONITORS, Flag.LOCK_SUPPORT);
+            if (NEMESIS.matcher(name).matches()) flags.add(Flag.NEMESIS);
+            NemesisGenerator nemesis = new NemesisGenerator(api, name, nemesisChance);
+
+            ShadowingTransformer transformer;
+            transformer = new ShadowingTransformer(InterceptClasses.BYTECODE_VERSION,
+                                                   originalType, shadowType, originalRootType, shadowRootType,
+                                                   originalOuterTypePrefix, shadowOuterTypePrefix,
+                                                   flags, monitorDelayChance, nemesis, nemesisFieldSelector, null);
+            transformer.readAndTransform(Utils.readDefinition(originalType + ".class"));
+            return transformer.toBytes();
+        }
+        catch (IOException e)
+        {
+            throw new UncheckedIOException(e);
+        }
+
+    }
+
+}
\ No newline at end of file
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/MethodLogger.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/MethodLogger.java
new file mode 100644
index 0000000..99252d3
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/MethodLogger.java
@@ -0,0 +1,259 @@
+/*
+ * 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.simulator.asm;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.util.Printer;
+import org.objectweb.asm.util.Textifier;
+import org.objectweb.asm.util.TraceMethodVisitor;
+
+import static java.util.Arrays.stream;
+import static org.apache.cassandra.simulator.asm.MethodLogger.Level.NONE;
+import static org.apache.cassandra.simulator.asm.MethodLogger.Level.valueOf;
+
+// TODO (config): support logging only for packages/classes matching a pattern
+interface MethodLogger
+{
+    static final Level LOG = valueOf(System.getProperty("cassandra.simulator.print_asm", "none").toUpperCase());
+    static final Set<TransformationKind> KINDS = System.getProperty("cassandra.simulator.print_asm_opts", "").isEmpty()
+                                                 ? EnumSet.allOf(TransformationKind.class)
+                                                 : stream(System.getProperty("cassandra.simulator.print_asm_opts", "").split(","))
+                                                   .map(TransformationKind::valueOf)
+                                                   .collect(() -> EnumSet.noneOf(TransformationKind.class), Collection::add, Collection::addAll);
+    static final Pattern LOG_CLASSES = System.getProperty("cassandra.simulator.print_asm_classes", "").isEmpty()
+                                                 ? null
+                                                 : Pattern.compile(System.getProperty("cassandra.simulator.print_asm_classes", ""));
+
+    // debug the output of each class at most once
+    static final Set<String> LOGGED_CLASS = LOG != NONE ? Collections.newSetFromMap(new ConcurrentHashMap<>()) : null;
+
+    enum Level { NONE, CLASS_SUMMARY, CLASS_DETAIL, METHOD_SUMMARY, METHOD_DETAIL, ASM }
+
+    MethodVisitor visitMethod(int access, String name, String descriptor, MethodVisitor parent);
+    void witness(TransformationKind kind);
+    void visitEndOfClass();
+
+    static MethodLogger log(int api, String className)
+    {
+        switch (LOG)
+        {
+            default:
+            case NONE:
+                return None.INSTANCE;
+            case ASM:
+                return (LOG_CLASSES == null || LOG_CLASSES.matcher(className).matches()) && LOGGED_CLASS.add(className)
+                       ? new Printing(api, className) : None.INSTANCE;
+            case CLASS_DETAIL:
+            case CLASS_SUMMARY:
+            case METHOD_DETAIL:
+            case METHOD_SUMMARY:
+                return (LOG_CLASSES == null || LOG_CLASSES.matcher(className).matches()) && LOGGED_CLASS.add(className)
+                       ? new Counting(api, className, LOG) : None.INSTANCE;
+        }
+    }
+
+    static class None implements MethodLogger
+    {
+        static final None INSTANCE = new None();
+
+        @Override
+        public MethodVisitor visitMethod(int access, String name, String descriptor, MethodVisitor parent)
+        {
+            return parent;
+        }
+
+        @Override
+        public void witness(TransformationKind kind)
+        {
+        }
+
+        @Override
+        public void visitEndOfClass()
+        {
+        }
+    }
+
+    static class Counting implements MethodLogger
+    {
+        final int api;
+        final String className;
+        final Level level;
+        StringWriter buffer = new StringWriter();
+        PrintWriter out = new PrintWriter(buffer);
+
+        boolean isMethodInProgress;
+        boolean printMethod;
+        boolean printClass;
+
+        int methodCount;
+        final int[] methodCounts = new int[TransformationKind.VALUES.size()];
+        final int[] classCounts = new int[TransformationKind.VALUES.size()];
+
+        public Counting(int api, String className, Level level)
+        {
+            this.api = api;
+            this.className = className;
+            this.level = level;
+        }
+
+        @Override
+        public MethodVisitor visitMethod(int access, String name, String descriptor, MethodVisitor parent)
+        {
+            ++methodCount;
+            if (isMethodInProgress)
+                return parent;
+
+            return new MethodVisitor(api, parent) {
+                @Override
+                public void visitEnd()
+                {
+                    super.visitEnd();
+                    if (printMethod)
+                    {
+                        for (int i = 0 ; i < methodCounts.length ; ++i)
+                            classCounts[i] += methodCounts[i];
+
+                        switch (level)
+                        {
+                            case METHOD_DETAIL:
+                                out.printf("Transformed %s.%s %s\n", className, name, descriptor);
+                                for (int i = 0 ; i < methodCounts.length ; ++i)
+                                {
+                                    if (methodCounts[i] > 0)
+                                        out.printf("    %3d %s\n", methodCounts[i], TransformationKind.VALUES.get(i));
+                                }
+                                break;
+
+                            case METHOD_SUMMARY:
+                                out.printf("Transformed %s.%s %s with %d modifications\n", className, name, descriptor, stream(methodCounts).sum());
+                                break;
+                        }
+                        printMethod = false;
+                        Arrays.fill(methodCounts, 0);
+                    }
+                    isMethodInProgress = false;
+                }
+            };
+        }
+
+        public void visitEndOfClass()
+        {
+            if (!printClass)
+                return;
+
+            switch (level)
+            {
+                case CLASS_DETAIL:
+                    out.printf("Transformed %s: %d methods\n", className, methodCount);
+                    for (int i = 0 ; i < classCounts.length ; ++i)
+                    {
+                        if (classCounts[i] > 0)
+                            out.printf("    %3d %s\n", classCounts[i], TransformationKind.VALUES.get(i));
+                    }
+                case CLASS_SUMMARY:
+                    out.printf("Transformed %s: %d methods with %d modifications\n", className, methodCount, stream(classCounts).sum());
+            }
+            System.out.print(buffer.toString());
+            buffer = null;
+            out = null;
+        }
+
+        @Override
+        public void witness(TransformationKind kind)
+        {
+            ++methodCounts[kind.ordinal()];
+            if (KINDS.contains(kind))
+            {
+                printMethod = true;
+                printClass = true;
+            }
+        }
+    }
+
+    static class Printing implements MethodLogger
+    {
+        final int api;
+        final String className;
+        final Textifier textifier = new Textifier();
+        StringWriter buffer = new StringWriter();
+        PrintWriter out = new PrintWriter(buffer);
+
+        boolean printClass;
+        boolean printMethod;
+        boolean isMethodInProgress;
+
+        public Printing(int api, String className)
+        {
+            this.api = api;
+            this.className = className;
+        }
+
+        @Override
+        public MethodVisitor visitMethod(int access, String name, String descriptor, MethodVisitor parent)
+        {
+            Printer printer = textifier.visitMethod(access, name, descriptor, null, null);
+            boolean isOuter = !isMethodInProgress;
+            if (isOuter) isMethodInProgress = true;
+            return new TraceMethodVisitor(new MethodVisitor(api, parent) {
+                @Override
+                public void visitEnd()
+                {
+                    super.visitEnd();
+                    if (printMethod)
+                    {
+                        out.println("====" + className + '.' + name + ' ' + descriptor + ' ');
+                        printer.print(out);
+                    }
+                    if (isOuter) isMethodInProgress = false;
+                }
+            }, printer);
+        }
+
+        @Override
+        public void witness(TransformationKind kind)
+        {
+            if (KINDS.contains(kind))
+            {
+                printMethod = true;
+                printClass = true;
+            }
+        }
+
+        @Override
+        public void visitEndOfClass()
+        {
+            if (printClass)
+                System.out.println(buffer.toString());
+            buffer = null;
+            out = null;
+        }
+    }
+
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/MethodWriterSink.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/MethodWriterSink.java
new file mode 100644
index 0000000..a4b8a8d
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/MethodWriterSink.java
@@ -0,0 +1,29 @@
+/*
+ * 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.simulator.asm;
+
+import org.objectweb.asm.tree.MethodNode;
+
+public interface MethodWriterSink
+{
+
+    void writeMethod(MethodNode method);
+    void writeSyntheticMethod(TransformationKind kind, MethodNode method);
+
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/MonitorEnterExitParkTransformer.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/MonitorEnterExitParkTransformer.java
new file mode 100644
index 0000000..f92013e
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/MonitorEnterExitParkTransformer.java
@@ -0,0 +1,123 @@
+/*
+ * 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.simulator.asm;
+
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.Opcodes;
+
+import static org.apache.cassandra.simulator.asm.TransformationKind.LOCK_SUPPORT;
+import static org.apache.cassandra.simulator.asm.TransformationKind.MONITOR;
+
+/**
+ * Handle simply thread signalling behaviours, namely monitorenter/monitorexit bytecodes to
+ * {@link org.apache.cassandra.simulator.systems.InterceptorOfGlobalMethods}, and LockSupport invocations to
+ * {@link org.apache.cassandra.simulator.systems.InterceptibleThread}.
+ *
+ * The global static methods we redirect monitors to take only one parameter (the monitor) and also return it,
+ * so that they have net zero effect on the stack, permitting the existing monitorenter/monitorexit instructions
+ * to remain where they are. LockSupport on the other hand is redirected entirely to the new method.
+ */
+class MonitorEnterExitParkTransformer extends MethodVisitor
+{
+    private final ClassTransformer transformer;
+    private final String className;
+    private final ChanceSupplier monitorDelayChance;
+
+    public MonitorEnterExitParkTransformer(ClassTransformer transformer,
+                                           int api,
+                                           MethodVisitor parent,
+                                           String className,
+                                           ChanceSupplier monitorDelayChance)
+    {
+        super(api, parent);
+        this.transformer = transformer;
+        this.className = className;
+        this.monitorDelayChance = monitorDelayChance;
+    }
+
+    @Override
+    public void visitMethodInsn(int opcode, String owner, String name, String descriptor, boolean isInterface)
+    {
+        if (opcode == Opcodes.INVOKEVIRTUAL && !isInterface && owner.equals("java/lang/Object"))
+        {
+            switch (name.charAt(0))
+            {
+                case 'w':
+                    assert name.equals("wait");
+                    switch (descriptor.charAt(2))
+                    {
+                        default:
+                            throw new AssertionError("Unexpected descriptor for method wait() in " + className + '.' + name);
+                        case 'V': // ()V
+                            transformer.witness(MONITOR);
+                            super.visitMethodInsn(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", "wait", "(Ljava/lang/Object;)V", false);
+                            return;
+                        case ')': // (J)V
+                            transformer.witness(MONITOR);
+                            super.visitMethodInsn(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", "wait", "(Ljava/lang/Object;J)V", false);
+                            return;
+                        case 'I': // (JI)V
+                            transformer.witness(MONITOR);
+                            super.visitMethodInsn(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", "wait", "(Ljava/lang/Object;JI)V", false);
+                            return;
+                    }
+                case 'n':
+                    switch (name.length())
+                    {
+                        default:
+                            throw new AssertionError();
+                        case 6: // notify
+                            transformer.witness(MONITOR);
+                            super.visitMethodInsn(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", "notify", "(Ljava/lang/Object;)V", false);
+                            return;
+
+                        case 9: // notifyAll
+                            transformer.witness(MONITOR);
+                            super.visitMethodInsn(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", "notifyAll", "(Ljava/lang/Object;)V", false);
+                            return;
+                    }
+            }
+        }
+        if (opcode == Opcodes.INVOKESTATIC && !isInterface && owner.equals("java/util/concurrent/locks/LockSupport"))
+        {
+            transformer.witness(LOCK_SUPPORT);
+            super.visitMethodInsn(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptibleThread", name, descriptor, false);
+            return;
+        }
+        super.visitMethodInsn(opcode, owner, name, descriptor, isInterface);
+    }
+
+    @Override
+    public void visitInsn(int opcode)
+    {
+        switch (opcode)
+        {
+            case Opcodes.MONITORENTER:
+                transformer.witness(MONITOR);
+                super.visitLdcInsn(monitorDelayChance.get());
+                super.visitMethodInsn(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", "preMonitorEnter", "(Ljava/lang/Object;F)Ljava/lang/Object;", false);
+                break;
+            case Opcodes.MONITOREXIT:
+                transformer.witness(MONITOR);
+                super.visitMethodInsn(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", "preMonitorExit", "(Ljava/lang/Object;)Ljava/lang/Object;", false);
+                break;
+        }
+        super.visitInsn(opcode);
+    }
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/MonitorMethodTransformer.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/MonitorMethodTransformer.java
new file mode 100644
index 0000000..d9c9c7a
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/MonitorMethodTransformer.java
@@ -0,0 +1,338 @@
+/*
+ * 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.simulator.asm;
+
+import java.util.Comparator;
+import java.util.ListIterator;
+
+import org.objectweb.asm.Label;
+import org.objectweb.asm.Opcodes;
+import org.objectweb.asm.Type;
+import org.objectweb.asm.tree.FrameNode;
+import org.objectweb.asm.tree.InsnNode;
+import org.objectweb.asm.tree.IntInsnNode;
+import org.objectweb.asm.tree.LdcInsnNode;
+import org.objectweb.asm.tree.LocalVariableNode;
+import org.objectweb.asm.tree.MethodInsnNode;
+import org.objectweb.asm.tree.MethodNode;
+import org.objectweb.asm.tree.TryCatchBlockNode;
+
+import static org.apache.cassandra.simulator.asm.TransformationKind.MONITOR;
+
+/**
+ * For synchronized methods, we generate a new method that contains the source method's body, and the original method
+ * instead invoke preMonitorEnter before invoking the new hidden method.
+ */
+class MonitorMethodTransformer extends MethodNode
+{
+    private final String className;
+    private final MethodWriterSink methodWriterSink;
+    private final ChanceSupplier monitorDelayChance;
+    private final String baseName;
+    private final boolean isInstanceMethod;
+    private int returnCode;
+
+    int maxLocalParams; // double counts long/double to match asm spec
+
+    public MonitorMethodTransformer(MethodWriterSink methodWriterSink, String className, int api, int access, String name, String descriptor, String signature, String[] exceptions, ChanceSupplier monitorDelayChance)
+    {
+        super(api, access, name, descriptor, signature, exceptions);
+        this.methodWriterSink = methodWriterSink;
+        this.className = className;
+        this.baseName = name;
+        this.isInstanceMethod = (access & Opcodes.ACC_STATIC) == 0;
+        this.monitorDelayChance = monitorDelayChance;
+    }
+
+    @Override
+    public void visitInsn(int opcode)
+    {
+        switch (opcode)
+        {
+            case Opcodes.RETURN:
+            case Opcodes.ARETURN:
+            case Opcodes.IRETURN:
+            case Opcodes.FRETURN:
+            case Opcodes.LRETURN:
+            case Opcodes.DRETURN:
+                if (returnCode != 0) assert returnCode == opcode;
+                else returnCode = opcode;
+        }
+        super.visitInsn(opcode);
+    }
+
+    int returnCode()
+    {
+        return returnCode;
+    }
+
+    // TODO (cleanup): this _should_ be possible to determine purely from the method signature
+    int loadParamsAndReturnInvokeCode()
+    {
+        if (isInstanceMethod)
+            instructions.add(new IntInsnNode(Opcodes.ALOAD, 0));
+
+        ListIterator<LocalVariableNode> it = localVariables.listIterator();
+        while (it.hasNext())
+        {
+            LocalVariableNode cur = it.next();
+            if (cur.index < maxLocalParams)
+            {
+                if (!isInstanceMethod || cur.index > 0)
+                {
+                    int opcode;
+                    switch (cur.desc.charAt(0))
+                    {
+                        case 'L':
+                        case '[':
+                            opcode = Opcodes.ALOAD;
+                            break;
+                        case 'J':
+                            opcode = Opcodes.LLOAD;
+                            break;
+                        case 'D':
+                            opcode = Opcodes.DLOAD;
+                            break;
+                        case 'F':
+                            opcode = Opcodes.FLOAD;
+                            break;
+                        default:
+                            opcode = Opcodes.ILOAD;
+                            break;
+                    }
+                    instructions.add(new IntInsnNode(opcode, cur.index));
+                }
+            }
+        }
+
+        int invokeCode;
+        if (isInstanceMethod && (access & Opcodes.ACC_PRIVATE) != 0) invokeCode = Opcodes.INVOKESPECIAL;
+        else if (isInstanceMethod) invokeCode = Opcodes.INVOKEVIRTUAL;
+        else invokeCode = Opcodes.INVOKESTATIC;
+        return invokeCode;
+    }
+
+    void pushRef()
+    {
+        if (isInstanceMethod) instructions.add(new IntInsnNode(Opcodes.ALOAD, 0));
+        else instructions.add(new LdcInsnNode(org.objectweb.asm.Type.getType('L' + className + ';')));
+    }
+
+    void pop()
+    {
+        instructions.add(new InsnNode(Opcodes.POP));
+    }
+
+    void invokePreMonitorExit()
+    {
+        pushRef();
+        instructions.add(new MethodInsnNode(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", "preMonitorExit", "(Ljava/lang/Object;)Ljava/lang/Object;", false));
+    }
+
+    void invokePreMonitorEnter()
+    {
+        pushRef();
+        instructions.add(new LdcInsnNode(monitorDelayChance.get()));
+        instructions.add(new MethodInsnNode(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", "preMonitorEnter", "(Ljava/lang/Object;F)Ljava/lang/Object;", false));
+    }
+
+    void invokeMonitor(int insn)
+    {
+        instructions.add(new InsnNode(insn));
+    }
+
+    void reset(Label start, Label end)
+    {
+        instructions.clear();
+        tryCatchBlocks.clear();
+        if (visibleLocalVariableAnnotations != null)
+            visibleLocalVariableAnnotations.clear();
+        if (invisibleLocalVariableAnnotations != null)
+            invisibleLocalVariableAnnotations.clear();
+
+        Type[] args = Type.getArgumentTypes(desc);
+        // remove all local variables that aren't parameters and the `this` parameter
+        maxLocals = args.length == 1 && Type.VOID_TYPE.equals(args[0]) ? 0 : args.length;
+        if (isInstanceMethod) ++maxLocals;
+
+        // sort our local variables and remove those that aren't parameters
+        localVariables.sort(Comparator.comparingInt(c -> c.index));
+        ListIterator<LocalVariableNode> it = localVariables.listIterator();
+        while (it.hasNext())
+        {
+            LocalVariableNode cur = it.next();
+            if (cur.index >= maxLocals)
+            {
+                it.remove();
+            }
+            else
+            {
+                it.set(new LocalVariableNode(cur.name, cur.desc, cur.signature, getLabelNode(start), getLabelNode(end), cur.index));
+                switch (cur.desc.charAt(0))
+                {
+                    case 'J':
+                    case 'D':
+                        // doubles and longs take two local variable positions
+                        ++maxLocals;
+                }
+            }
+        }
+
+        // save the number of pure-parameters for use elsewhere
+        maxLocalParams = maxLocals;
+    }
+
+    void writeOriginal()
+    {
+        access &= ~Opcodes.ACC_SYNCHRONIZED;
+        access |= Opcodes.ACC_SYNTHETIC;
+        name = baseName + "$unsync";
+        methodWriterSink.writeMethod(this);
+    }
+
+    // alternative approach (with writeInnerTryCatchSynchronized)
+    @SuppressWarnings("unused")
+    void writeOuterUnsynchronized()
+    {
+        access &= ~(Opcodes.ACC_SYNCHRONIZED | Opcodes.ACC_SYNTHETIC);
+        name = baseName;
+
+        Label start = new Label();
+        Label end = new Label();
+
+        reset(start, end);
+        maxStack = maxLocalParams;
+
+        instructions.add(getLabelNode(start));
+        invokePreMonitorEnter();
+        pop();
+
+        int invokeCode = loadParamsAndReturnInvokeCode();
+        instructions.add(new MethodInsnNode(invokeCode, className, baseName + "$catch", desc));
+        instructions.add(new InsnNode(returnCode()));
+        instructions.add(getLabelNode(end));
+        methodWriterSink.writeMethod(this);
+    }
+
+    // alternative approach (with writeOuterUnsynchronized)
+    @SuppressWarnings("unused")
+    void writeInnerTryCatchSynchronized()
+    {
+        access |= Opcodes.ACC_SYNCHRONIZED | Opcodes.ACC_SYNTHETIC;
+        name = baseName + "$catch";
+
+        Label start = new Label();
+        Label normal = new Label();
+        Label except = new Label();
+        Label end = new Label();
+        reset(start, end);
+        maxStack = Math.max(maxLocalParams, returnCode == Opcodes.RETURN ? 1 : 2); // must load self or class onto stack, and return value (if any)
+        ++maxLocals;
+        tryCatchBlocks.add(new TryCatchBlockNode(getLabelNode(start), getLabelNode(normal), getLabelNode(except), null));
+        instructions.add(getLabelNode(start));
+        int invokeCode = loadParamsAndReturnInvokeCode();
+        instructions.add(new MethodInsnNode(invokeCode, className, baseName + "$unsync", desc));
+        instructions.add(getLabelNode(normal));
+        invokePreMonitorExit();
+        instructions.add(new InsnNode(returnCode()));
+        instructions.add(getLabelNode(except));
+        instructions.add(new FrameNode(Opcodes.F_SAME1, 0, null, 1, new Object[]{ "java/lang/Throwable" }));
+        instructions.add(new IntInsnNode(Opcodes.ASTORE, maxLocalParams));
+        invokePreMonitorExit();
+        instructions.add(new IntInsnNode(Opcodes.ALOAD, maxLocalParams));
+        instructions.add(new InsnNode(Opcodes.ATHROW));
+        instructions.add(getLabelNode(end));
+        methodWriterSink.writeSyntheticMethod(MONITOR, this);
+    }
+
+    void writeTryCatchMonitorEnterExit()
+    {
+        access |= Opcodes.ACC_SYNTHETIC;
+        name = baseName;
+
+        Label start = new Label();
+        Label inmonitor = new Label();
+        Label normal = new Label();
+        Label except = new Label(); // normal
+        Label normalRetExcept = new Label(); // normal return failed
+        Label exceptRetNormal = new Label(); // exceptional return success
+        Label exceptRetExcept = new Label(); // exceptional return failed
+        Label end = new Label();
+        reset(start, end);
+        ++maxLocals; // add a local variable slot to save any exceptions into (at maxLocalParams position)
+        maxStack = Math.max(maxLocalParams, returnCode == Opcodes.RETURN ? 2 : 3); // must load self or class onto stack, and return value (if any)
+        tryCatchBlocks.add(new TryCatchBlockNode(getLabelNode(inmonitor), getLabelNode(normal), getLabelNode(except), null));
+        tryCatchBlocks.add(new TryCatchBlockNode(getLabelNode(normal), getLabelNode(normalRetExcept), getLabelNode(normalRetExcept), null));
+        tryCatchBlocks.add(new TryCatchBlockNode(getLabelNode(except), getLabelNode(exceptRetNormal), getLabelNode(exceptRetExcept), null));
+        // preMonitorEnter
+        // monitorenter
+        instructions.add(getLabelNode(start));
+        invokePreMonitorEnter();
+        invokeMonitor(Opcodes.MONITORENTER);
+        {
+            // try1 { val = original();
+            instructions.add(getLabelNode(inmonitor));
+            int invokeCode = loadParamsAndReturnInvokeCode();
+            instructions.add(new MethodInsnNode(invokeCode, className, baseName + "$unsync", desc));
+            {
+                // try2 { preMonitorExit(); monitorexit; return val; }
+                instructions.add(getLabelNode(normal));
+                invokePreMonitorExit();
+                invokeMonitor(Opcodes.MONITOREXIT);
+                instructions.add(new InsnNode(returnCode())); // success
+                // }
+                // catch2 { monitorexit; throw }
+                instructions.add(getLabelNode(normalRetExcept));
+                instructions.add(new FrameNode(Opcodes.F_SAME1, 0, null, 1, new Object[]{ "java/lang/Throwable" }));
+                instructions.add(new IntInsnNode(Opcodes.ASTORE, maxLocalParams));
+                pushRef();
+                invokeMonitor(Opcodes.MONITOREXIT);
+                instructions.add(new IntInsnNode(Opcodes.ALOAD, maxLocalParams));
+                instructions.add(new InsnNode(Opcodes.ATHROW));
+                // }
+            }
+            // catch1 { try3 { preMonitorExit; monitorexit; throw
+            instructions.add(getLabelNode(except));
+            instructions.add(new FrameNode(Opcodes.F_SAME1, 0, null, 1, new Object[]{ "java/lang/Throwable" }));
+            instructions.add(new IntInsnNode(Opcodes.ASTORE, maxLocalParams));
+            invokePreMonitorExit();
+            invokeMonitor(Opcodes.MONITOREXIT);
+            instructions.add(new IntInsnNode(Opcodes.ALOAD, maxLocalParams));
+            instructions.add(getLabelNode(exceptRetNormal));
+            instructions.add(new InsnNode(Opcodes.ATHROW));
+            instructions.add(getLabelNode(exceptRetExcept));
+            instructions.add(new FrameNode(Opcodes.F_SAME1, 0, null, 1, new Object[]{ "java/lang/Throwable" }));
+            instructions.add(new IntInsnNode(Opcodes.ASTORE, maxLocalParams));
+            pushRef();
+            invokeMonitor(Opcodes.MONITOREXIT);
+            instructions.add(new IntInsnNode(Opcodes.ALOAD, maxLocalParams));
+            instructions.add(new InsnNode(Opcodes.ATHROW));
+        }
+        instructions.add(getLabelNode(end));
+        methodWriterSink.writeSyntheticMethod(MONITOR, this);
+    }
+
+    @Override
+    public void visitEnd()
+    {
+        writeOriginal();
+        writeTryCatchMonitorEnterExit();
+        super.visitEnd();
+    }
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/NemesisFieldKind.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/NemesisFieldKind.java
new file mode 100644
index 0000000..d26f196
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/NemesisFieldKind.java
@@ -0,0 +1,50 @@
+/*
+ * 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.simulator.asm;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicLongFieldUpdater;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+
+public enum NemesisFieldKind
+{
+    SIMPLE(),
+    ATOMICX(AtomicBoolean.class, AtomicInteger.class, AtomicLong.class, AtomicReference.class),
+    ATOMICUPDATERX(AtomicIntegerFieldUpdater.class, AtomicLongFieldUpdater.class, AtomicReferenceFieldUpdater.class);
+
+    public interface Selector
+    {
+        NemesisFieldKind get(String className, String fieldName);
+    }
+
+    final Set<Class<?>> classes;
+
+    NemesisFieldKind(Class<?> ... classes)
+    {
+        this.classes = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(classes)));
+    }
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/NemesisGenerator.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/NemesisGenerator.java
new file mode 100644
index 0000000..814348b
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/NemesisGenerator.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.simulator.asm;
+
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.Opcodes;
+import org.objectweb.asm.tree.InsnNode;
+import org.objectweb.asm.tree.LabelNode;
+import org.objectweb.asm.tree.LdcInsnNode;
+import org.objectweb.asm.tree.MethodInsnNode;
+import org.objectweb.asm.tree.MethodNode;
+
+/**
+ * Generate a new static method in the class with a randomly generated constant chance of triggering the nemesis.
+ * Generate also the invocation of this method at the relevant point(s).
+ *
+ * A static method with no parameters or return values is created, so that only the method invocation instruction is
+ * needed in the original method, simplifying the transformation.
+ */
+class NemesisGenerator extends MethodNode
+{
+    private final ChanceSupplier chanceSupplier;
+
+    private final String className;
+    private String baseName;
+    private int methodsCounter = 0; // avoid nemesis method name clashes when weaving two or more methods with same name
+    private int withinMethodCounter = 0;
+    private final LdcInsnNode ldc = new LdcInsnNode(null);
+
+    NemesisGenerator(int api, String className, ChanceSupplier chanceSupplier)
+    {
+        super(api, Opcodes.ACC_STATIC | Opcodes.ACC_SYNTHETIC | Opcodes.ACC_PRIVATE, null, "()V", "", null);
+        this.chanceSupplier = chanceSupplier;
+        this.className = className;
+        this.maxLocals = 0;
+        this.maxStack = 1;
+        instructions.add(new LabelNode());
+        instructions.add(ldc);
+        instructions.add(new MethodInsnNode(Opcodes.INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", "nemesis", "(F)V", false));
+        instructions.add(new LabelNode());
+        instructions.add(new InsnNode(Opcodes.RETURN));
+    }
+
+    void newMethod(String name)
+    {
+        this.baseName = name.replaceAll("[<>]", "") + '$' + (methodsCounter++) + '$';
+        this.withinMethodCounter = 0;
+    }
+
+    void generateAndCall(TransformationKind kind, ClassTransformer writeDefinitionTo, MethodVisitor writeInvocationTo)
+    {
+        this.name = baseName + '$' + (withinMethodCounter++) + "$nemesis";
+        ldc.cst = chanceSupplier.get();
+        writeDefinitionTo.writeSyntheticMethod(kind, this);
+        writeInvocationTo.visitMethodInsn(Opcodes.INVOKESTATIC, className, name, "()V", false);
+    }
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/NemesisTransformer.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/NemesisTransformer.java
new file mode 100644
index 0000000..94ed1f4
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/NemesisTransformer.java
@@ -0,0 +1,134 @@
+/*
+ * 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.simulator.asm;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.Opcodes;
+
+import static org.apache.cassandra.simulator.asm.TransformationKind.FIELD_NEMESIS;
+import static org.apache.cassandra.simulator.asm.TransformationKind.SIGNAL_NEMESIS;
+
+/**
+ * Insert nemesis points at all obvious thread signalling points (execution and blocking primitive methods),
+ * as well as to any fields annotated with {@link org.apache.cassandra.utils.Nemesis}.
+ *
+ * If the annotated field is an AtomicX or AtomicXFieldUpdater, we insert nemesis points either side of the next
+ * invocation of
+ *
+ * TODO (config): permit Nemesis on a class as well as a field, so as to mark all (at least volatile or atomic) members
+ */
+class NemesisTransformer extends MethodVisitor
+{
+    private final ClassTransformer transformer;
+    final NemesisGenerator generator;
+    final NemesisFieldKind.Selector nemesisFieldSelector;
+
+    // for simplicity, we simply activate nemesis for all atomic operations on the relevant type once any such
+    // field is loaded in a method
+    Set<String> onForTypes;
+
+    public NemesisTransformer(ClassTransformer transformer, int api, String name, MethodVisitor parent, NemesisGenerator generator, NemesisFieldKind.Selector nemesisFieldSelector)
+    {
+        super(api, parent);
+        this.transformer = transformer;
+        this.generator = generator;
+        this.nemesisFieldSelector = nemesisFieldSelector;
+        generator.newMethod(name);
+    }
+
+    @Override
+    public void visitMethodInsn(int opcode, String owner, String name, String descriptor, boolean isInterface)
+    {
+        boolean nemesisAfter = false;
+        if (isInterface && opcode == Opcodes.INVOKEINTERFACE
+        && (owner.startsWith("org/apache/cassandra/concurrent") || owner.startsWith("org/apache/cassandra/utils/concurrent")) && (
+               (owner.equals("org/apache/cassandra/utils/concurrent/CountDownLatch") && name.equals("decrement"))
+            || (owner.equals("org/apache/cassandra/utils/concurrent/Condition") && name.equals("signal"))
+            || (owner.equals("org/apache/cassandra/utils/concurrent/Semaphore") && name.equals("release"))
+            || ((owner.equals("org/apache/cassandra/concurrent/ExecutorPlus")
+                 || owner.equals("org/apache/cassandra/concurrent/LocalAwareExecutorPlus")
+                 || owner.equals("org/apache/cassandra/concurrent/ScheduledExecutorPlus")
+                 || owner.equals("org/apache/cassandra/concurrent/SequentialExecutorPlus")
+                 || owner.equals("org/apache/cassandra/concurrent/LocalAwareSequentialExecutorPlus")
+                ) && (name.equals("execute") || name.equals("submit") || name.equals("maybeExecuteImmediately")))
+        ))
+        {
+            generateAndCall(SIGNAL_NEMESIS);
+        }
+        else if ((opcode == Opcodes.INVOKESPECIAL || opcode == Opcodes.INVOKEVIRTUAL)
+                 && (onForTypes != null && onForTypes.contains(owner)))
+        {
+            nemesisAfter = true;
+            generateAndCall(FIELD_NEMESIS);
+        }
+
+        super.visitMethodInsn(opcode, owner, name, descriptor, isInterface);
+        if (nemesisAfter)
+            generateAndCall(FIELD_NEMESIS);
+    }
+
+    @Override
+    public void visitFieldInsn(int opcode, String owner, String name, String descriptor)
+    {
+        boolean nemesisAfter = false;
+        NemesisFieldKind nemesis = nemesisFieldSelector.get(owner, name);
+        if (nemesis != null)
+        {
+            switch (nemesis)
+            {
+                case SIMPLE:
+                    switch (opcode)
+                    {
+                        default:
+                            throw new AssertionError();
+                        case Opcodes.PUTFIELD:
+                        case Opcodes.PUTSTATIC:
+                            generateAndCall(FIELD_NEMESIS);
+                            break;
+                        case Opcodes.GETFIELD:
+                        case Opcodes.GETSTATIC:
+                            nemesisAfter = true;
+                    }
+                    break;
+                case ATOMICX:
+                case ATOMICUPDATERX:
+                    switch (opcode)
+                    {
+                        case Opcodes.GETFIELD:
+                        case Opcodes.GETSTATIC:
+                            if (onForTypes == null)
+                                onForTypes = new HashSet<>();
+                            onForTypes.add(descriptor.substring(1, descriptor.length() - 1));
+                    }
+                    break;
+            }
+        }
+        super.visitFieldInsn(opcode, owner, name, descriptor);
+        if (nemesisAfter)
+            generateAndCall(FIELD_NEMESIS);
+    }
+
+    private void generateAndCall(TransformationKind kind)
+    {
+        generator.generateAndCall(kind, transformer, mv);
+    }
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/ShadowingTransformer.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/ShadowingTransformer.java
new file mode 100644
index 0000000..f75bdee
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/ShadowingTransformer.java
@@ -0,0 +1,278 @@
+/*
+ * 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.simulator.asm;
+
+import java.util.EnumSet;
+
+import org.objectweb.asm.AnnotationVisitor;
+import org.objectweb.asm.FieldVisitor;
+import org.objectweb.asm.Handle;
+import org.objectweb.asm.Label;
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.Type;
+import org.objectweb.asm.TypePath;
+
+/**
+ * A SORT OF general purpose facility for creating a copy of a system class that we want to transform
+ * and use in place of the system class without transforming the system class itself.
+ *
+ * NOTE that this does not implement this translation perfectly, so care must be taken when extending its usage
+ * Some things not handled:
+ *   - generic type signatures in class files
+ *   - 
+ *
+ * While it is possible and safe in principle to modify ConcurrentHashMap in particular, in practice it messes
+ * with class loading, as ConcurrentHashMap is used widely within the JDK, including for things like class loaders
+ * and method handle caching. It seemed altogether more tractable and safe to selectively replace ConcurrentHashMap
+ * with a shadowed variety.
+ *
+ * This approach makes some rough assumptions, namely that any public method on the root class should accept the
+ * shadowed type, but that any inner class may safely use the shadow type.
+ */
+public class ShadowingTransformer extends ClassTransformer
+{
+    final String originalType;
+    final String originalRootType;
+    final String shadowRootType;
+    final String originalOuterTypePrefix;
+    final String shadowOuterTypePrefix;
+    String originalSuperName;
+
+    ShadowingTransformer(int api, String originalType, String shadowType, String originalRootType, String shadowRootType, String originalOuterTypePrefix, String shadowOuterTypePrefix, EnumSet<Flag> flags, ChanceSupplier monitorDelayChance, NemesisGenerator nemesis, NemesisFieldKind.Selector nemesisFieldSelector, Hashcode insertHashcode)
+    {
+        super(api, shadowType, flags, monitorDelayChance, nemesis, nemesisFieldSelector, insertHashcode);
+        this.originalType = originalType;
+        this.originalRootType = originalRootType;
+        this.shadowRootType = shadowRootType;
+        this.originalOuterTypePrefix = originalOuterTypePrefix;
+        this.shadowOuterTypePrefix = shadowOuterTypePrefix;
+    }
+
+    private String toShadowType(String type)
+    {
+        if (type.startsWith("["))
+            return toShadowTypeDescriptor(type);
+        else if (type.equals(originalRootType))
+            type = shadowRootType;
+        else if (type.startsWith(originalOuterTypePrefix))
+            type = shadowOuterTypePrefix + type.substring(originalOuterTypePrefix.length());
+        else
+            return type;
+
+        witness(TransformationKind.SHADOW);
+        return type;
+    }
+
+    private String toShadowTypeDescriptor(String owner)
+    {
+        return toShadowTypeDescriptor(owner, false);
+    }
+
+    private String toShadowTypeDescriptor(String desc, boolean innerTypeOnly)
+    {
+        int i = 0;
+        while (i < desc.length() && desc.charAt(i) == '[') ++i;
+        if (desc.charAt(i) != 'L')
+            return desc;
+        ++i;
+
+        if (!innerTypeOnly && desc.regionMatches(i, originalRootType, 0, originalRootType.length()) && desc.length() == originalRootType.length() + 1 + i && desc.charAt(i + originalRootType.length()) == ';')
+            desc = desc.substring(0, i) + shadowRootType + ';';
+        else if (desc.regionMatches(i, originalOuterTypePrefix, 0, originalOuterTypePrefix.length()))
+            desc = desc.substring(0, i) + shadowOuterTypePrefix + desc.substring(i + originalOuterTypePrefix.length());
+        else
+            return desc;
+
+        witness(TransformationKind.SHADOW);
+        return desc;
+    }
+
+    private Type toShadowTypeDescriptor(Type type)
+    {
+        String in = type.getDescriptor();
+        String out = toShadowTypeDescriptor(in, false);
+        if (in == out) return type;
+        return Type.getType(out);
+    }
+
+    private Type toShadowInnerTypeDescriptor(Type type)
+    {
+        String in = type.getDescriptor();
+        String out = toShadowTypeDescriptor(in, true);
+        if (in == out) return type;
+        return Type.getType(out);
+    }
+
+    Object[] toShadowTypes(Object[] in)
+    {
+        Object[] out = null;
+        for (int i = 0 ; i < in.length ; ++i)
+        {
+            if (in[i] instanceof String)
+            {
+                // TODO (broader correctness): in some cases we want the original type, and others the new type
+                String inv = (String) in[i];
+                String outv = toShadowType(inv);
+                if (inv != outv)
+                {
+                    if (out == null)
+                    {
+                        out = new Object[in.length];
+                        System.arraycopy(in, 0, out, 0, i);
+                    }
+                    out[i] = outv;
+                    continue;
+                }
+            }
+
+            if (out != null)
+                out[i] = in[i];
+        }
+        return out != null ? out : in;
+    }
+
+    String methodDescriptorToShadowInnerArgumentTypes(String descriptor)
+    {
+        Type ret = toShadowTypeDescriptor(Type.getReturnType(descriptor));
+        Type[] args = Type.getArgumentTypes(descriptor);
+        for (int i = 0 ; i < args.length ; ++i)
+            args[i] = toShadowInnerTypeDescriptor(args[i]);
+        return Type.getMethodDescriptor(ret, args);
+    }
+
+    String methodDescriptorToShadowTypes(String descriptor)
+    {
+        Type ret = toShadowTypeDescriptor(Type.getReturnType(descriptor));
+        Type[] args = Type.getArgumentTypes(descriptor);
+        for (int i = 0 ; i < args.length ; ++i)
+            args[i] = toShadowTypeDescriptor(args[i]);
+        return Type.getMethodDescriptor(ret, args);
+    }
+
+    class ShadowingMethodVisitor extends MethodVisitor
+    {
+        final boolean isConstructor;
+        public ShadowingMethodVisitor(int api, boolean isConstructor, MethodVisitor methodVisitor)
+        {
+            super(api, methodVisitor);
+            this.isConstructor = isConstructor;
+        }
+
+        @Override
+        public AnnotationVisitor visitTypeAnnotation(int typeRef, TypePath typePath, String descriptor, boolean visible)
+        {
+            return super.visitTypeAnnotation(typeRef, typePath, descriptor, visible);
+        }
+
+        @Override
+        public void visitFieldInsn(int opcode, String owner, String name, String descriptor)
+        {
+            super.visitFieldInsn(opcode, toShadowType(owner), name, toShadowTypeDescriptor(descriptor));
+        }
+
+        @Override
+        public void visitTypeInsn(int opcode, String type)
+        {
+            // TODO (broader correctness): in some cases we want the original type, and others the new type
+            super.visitTypeInsn(opcode, toShadowType(type));
+        }
+
+        @Override
+        public void visitLocalVariable(String name, String descriptor, String signature, Label start, Label end, int index)
+        {
+            super.visitLocalVariable(name, toShadowTypeDescriptor(descriptor), signature, start, end, index);
+        }
+
+        @Override
+        public void visitFrame(int type, int numLocal, Object[] local, int numStack, Object[] stack)
+        {
+            super.visitFrame(type, numLocal, toShadowTypes(local), numStack, toShadowTypes(stack));
+        }
+
+        @Override
+        public void visitMethodInsn(int opcode, String owner, String name, String descriptor, boolean isInterface)
+        {
+            // TODO (broader correctness): this is incorrect, but will do for ConcurrentHashMap (no general guarantee of same constructors)
+            if (owner.equals(originalRootType)) descriptor = methodDescriptorToShadowInnerArgumentTypes(descriptor);
+            else descriptor = methodDescriptorToShadowTypes(descriptor);
+            if (isConstructor && name.equals("<init>") && owner.equals(originalSuperName) && originalType.equals(originalRootType)) owner = originalRootType;
+            else owner = toShadowType(owner);
+            super.visitMethodInsn(opcode, owner, name, descriptor, isInterface);
+        }
+
+        @Override
+        public void visitInvokeDynamicInsn(String name, String descriptor, Handle bootstrapMethodHandle, Object... bootstrapMethodArguments)
+        {
+            if (bootstrapMethodHandle.getOwner().startsWith(originalRootType))
+            {
+                bootstrapMethodHandle = new Handle(bootstrapMethodHandle.getTag(), toShadowType(bootstrapMethodHandle.getOwner()),
+                                                   bootstrapMethodHandle.getName(), bootstrapMethodHandle.getDesc(),
+                                                   bootstrapMethodHandle.isInterface());
+            }
+            super.visitInvokeDynamicInsn(name, descriptor, bootstrapMethodHandle, bootstrapMethodArguments);
+        }
+
+        @Override
+        public void visitLdcInsn(Object value)
+        {
+            if (value instanceof Type)
+                value = toShadowTypeDescriptor((Type) value);
+            super.visitLdcInsn(value);
+        }
+    }
+
+    @Override
+    public void visitInnerClass(String name, String outerName, String innerName, int access)
+    {
+        super.visitInnerClass(name, toShadowType(outerName), innerName, access);
+    }
+
+    @Override
+    public FieldVisitor visitField(int access, String name, String descriptor, String signature, Object value)
+    {
+        return super.visitField(access, name, toShadowTypeDescriptor(descriptor), signature, value);
+    }
+
+    @Override
+    public MethodVisitor visitMethod(int access, String name, String descriptor, String signature, String[] exceptions)
+    {
+        if (originalType.equals(originalRootType)) descriptor = methodDescriptorToShadowInnerArgumentTypes(descriptor);
+        else descriptor = methodDescriptorToShadowTypes(descriptor);
+        return new ShadowingMethodVisitor(api, name.equals("<init>"), super.visitMethod(access, name, descriptor, signature, exceptions));
+    }
+
+    @Override
+    public void visit(int version, int access, String name, String signature, String superName, String[] interfaces)
+    {
+        originalSuperName = superName;
+        if (originalType.equals(originalRootType))
+        {
+            superName = name;
+            name = shadowRootType;
+        }
+        else
+        {
+            name = toShadowType(name);
+            superName = toShadowType(superName);
+        }
+
+        super.visit(version, access, name, signature, superName, interfaces);
+    }
+
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/TransformationKind.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/TransformationKind.java
new file mode 100644
index 0000000..841a28c
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/TransformationKind.java
@@ -0,0 +1,29 @@
+/*
+ * 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.simulator.asm;
+
+import java.util.Arrays;
+import java.util.List;
+
+public enum TransformationKind
+{
+    MONITOR, SYNCHRONIZED, LOCK_SUPPORT, GLOBAL_METHOD, SIGNAL_NEMESIS, FIELD_NEMESIS, SYNTHETIC_METHOD, HASHCODE,
+    IDENTITY_HASH_MAP, CONCURRENT_HASH_MAP, SHADOW;
+    static final List<TransformationKind> VALUES = Arrays.asList(values());
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/Utils.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/Utils.java
new file mode 100644
index 0000000..0422bf0
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/Utils.java
@@ -0,0 +1,265 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.simulator.asm;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.BiConsumer;
+
+import org.objectweb.asm.AnnotationVisitor;
+import org.objectweb.asm.Label;
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.Opcodes;
+import org.objectweb.asm.Type;
+import org.objectweb.asm.tree.AbstractInsnNode;
+
+import static org.apache.cassandra.simulator.asm.InterceptClasses.BYTECODE_VERSION;
+import static org.objectweb.asm.Opcodes.ATHROW;
+import static org.objectweb.asm.Opcodes.F_SAME1;
+import static org.objectweb.asm.Opcodes.INVOKESTATIC;
+
+public class Utils
+{
+    public static String toInternalName(Class<?> clazz)
+    {
+        return toInternalName(clazz.getName());
+    }
+
+    public static String toInternalName(String className)
+    {
+        return className.replace('.', '/');
+    }
+
+    public static String toPath(Class<?> clazz)
+    {
+        return toInternalName(clazz) + ".class";
+    }
+
+    public static byte[] readDefinition(Class<?> clazz) throws IOException
+    {
+        return readDefinition(toPath(clazz));
+    }
+
+    public static byte[] readDefinition(String path) throws IOException
+    {
+        byte[] bytes = new byte[1024];
+        try (InputStream in = ClassLoader.getSystemResourceAsStream(path))
+        {
+            int count = 0;
+            while (true)
+            {
+                int add = in.read(bytes, count, bytes.length - count);
+                if (add < 0)
+                    break;
+                if (add == 0)
+                    bytes = Arrays.copyOf(bytes, bytes.length * 2);
+                count += add;
+            }
+            return Arrays.copyOf(bytes, count);
+        }
+    }
+
+    /**
+     * Generate a proxy method call, i.e. one whose only job is forwarding the parameters to a different method
+     * (and perhaps within a superclass, or another class entirely if static) with the same signature but perhaps
+     * different properties.
+     */
+    private static long visitProxyCall(MethodVisitor visitor, String calleeClassName, String calleeMethodName, String descriptor, int access, boolean isInstanceMethod, boolean isInterface)
+    {
+        Type[] argTypes = Type.getArgumentTypes(descriptor);
+        Type returnType = Type.getReturnType(descriptor);
+
+        int stackSize = argTypes.length;
+        int localsSize = 0;
+        if (isInstanceMethod)
+        {
+            visitor.visitIntInsn(Opcodes.ALOAD, 0);
+            localsSize += 2;
+            stackSize += 1;
+        }
+
+        int i = 1;
+        for (Type type : argTypes)
+        {
+            int opcode;
+            switch (type.getDescriptor().charAt(0))
+            {
+                case 'L':
+                case '[':
+                    opcode = Opcodes.ALOAD;
+                    localsSize += 1;
+                    break;
+                case 'J':
+                    opcode = Opcodes.LLOAD;
+                    localsSize += 2;
+                    break;
+                case 'D':
+                    opcode = Opcodes.DLOAD;
+                    localsSize += 2;
+                    break;
+                case 'F':
+                    opcode = Opcodes.FLOAD;
+                    localsSize += 1;
+                    break;
+                default:
+                    opcode = Opcodes.ILOAD;
+                    localsSize += 1;
+                    break;
+            }
+            visitor.visitIntInsn(opcode, i++);
+        }
+
+        int returnCode;
+        switch (returnType.getDescriptor().charAt(0))
+        {
+            case 'L':
+            case '[':
+                returnCode = Opcodes.ARETURN;
+                localsSize = Math.max(localsSize, 1);
+                break;
+            case 'J':
+                returnCode = Opcodes.LRETURN;
+                localsSize = Math.max(localsSize, 2);
+                break;
+            case 'D':
+                returnCode = Opcodes.DRETURN;
+                localsSize = Math.max(localsSize, 2);
+                break;
+            case 'F':
+                returnCode = Opcodes.FRETURN;
+                localsSize = Math.max(localsSize, 1);
+                break;
+            case 'V':
+                returnCode = Opcodes.RETURN;
+                break;
+            default:
+                returnCode = Opcodes.IRETURN;
+                localsSize = Math.max(localsSize, 1);
+                break;
+        }
+
+        int invokeCode;
+        if (isInstanceMethod && (access & Opcodes.ACC_PRIVATE) != 0 || calleeMethodName.equals("<init>")) invokeCode = Opcodes.INVOKESPECIAL;
+        else if (isInstanceMethod) invokeCode = Opcodes.INVOKEVIRTUAL;
+        else invokeCode = Opcodes.INVOKESTATIC;
+
+        visitor.visitMethodInsn(invokeCode, calleeClassName, calleeMethodName, descriptor, isInterface);
+
+        return localsSize | (((long)stackSize) << 28) | (((long) returnCode) << 56);
+    }
+
+    /**
+     * Generate a proxy method call, i.e. one whose only job is forwarding the parameters to a different method
+     * (and perhaps within a superclass, or another class entirely if static) with the same signature but perhaps
+     * different properties.
+     */
+    public static void generateProxyCall(MethodVisitor visitor, String calleeClassName, String calleeMethodName, String descriptor, int access, boolean isInstanceMethod, boolean isInterface)
+    {
+        Label start = new Label(), end = new Label();
+        visitor.visitLabel(start);
+
+        long sizesAndReturnCode = visitProxyCall(visitor, calleeClassName, calleeMethodName, descriptor, access, isInstanceMethod, isInterface);
+        visitor.visitLabel(end);
+        visitor.visitInsn((int)(sizesAndReturnCode >>> 56) & 0xff);
+        visitor.visitMaxs((int)(sizesAndReturnCode >>> 28) & 0xfffffff, (int)(sizesAndReturnCode & 0xfffffff));
+        visitor.visitEnd();
+    }
+
+    /**
+     * Generate a proxy method call, i.e. one whose only job is forwarding the parameters to a different method
+     * (and perhaps within a superclass, or another class entirely if static) with the same signature but perhaps
+     * different properties.
+     *
+     * Invoke within a try/catch block, invoking the provided setup/cleanup instructions.
+     * As designed these must not assign any local variables, and the catch block must be exception free.
+     */
+    public static void generateTryFinallyProxyCall(MethodVisitor visitor, String calleeClassName, String calleeMethodName, String descriptor, int access, boolean isInstanceMethod, boolean isInterface,
+                                                   List<AbstractInsnNode> setup, List<AbstractInsnNode> cleanup)
+    {
+        Label startMethod = new Label(), startTry = new Label(), endTry = new Label(), startCatch = new Label(), endMethod = new Label();
+        visitor.visitLabel(startMethod);
+        visitor.visitTryCatchBlock(startTry, endTry, startCatch, null);
+        setup.forEach(i -> i.accept(visitor));
+        visitor.visitLabel(startTry);
+        long sizesAndReturnCode = visitProxyCall(visitor, calleeClassName, calleeMethodName, descriptor, access, isInstanceMethod, isInterface);
+        int returnCode = (int)(sizesAndReturnCode >>> 56) & 0xff;
+        visitor.visitLabel(endTry);
+        cleanup.forEach(i -> i.accept(visitor));
+        visitor.visitInsn(returnCode);
+        visitor.visitLabel(startCatch);
+        visitor.visitFrame(F_SAME1, 0, null, 1, new Object[] { "java/lang/Throwable" });
+        cleanup.forEach(i -> i.accept(visitor));
+        visitor.visitInsn(ATHROW);
+        visitor.visitLabel(endMethod);
+        if (isInstanceMethod)
+            visitor.visitLocalVariable("this", "L" + calleeClassName + ';', null, startMethod, endMethod, 0);
+        visitor.visitMaxs((int)(sizesAndReturnCode >>> 28) & 0xfffffff, (int)(sizesAndReturnCode & 0xfffffff));
+        visitor.visitEnd();
+    }
+
+    public static AnnotationVisitor checkForSimulationAnnotations(int api, String descriptor, AnnotationVisitor wrap, BiConsumer<Flag, Boolean> annotations)
+    {
+        if (!descriptor.equals("Lorg/apache/cassandra/utils/Simulate;"))
+            return wrap;
+
+        return new AnnotationVisitor(api, wrap)
+        {
+            @Override
+            public AnnotationVisitor visitArray(String name)
+            {
+                if (!name.equals("with") && !name.equals("without"))
+                    return super.visitArray(name);
+
+                boolean add = name.equals("with");
+                return new AnnotationVisitor(api, super.visitArray(name))
+                {
+                    @Override
+                    public void visitEnum(String name, String descriptor, String value)
+                    {
+                        super.visitEnum(name, descriptor, value);
+                        if (descriptor.equals("Lorg/apache/cassandra/utils/Simulate$With;"))
+                            annotations.accept(Flag.valueOf(value), add);
+                    }
+                };
+            }
+        };
+    }
+
+    public static MethodVisitor deterministicToString(MethodVisitor wrap)
+    {
+        return new MethodVisitor(BYTECODE_VERSION, wrap)
+        {
+            @Override
+            public void visitMethodInsn(int opcode, String owner, String name, String descriptor, boolean isInterface)
+            {
+                if (name.equals("hashCode") && owner.equals("java/lang/Object"))
+                {
+                    super.visitMethodInsn(INVOKESTATIC, "java/lang/System", "identityHashCode", "(Ljava/lang/Object;)I", false);
+                }
+                else
+                {
+                    super.visitMethodInsn(opcode, owner, name, descriptor, isInterface);
+                }
+            }
+        };
+    }
+
+}
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/package-info.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/package-info.java
new file mode 100644
index 0000000..fe9394b
--- /dev/null
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/package-info.java
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package has no dependencies besides asm.*\.jar (and should perhaps be a separate module)
+ *
+ * This is to permit it to be built as a standlone javaagent.
+ */
+package org.apache.cassandra.simulator.asm;
+
diff --git a/test/simulator/bootstrap/org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods.java b/test/simulator/bootstrap/org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods.java
new file mode 100644
index 0000000..f66a44f
--- /dev/null
+++ b/test/simulator/bootstrap/org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods.java
@@ -0,0 +1,373 @@
+/*
+ * 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.simulator.systems;
+
+import java.lang.reflect.Field;
+import java.security.SecureRandom;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.LockSupport;
+import java.util.function.ToIntFunction;
+
+import sun.misc.Unsafe;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
+/**
+ * A superclass of InterceptorOfGlobalMethods exposing those methods we might want to use for byte-weaving classes
+ * loaded by the system classloader (such as concurrency primitives). Today we byte weave Enum, Object, Random,
+ * ThreadLocalRandom, ConcurrentHashMap (only for determinism) and all of java.util.concurrent.locks (for park/unpark).
+ * See {@link org.apache.cassandra.simulator.asm.InterceptAgent} for more details.
+ */
+@SuppressWarnings("unused")
+public interface InterceptorOfSystemMethods
+{
+    void waitUntil(long deadlineNanos) throws InterruptedException;
+    void sleep(long period, TimeUnit units) throws InterruptedException;
+    void sleepUninterriptibly(long period, TimeUnit units);
+    boolean waitUntil(Object monitor, long deadlineNanos) throws InterruptedException;
+    void wait(Object monitor) throws InterruptedException;
+    void wait(Object monitor, long millis) throws InterruptedException;
+    void wait(Object monitor, long millis, int nanos) throws InterruptedException;
+    void preMonitorEnter(Object object, float chanceOfSwitch);
+    void preMonitorExit(Object object);
+    void notify(Object monitor);
+    void notifyAll(Object monitor);
+    void nemesis(float chance);
+
+    void park();
+    void parkNanos(long nanos);
+    void parkUntil(long millis);
+    void park(Object blocker);
+    void parkNanos(Object blocker, long nanos);
+    void parkUntil(Object blocker, long millis);
+    void unpark(Thread thread);
+
+    long randomSeed();
+    UUID randomUUID();
+
+    @SuppressWarnings("unused")
+    public static class Global
+    {
+        private static InterceptorOfSystemMethods methods = new None();
+        private static ToIntFunction<Object> identityHashCode;
+
+        public static void waitUntil(long deadlineNanos) throws InterruptedException
+        {
+            methods.waitUntil(deadlineNanos);
+        }
+
+        public static void sleep(long millis) throws InterruptedException
+        {
+            sleep(MILLISECONDS, millis);
+        }
+
+        // slipped param order to replace instance method call without other ASM modification
+        public static void sleep(TimeUnit units, long period) throws InterruptedException
+        {
+            methods.sleep(period, units);
+        }
+
+        // to match Guava Uninterruptibles
+        public static void sleepUninterruptibly(long period, TimeUnit units)
+        {
+            methods.sleepUninterriptibly(period, units);
+        }
+
+        public static boolean waitUntil(Object monitor, long deadlineNanos) throws InterruptedException
+        {
+            return methods.waitUntil(monitor, deadlineNanos);
+        }
+
+        public static void wait(Object monitor) throws InterruptedException
+        {
+            methods.wait(monitor);
+        }
+
+        public static void wait(Object monitor, long millis) throws InterruptedException
+        {
+            methods.wait(monitor, millis);
+        }
+
+        @SuppressWarnings("unused")
+        public static Object preMonitorEnter(Object object, float chance)
+        {
+            methods.preMonitorEnter(object, chance);
+            return object;
+        }
+
+        public static Object preMonitorExit(Object object)
+        {
+            methods.preMonitorExit(object);
+            return object;
+        }
+
+        public static void notify(Object monitor)
+        {
+            methods.notify(monitor);
+        }
+
+        public static void notifyAll(Object monitor)
+        {
+            methods.notifyAll(monitor);
+        }
+
+        public static void park()
+        {
+            methods.park();
+        }
+
+        public static void parkNanos(long nanos)
+        {
+            methods.parkNanos(nanos);
+        }
+
+        public static void parkUntil(long millis)
+        {
+            methods.parkUntil(millis);
+        }
+
+        public static void park(Object blocker)
+        {
+            methods.park(blocker);
+        }
+
+        public static void parkNanos(Object blocker, long nanos)
+        {
+            methods.parkNanos(blocker, nanos);
+        }
+
+        public static void parkUntil(Object blocker, long millis)
+        {
+            methods.parkUntil(blocker, millis);
+        }
+
+        public static void unpark(Thread thread)
+        {
+            methods.unpark(thread);
+        }
+        
+        public static void nemesis(float chance)
+        {
+            methods.nemesis(chance);
+        }
+
+        public static int advanceProbe(int probe)
+        {
+            return probe + 1;
+        }
+
+        public static long randomSeed()
+        {
+            return methods.randomSeed();
+        }
+
+        public static UUID randomUUID()
+        {
+            return methods.randomUUID();
+        }
+
+        public static int identityHashCode(Object object)
+        {
+            return identityHashCode.applyAsInt(object);
+        }
+
+        public static Unsafe getUnsafe()
+        {
+            try
+            {
+                Field field = Unsafe.class.getDeclaredField("theUnsafe");
+                field.setAccessible(true);
+                return (Unsafe) field.get(null);
+            }
+            catch (Exception e)
+            {
+                throw new AssertionError(e);
+            }
+        }
+
+        public static void unsafeSet(InterceptorOfSystemMethods methods)
+        {
+            Global.methods = methods;
+        }
+
+        public static void unsafeSet(InterceptorOfSystemMethods methods, ToIntFunction<Object> identityHashCode)
+        {
+            Global.methods = methods;
+            Global.identityHashCode = identityHashCode;
+        }
+    }
+
+    public static class None implements InterceptorOfSystemMethods
+    {
+        @Override
+        public void waitUntil(long deadlineNanos) throws InterruptedException
+        {
+            long waitNanos = System.nanoTime() - deadlineNanos;
+            if (waitNanos > 0)
+                TimeUnit.NANOSECONDS.sleep(waitNanos);
+        }
+
+        @Override
+        public void sleep(long period, TimeUnit units) throws InterruptedException
+        {
+            waitUntil(System.nanoTime() + units.toNanos(period));
+        }
+
+        @Override
+        public void sleepUninterriptibly(long period, TimeUnit units)
+        {
+            long until = System.nanoTime() + units.toNanos(period);
+            boolean isInterrupted = false;
+            while (true)
+            {
+                try
+                {
+                    waitUntil(until);
+                    break;
+                }
+                catch (InterruptedException e)
+                {
+                    isInterrupted = true;
+                }
+            }
+
+            if (isInterrupted)
+                Thread.currentThread().interrupt();
+        }
+
+        @Override
+        public boolean waitUntil(Object monitor, long deadlineNanos) throws InterruptedException
+        {
+            long wait = deadlineNanos - System.nanoTime();
+            if (wait <= 0)
+                return false;
+
+            monitor.wait((wait + 999999) / 1000000);
+            return true;
+        }
+
+        @Override
+        public void wait(Object monitor) throws InterruptedException
+        {
+            monitor.wait();
+        }
+
+        @Override
+        public void wait(Object monitor, long millis) throws InterruptedException
+        {
+            monitor.wait(millis);
+        }
+
+        @Override
+        public void wait(Object monitor, long millis, int nanos) throws InterruptedException
+        {
+            monitor.wait(millis, nanos);
+        }
+
+        @Override
+        public void preMonitorEnter(Object object, float chanceOfSwitch)
+        {
+        }
+
+        @Override
+        public void preMonitorExit(Object object)
+        {
+        }
+
+        @Override
+        public void notify(Object monitor)
+        {
+            monitor.notify();
+        }
+
+        @Override
+        public void notifyAll(Object monitor)
+        {
+            monitor.notifyAll();
+        }
+
+        @Override
+        public void nemesis(float chance)
+        {
+        }
+
+        @Override
+        public void park()
+        {
+            LockSupport.park();
+        }
+
+        @Override
+        public void parkNanos(long nanos)
+        {
+            LockSupport.parkNanos(nanos);
+        }
+
+        @Override
+        public void parkUntil(long millis)
+        {
+            LockSupport.parkUntil(millis);
+        }
+
+        @Override
+        public void park(Object blocker)
+        {
+            LockSupport.park(blocker);
+        }
+
+        @Override
+        public void parkNanos(Object blocker, long nanos)
+        {
+            LockSupport.parkNanos(blocker, nanos);
+        }
+
+        @Override
+        public void parkUntil(Object blocker, long millis)
+        {
+            LockSupport.parkUntil(blocker, millis);
+        }
+
+        @Override
+        public void unpark(Thread thread)
+        {
+            LockSupport.unpark(thread);
+        }
+
+        private static final long SEED_MULTIPLIER = 2862933555777941757L;
+        private static final long SEED_CONSTANT = 0x121d34a;
+        private long nextSeed = 0x10523dfe2L;
+        @Override
+        public synchronized long randomSeed()
+        {
+            long next = nextSeed;
+            nextSeed *= SEED_MULTIPLIER;
+            nextSeed += SEED_CONSTANT;
+            return next;
+        }
+
+        @Override
+        public UUID randomUUID()
+        {
+            return UUID.randomUUID();
+        }
+    }
+
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/Action.java b/test/simulator/main/org/apache/cassandra/simulator/Action.java
new file mode 100644
index 0000000..17cb628
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/Action.java
@@ -0,0 +1,929 @@
+/*
+ * 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.simulator;
+
+import java.io.Serializable;
+import java.util.EnumSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+
+import com.google.common.base.Preconditions;
+
+import io.netty.util.internal.DefaultPriorityQueue;
+import io.netty.util.internal.PriorityQueue;
+import io.netty.util.internal.PriorityQueueNode;
+import org.apache.cassandra.simulator.Ordered.StrictlyOrdered;
+import org.apache.cassandra.simulator.systems.SimulatedTime;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_SIMULATOR_DEBUG;
+import static org.apache.cassandra.simulator.Action.Modifier.*;
+import static org.apache.cassandra.simulator.Action.Modifiers.NONE;
+import static org.apache.cassandra.simulator.Action.Phase.CANCELLED;
+import static org.apache.cassandra.simulator.Action.Phase.CONSEQUENCE;
+import static org.apache.cassandra.simulator.Action.Phase.FINISHED;
+import static org.apache.cassandra.simulator.Action.Phase.INVALIDATED;
+import static org.apache.cassandra.simulator.Action.Phase.NASCENT;
+import static org.apache.cassandra.simulator.Action.Phase.STARTED;
+import static org.apache.cassandra.simulator.Action.Phase.WITHHELD;
+import static org.apache.cassandra.simulator.Action.RegisteredType.CHILD;
+import static org.apache.cassandra.simulator.Action.RegisteredType.LISTENER;
+import static org.apache.cassandra.simulator.ActionListener.Before.DROP;
+import static org.apache.cassandra.simulator.ActionListener.Before.INVALIDATE;
+import static org.apache.cassandra.simulator.ActionListener.Before.EXECUTE;
+import static org.apache.cassandra.simulator.utils.CompactLists.append;
+import static org.apache.cassandra.simulator.utils.CompactLists.remove;
+import static org.apache.cassandra.simulator.utils.CompactLists.safeForEach;
+
+public abstract class Action implements PriorityQueueNode
+{
+    private static final boolean DEBUG = TEST_SIMULATOR_DEBUG.getBoolean();
+
+    public enum Modifier
+    {
+        /**
+         * Never drop an action. Primarily intended to transitively mark an action's descendants as
+         * required to succeed (e.g. in the case of repair and other cluster actions that are brittle)
+         */
+        RELIABLE('r', true),
+
+        /**
+         * Mark an action to be discarded, which may result in some alternative action being undertaken.
+         * This marker is only to ensure correctness, as the simulator will fail if an action is performed
+         * that is marked DROP and RELIABLE.
+         */
+        DROP('f', false),
+
+        // indicates some scheduler delay should be added to the execution time of this action
+        // TODO (feature): support per node scheduler delay to simulate struggling nodes
+        THREAD_SIGNAL('t', false),
+
+        /**
+         * a general purpose mechanism for withholding actions until all other actions have had an opportunity to run
+         * (intended primarily for TIMEOUT+NO_TIMEOUTS, but may be used elsewhere)
+         * this is a very similar feature to Ordered, but it was easier to model this way, as the predecessors are
+         * all other child actions in the entire transitive closure of operations, with the premise that the action
+         * will no longer be valid by the time it has an opportunity run
+         */
+        WITHHOLD((char)0, false),
+
+        // Mark operations as a THREAD_TIMEOUT, and parent operations as forbidding such timeouts (unless all else has failed)
+        NO_TIMEOUTS('n', true, null, true), TIMEOUT('t', false, NO_TIMEOUTS),
+
+        /**
+         * All children of this action should be performed in strict order wrt the parent's consequences
+         * i.e. this is the continuation version of {@link #STRICT_CHILD_ORDER}
+         * this is a bit clunky, but not the end of the world
+         */
+        STRICT_CHILD_OF_PARENT_ORDER('c', false),
+
+        /**
+         * All children of this action should be performed in strict order, which means not only that
+         * they must be performed in the provided order, but all of their consequences must finish before
+         * the next sibling is permitted to run
+         */
+        STRICT_CHILD_ORDER('s', true, null, STRICT_CHILD_OF_PARENT_ORDER),
+
+        /**
+         * InfiniteLoopExecutors, when started, should be treated as detached from the action that happens to start them
+         * so the child action is considered to be orphaned, and not registered or counted against its parent action
+         */
+        ORPHAN('o', false),
+
+        /**
+         * Must be combined with ORPHAN. Unlinks an Action from its direct parent, attaching it as a child of its
+         * grandparent. This is used to support streams of streams.
+         */
+        ORPHAN_TO_GRANDPARENT((char)0, false),
+
+        /**
+         * Recurring tasks, that the schedule may discount when determining if has terminated
+         */
+        STREAM((char)0, false),
+
+        /**
+         * Recurring scheduled tasks, that the schedule should discount when determining if has terminated
+         */
+        DAEMON((char)0, false),
+
+        /**
+         * Informational messages produced for logging only
+         */
+        INFO((char)0, false),
+
+        /**
+         * A thread wakeup action that is ordinarily filtered from logging.
+         */
+        WAKEUP('w', false),
+
+        /**
+         * Show this action in the chain of origin actions
+         */
+        DISPLAY_ORIGIN('d', false);
+
+        final char displayId;
+        final boolean heritable;
+        final Modifier withholdIfPresent;
+        final Modifier inheritIfContinuation;
+
+        private Modifiers asSet;
+
+        Modifier(char displayId, boolean heritable)
+        {
+            this(displayId, heritable, null);
+        }
+
+        Modifier(char displayId, boolean heritable, Modifier withholdIfPresent)
+        {
+            this(displayId, heritable, withholdIfPresent, null);
+        }
+
+        Modifier(char displayId, boolean heritable, Modifier withholdIfPresent, boolean inheritIfContinuation)
+        {
+            this.displayId = displayId;
+            this.heritable = heritable;
+            this.withholdIfPresent = withholdIfPresent;
+            this.inheritIfContinuation = inheritIfContinuation ? this : null;
+        }
+
+        Modifier(char displayId, boolean heritable, Modifier withholdIfPresent, Modifier inheritIfContinuation)
+        {
+            this.displayId = displayId;
+            this.heritable = heritable;
+            this.withholdIfPresent = withholdIfPresent;
+            this.inheritIfContinuation = inheritIfContinuation;
+        }
+
+        Modifiers asSet()
+        {
+            if (asSet == null)
+                asSet = Modifiers.of(this);
+            return asSet;
+        }
+    }
+
+    public static class Modifiers implements Serializable
+    {
+        public static final Modifiers NONE = of();
+        public static final Modifiers INFO = Modifier.INFO.asSet();
+        public static final Modifiers RELIABLE = Modifier.RELIABLE.asSet();
+        public static final Modifiers DROP = Modifier.DROP.asSet();
+        public static final Modifiers STREAM = of(Modifier.STREAM);
+        public static final Modifiers INFINITE_STREAM = of(Modifier.STREAM, DAEMON);
+        public static final Modifiers STREAM_ITEM = of(Modifier.STREAM, ORPHAN, ORPHAN_TO_GRANDPARENT);
+        public static final Modifiers INFINITE_STREAM_ITEM = of(Modifier.STREAM, DAEMON, ORPHAN, ORPHAN_TO_GRANDPARENT);
+
+        public static final Modifiers START_TASK = of(THREAD_SIGNAL);
+        public static final Modifiers START_THREAD = of(THREAD_SIGNAL);
+        public static final Modifiers START_INFINITE_LOOP = of(ORPHAN, THREAD_SIGNAL);
+        public static final Modifiers START_SCHEDULED_TASK = of(THREAD_SIGNAL);
+        public static final Modifiers START_TIMEOUT_TASK = of(Modifier.TIMEOUT, THREAD_SIGNAL);
+        public static final Modifiers START_DAEMON_TASK = of(ORPHAN, Modifier.DAEMON, THREAD_SIGNAL);
+
+        public static final Modifiers WAKE_UP_THREAD = of(THREAD_SIGNAL, WAKEUP);
+
+        public static final Modifiers STRICT = of(STRICT_CHILD_ORDER);
+        public static final Modifiers NO_TIMEOUTS = Modifier.NO_TIMEOUTS.asSet();
+
+        public static final Modifiers RELIABLE_NO_TIMEOUTS = of(Modifier.NO_TIMEOUTS, Modifier.RELIABLE);
+        public static final Modifiers DISPLAY_ORIGIN = of(Modifier.DISPLAY_ORIGIN);
+
+        public static Modifiers of()
+        {
+            return new Modifiers(EnumSet.noneOf(Modifier.class));
+        }
+
+        public static Modifiers of(Modifier first, Modifier ... rest)
+        {
+            return new Modifiers(EnumSet.of(first, rest));
+        }
+
+        final EnumSet<Modifier> contents;
+        Modifiers(EnumSet<Modifier> contents)
+        {
+            this.contents = contents;
+        }
+
+        public Modifiers with(Modifiers add)
+        {
+            if (add == NONE)
+                return this;
+
+            if (this == NONE)
+                return add;
+
+            if (contents.containsAll(add.contents))
+                return this;
+
+            return add(add.contents);
+        }
+
+        public Modifiers with(Modifier add)
+        {
+            if (this == NONE)
+                return add.asSet();
+
+            if (contents.contains(add))
+                return this;
+
+            return add(add.asSet().contents);
+        }
+
+        private Modifiers add(EnumSet<Modifier> add)
+        {
+            EnumSet<Modifier> merge = EnumSet.noneOf(Modifier.class);
+            for (Modifier modifier : this.contents) add(modifier, merge, add);
+            for (Modifier modifier : add) add(modifier, merge, this.contents);
+            return new Modifiers(merge);
+        }
+
+        private static void add(Modifier modifier, EnumSet<Modifier> to, EnumSet<Modifier> mergingWith)
+        {
+            if (modifier.withholdIfPresent != null && mergingWith.contains(modifier.withholdIfPresent))
+                to.add(WITHHOLD);
+            to.add(modifier);
+        }
+
+        // for continuations to inherit the relevant modifiers from their immediate parent
+        // (since we represent a continuation of the same execution)
+        public Modifiers inheritIfContinuation(Modifiers inheritIfContinuation)
+        {
+            EnumSet<Modifier> merge = null;
+            for (Modifier modifier : inheritIfContinuation.contents)
+            {
+                if (modifier.inheritIfContinuation != null)
+                {
+                    if (merge == null && !contents.contains(modifier.inheritIfContinuation)) merge = EnumSet.copyOf(contents);
+                    if (merge != null) merge.add(modifier.inheritIfContinuation);
+                }
+            }
+
+            if (merge == null)
+                return this;
+
+            if (!merge.contains(WITHHOLD))
+            {
+                for (Modifier modifier : merge)
+                {
+                    if (modifier.withholdIfPresent != null && merge.contains(modifier.withholdIfPresent))
+                        merge.add(WITHHOLD);
+                }
+            }
+            return new Modifiers(merge);
+        }
+
+        public Modifiers without(Modifier modifier)
+        {
+            if (!contents.contains(modifier))
+                return this;
+
+            EnumSet<Modifier> remove = EnumSet.noneOf(Modifier.class);
+            remove.addAll(this.contents);
+            remove.remove(modifier);
+            return new Modifiers(remove);
+        }
+
+        public boolean is(Modifier modifier)
+        {
+            return contents.contains(modifier);
+        }
+    }
+
+    enum RegisteredType { LISTENER, CHILD }
+
+    enum Phase
+    {
+        NASCENT,
+        WITHHELD,
+        CONSEQUENCE,
+        READY_TO_SCHEDULE,
+        SEQUENCED_PRE_SCHEDULED,
+        SCHEDULED,
+        SEQUENCED_POST_SCHEDULED,
+        RUNNABLE,
+        STARTED,
+        FINISHED,
+        CANCELLED,
+        INVALIDATED
+    }
+
+    // configuration/status
+    private final Object description;
+    private OrderOns orderOn;
+    private Modifiers self, transitive;
+    private Phase phase = NASCENT;
+    Ordered ordered;
+
+    /** The listeners (and, if DEBUG, children) we have already registered */
+    private final Map<Object, RegisteredType> registered = new IdentityHashMap<>(2);
+
+    /** The list of listeners (for deterministic evaluation order) to notify on any event */
+    private List<ActionListener> listeners;
+
+    /** The immediate parent, and furthest ancestor of this Action */
+    protected Action parent, origin = this;
+
+    /** The number of direct consequences of this action that have not <i>transitively</i> terminated */
+    private int childCount;
+
+    /**
+     * Consequences marked WITHHOLD are kept in their parent (or parent thread's) {@code withheld} queue until all
+     * other immediate children have <i>transitively</i> terminated their execution
+     */
+    private DefaultPriorityQueue<Action> withheld;
+
+    // scheduler and scheduled state
+    protected RunnableActionScheduler scheduler;
+
+    private long deadline; // some actions have an associated wall clock time to execute and are first scheduled by this
+    private double priority; // all actions eventually get prioritised for execution in some order "now"
+
+    // used to track the index and priority queue we're being managed for execution by (either by scheduledAt or priority)
+    private PriorityQueue<?> scheduledIn;
+    private int scheduledIndex = -1;
+
+    // used to track the scheduledAt of events we have moved to actively scheduling/prioritising
+    private PriorityQueue<?> savedIn;
+    private int savedIndex = -1;
+
+    public Action(Object description, Modifiers self)
+    {
+        this(description, self, NONE);
+    }
+    public Action(Object description, Modifiers self, Modifiers transitive)
+    {
+        this(description, OrderOn.NONE, self, transitive);
+    }
+
+    public Action(Object description, OrderOns orderOn, Modifiers self, Modifiers transitive)
+    {
+        this.description = description;
+        if (orderOn == null || self == null || transitive == null)
+            throw new IllegalArgumentException();
+        assert transitive.contents.stream().allMatch(m -> m.heritable) : transitive.contents.toString();
+        this.orderOn = orderOn;
+        this.self = self;
+        this.transitive = transitive;
+    }
+
+    public Object description()
+    {
+        return description;
+    }
+    public OrderOns orderOns() { return orderOn; }
+    public Phase phase() { return phase; }
+    public Modifiers self() { return self; }
+    public Modifiers transitive() { return transitive; }
+    public boolean is(Modifier modifier)
+    {
+        return self.contents.contains(modifier);
+    }
+    public void inherit(Modifiers add)
+    {
+        if (add != NONE)
+            add(add, add);
+    }
+    public void add(Modifiers self, Modifiers children)
+    {
+        this.self = this.self.with(self);
+        this.transitive = this.transitive.with(children);
+    }
+
+    public boolean isStarted()
+    {
+        return phase.compareTo(STARTED) >= 0;
+    }
+    public boolean isFinished()
+    {
+        return phase.compareTo(FINISHED) >= 0;
+    }
+    public boolean isInvalidated()
+    {
+        return phase.compareTo(INVALIDATED) >= 0;
+    }
+
+    public Action parent()
+    {
+        return parent;
+    }
+    public int childCount()
+    {
+        return childCount;
+    }
+
+    /**
+     * Main implementation of {@link #perform()}, that must be completed by an extending classes.
+     *
+     * Does not need to handle consequences, registration etc.
+     *
+     * @return the action consequences
+     */
+    protected abstract ActionList performSimple();
+
+    /**
+     * Invokes {@link #performSimple} before invoking {@link #performed}.
+     *
+     * May be overridden by an extending classes that does not finish immediately (e.g, SimulatedAction).
+     *
+     * MUST handle consequences, registration etc by invoking performed() on its results before returning,
+     * to register children and record the action's state
+     *
+     * @return the action consequences
+     */
+    protected ActionList performAndRegister()
+    {
+        return performed(performSimple(), true, true);
+    }
+
+    /**
+     * Invoke the action, and return its consequences, i.e. any follow up actions.
+     */
+    public final ActionList perform()
+    {
+        Preconditions.checkState(!(is(RELIABLE) && is(Modifier.DROP)));
+        Throwable fail = safeForEach(listeners, ActionListener::before, this, is(Modifier.DROP) ? DROP : EXECUTE);
+        if (fail != null)
+        {
+            invalidate(false);
+            Throwables.maybeFail(fail);
+        }
+
+        if (DEBUG && parent != null && parent.registered.get(this) != CHILD) throw new AssertionError();
+
+        ActionList next = performAndRegister();
+        next.forEach(Action::setConsequence);
+
+        if (is(STRICT_CHILD_ORDER)) next.setStrictlySequentialOn(this);
+        else if (is(STRICT_CHILD_OF_PARENT_ORDER)) next.setStrictlySequentialOn(parent);
+
+        return next;
+    }
+
+    /**
+     * To be invoked on the results of {@link #performSimple()} by its implementations.
+     * We invite the implementation to invoke it so that it may control state either side of its invocation.
+     *
+     * {@link #register(ActionList)}'s the consequences, restores any old withheld actions,
+     * and updates this Action's internal state.
+     *
+     * @return the provided actions, minus any withheld
+     */
+    protected ActionList performed(ActionList consequences, boolean isStart, boolean isFinish)
+    {
+        assert isStarted() != isStart;
+        assert !isFinished();
+
+        consequences = register(consequences);
+        assert !consequences.anyMatch(c -> c.is(WITHHOLD));
+
+        if (isFinish) finishedSelf();
+        else if (isStart) phase = STARTED;
+
+        return restoreWithheld(consequences);
+    }
+
+    /**
+     * Similar to cancel() but invoked under abnormal termination
+     */
+    public void invalidate()
+    {
+        invalidate(INVALIDATED);
+    }
+
+    /**
+     * To be invoked when this action has become redundant.
+     *  - Marks itself invalidated
+     *  - Notifies its listeners (which may remove it from any ordered sequences in the ActionSchedule)
+     *  - If present, clears itself directly from:
+     *    - its parent's withholding space
+     *    - the schedule's priority queue
+     */
+    public void cancel()
+    {
+        assert !isStarted();
+        invalidate(CANCELLED);
+    }
+
+    private void invalidate(Phase advanceTo)
+    {
+        if (phase.compareTo(CANCELLED) >= 0)
+            return;
+
+        advanceTo(advanceTo);
+        Throwable fail = safeForEach(listeners, ActionListener::before, this, INVALIDATE);
+        fail = Throwables.merge(fail, safeInvalidate(phase == CANCELLED));
+        invalidate(phase == CANCELLED);
+        finishedSelf();
+        Throwables.maybeFail(fail);
+    }
+
+    protected Throwable safeInvalidate(boolean isCancellation)
+    {
+        return null;
+    }
+
+    private void invalidate(boolean isCancellation)
+    {
+        if (parent != null && parent.withheld != null && is(WITHHOLD))
+        {
+            if (parent.withheld.remove(this))
+                parent.cleanupWithheld();
+        }
+        if (scheduledIndex >= 0) scheduledIn.remove(this);
+        if (savedIndex >= 0) savedIn.remove(this);
+        if (ordered != null) ordered.invalidate(isCancellation);
+    }
+
+    /**
+     * Register consequences of this action, i.e.:
+     *  - attach a scheduler to them for prioritising when they are permitted to execute
+     *  - pass them to any listeners as consequences
+     *  - count them as children, and mark ourselves as parent, so that we may track transitive completion
+     *  - withhold any actions that are so marked, to be {@link #restoreWithheld}d once we have transitively completed
+     *    all non-withheld actions.
+     */
+    protected ActionList register(ActionList consequences)
+    {
+        assert !isFinished();
+        if (consequences.isEmpty())
+            return consequences;
+
+        scheduler.attachTo(consequences);
+        Throwable fail = safeForEach(listeners, ActionListener::consequences, consequences);
+        if (fail != null)
+        {
+            invalidate(false);
+            Throwables.merge(fail, consequences.safeForEach(Action::invalidate));
+            Throwables.maybeFail(fail);
+        }
+
+        boolean withhold = false;
+        int orphanCount = 0;
+        for (int i = 0 ; i < consequences.size() ; ++i)
+        {
+            Action child = consequences.get(i);
+            if (child.is(ORPHAN))
+            {
+                Preconditions.checkState(!child.is(WITHHOLD));
+                ++orphanCount;
+                if (parent != null && child.is(ORPHAN_TO_GRANDPARENT))
+                {
+                    ++parent.childCount;
+                    parent.registerChild(child);
+                }
+            }
+            else
+            {
+                child.inherit(transitive);
+                if (child.is(WITHHOLD))
+                {
+                    // this could be supported in principle by applying the ordering here, but it would be
+                    // some work to ensure it doesn't lead to deadlocks so for now just assert we don't use it
+                    Preconditions.checkState(!is(STRICT_CHILD_ORDER) && !is(STRICT_CHILD_OF_PARENT_ORDER));
+                    withhold = true;
+                    addWithheld(child);
+                }
+
+                registerChild(child);
+            }
+        }
+
+        int addChildCount = consequences.size() - orphanCount;
+        childCount += addChildCount;
+
+        if (!withhold)
+            return consequences;
+
+        return consequences.filter(child -> !child.is(WITHHOLD));
+    }
+
+    // setup the child relationship, but do not update childCount
+    private void registerChild(Action child)
+    {
+        assert child.parent == null;
+        child.parent = this;
+        if (is(Modifier.DISPLAY_ORIGIN)) child.origin = this;
+        else if (origin != this) child.origin = origin;
+        if (DEBUG && !register(child, CHILD)) throw new AssertionError();
+    }
+
+    private boolean register(Object object, RegisteredType type)
+    {
+        RegisteredType prev = registered.putIfAbsent(object, type);
+        if (prev != null && prev != type)
+            throw new AssertionError();
+        return prev == null;
+    }
+
+    public void register(ActionListener listener)
+    {
+        if (register(listener, LISTENER))
+            listeners = append(listeners, listener);
+    }
+
+    private boolean deregister(Object object, RegisteredType type)
+    {
+        return registered.remove(object, type);
+    }
+
+    public void deregister(ActionListener listener)
+    {
+        if (deregister(listener, LISTENER))
+            listeners = remove(listeners, listener);
+    }
+
+    private void addWithheld(Action action)
+    {
+        if (withheld == null)
+            withheld = new DefaultPriorityQueue<>(Action::compareByPriority, 2);
+        action.advanceTo(WITHHELD);
+        action.saveIn(withheld);
+    }
+
+    /**
+     * Restore withheld (by ourselves or a parent) actions, when no other outstanding actions remain
+     */
+    public ActionList restoreWithheld(ActionList consequences)
+    {
+        if (withheld != null && childCount == withheld.size())
+        {
+            Action next = withheld.poll();
+            cleanupWithheld();
+            consequences = consequences.andThen(next);
+        }
+        else if (childCount == 0 && parent != null)
+        {
+            Action cur = parent;
+            while (cur.childCount == 0 && cur.parent != null)
+                cur = cur.parent;
+            consequences = cur.restoreWithheld(consequences);
+        }
+        return consequences;
+    }
+
+    private void cleanupWithheld()
+    {
+        Action cur = this;
+        if (cur.withheld.isEmpty())
+            cur.withheld = null;
+    }
+
+    /**
+     * Invoked once we finish executing ourselves. Typically this occurs immediately after invocation,
+     * but for SimulatedAction it occurs only once the thread terminates its execution.
+     *
+     * In practice this is entirely determined by the {@code isFinished} parameter supplied to
+     * {@link #performed(ActionList, boolean, boolean)}.
+     */
+    void finishedSelf()
+    {
+        if (phase.compareTo(CANCELLED) < 0)
+            advanceTo(FINISHED);
+
+        scheduler = null;
+        if (withheld != null)
+        {
+            Queue<Action> withheld = this.withheld;
+            this.withheld = null;
+            withheld.forEach(Action::cancel);
+        }
+        Throwable fail = safeForEach(listeners, ActionListener::after, this);
+        if (childCount == 0)
+            fail = Throwables.merge(fail, transitivelyFinished());
+
+        if (fail != null)
+        {
+            invalidate(false);
+            Throwables.maybeFail(fail);
+        }
+    }
+
+    /**
+     * Invoked once all of the consequences of this action, and of those actions (recursively) have completed.
+     */
+    Throwable transitivelyFinished()
+    {
+        return transitivelyFinished(this);
+    }
+
+    static Throwable transitivelyFinished(Action cur)
+    {
+        Throwable fail = null;
+        while (true)
+        {
+            Action parent = cur.parent;
+            assert 0 == cur.childCount && cur.isFinished();
+            if (DEBUG && cur.registered.values().stream().anyMatch(t -> t == CHILD)) throw new AssertionError();
+            fail = Throwables.merge(fail, safeForEach(cur.listeners, ActionListener::transitivelyAfter, cur));
+            if (parent == null)
+                break;
+            if (DEBUG && CHILD != parent.registered.remove(cur)) throw new AssertionError();
+            if (--parent.childCount == 0 && parent.isFinished()) cur = parent;
+            else break;
+        }
+        return fail;
+    }
+
+    void orderOn(OrderOn orderOn)
+    {
+        this.orderOn = this.orderOn.with(orderOn);
+    }
+
+    void setupOrdering(ActionSchedule schedule)
+    {
+        if (orderOn.isOrdered())
+        {
+            ordered = orderOn.isStrict() ? new StrictlyOrdered(this, schedule) : new Ordered(this, schedule);
+            for (int i = 0, maxi = orderOn.size(); i < maxi ; ++i)
+                ordered.join(orderOn.get(i));
+        }
+    }
+
+    void advanceTo(Phase phase)
+    {
+        Preconditions.checkArgument(phase.compareTo(this.phase) > 0);
+        this.phase = phase;
+    }
+
+    void addTo(PriorityQueue<Action> byDeadline)
+    {
+        Preconditions.checkState(scheduledIndex < 0);
+        scheduledIn = byDeadline;
+        byDeadline.add(this);
+    }
+
+    void saveIn(PriorityQueue<Action> saveIn)
+    {
+        Preconditions.checkState(savedIndex < 0);
+        savedIn = saveIn;
+        saveIn.add(this);
+    }
+
+    void setScheduler(RunnableActionScheduler scheduler)
+    {
+        Preconditions.checkState(this.scheduler == null);
+        Preconditions.checkState(this.phase == NASCENT);
+        this.scheduler = scheduler;
+    }
+
+    void setConsequence()
+    {
+        advanceTo(CONSEQUENCE);
+    }
+
+    void schedule(SimulatedTime time, FutureActionScheduler future)
+    {
+        setPriority(scheduler.priority());
+        if (deadline == 0) deadline = time.nanoTime();
+        if (is(THREAD_SIGNAL))
+            deadline += future.schedulerDelayNanos();
+    }
+
+    public void setDeadline(long deadlineNanos)
+    {
+        Preconditions.checkState(deadline == 0);
+        Preconditions.checkArgument(deadlineNanos >= deadline);
+        deadline = deadlineNanos;
+    }
+
+    void setPriority(double priority)
+    {
+        this.priority = priority;
+    }
+
+    public long deadline()
+    {
+        if (deadline < 0) throw new AssertionError();
+        return deadline;
+    }
+
+    public double priority()
+    {
+        return priority;
+    }
+
+    @Override
+    public int priorityQueueIndex(DefaultPriorityQueue<?> queue)
+    {
+        if (queue == scheduledIn) return scheduledIndex;
+        else if (queue == savedIn) return savedIndex;
+        else return -1;
+    }
+
+    @Override
+    public void priorityQueueIndex(DefaultPriorityQueue<?> queue, int i)
+    {
+        if (queue == scheduledIn) { scheduledIndex = i; if (i < 0) scheduledIn = null; }
+        else if (queue == savedIn) { savedIndex = i; if (i < 0) savedIn = null; }
+        else throw new IllegalStateException();
+    }
+
+    public int compareByDeadline(Action that)
+    {
+        return Long.compare(this.deadline, that.deadline);
+    }
+
+    public int compareByPriority(Action that)
+    {
+        return Double.compare(this.priority, that.priority);
+    }
+
+    private String describeModifiers()
+    {
+        StringBuilder builder = new StringBuilder("[");
+        for (Modifier modifier : self.contents)
+        {
+            if (modifier.displayId == 0)
+                continue;
+
+            if (!transitive.is(modifier)) builder.append(modifier.displayId);
+            else builder.append(Character.toUpperCase(modifier.displayId));
+        }
+
+        boolean hasTransitiveOnly = false;
+        for (Modifier modifier : transitive.contents)
+        {
+            if (modifier.displayId == 0)
+                continue;
+
+            if (!self.is(modifier))
+            {
+                if (!hasTransitiveOnly)
+                {
+                    hasTransitiveOnly = true;
+                    builder.append('(');
+                }
+                builder.append(modifier.displayId);
+            }
+        }
+
+        if (builder.length() == 1)
+            return "";
+
+        if (hasTransitiveOnly)
+            builder.append(')');
+        builder.append(']');
+
+        return builder.toString();
+    }
+
+    public String toString()
+    {
+        return describeModifiers() + description() + (origin != this ? " for " + origin : "");
+    }
+
+    public String describeCurrentState()
+    {
+        return describeCurrentState(new StringBuilder(), "").toString();
+    }
+
+    private StringBuilder describeCurrentState(StringBuilder sb, String prefix)
+    {
+        if (!prefix.isEmpty())
+        {
+            sb.append(prefix);
+        }
+        if (!isStarted()) sb.append("NOT_STARTED ");
+        else if (!isFinished()) sb.append("NOT_FINISHED ");
+        if (childCount > 0)
+        {
+            sb.append('(');
+            sb.append(childCount);
+            sb.append(") ");
+        }
+        if (orderOn.isOrdered())
+        {
+            sb.append(orderOn);
+            sb.append(": ");
+        }
+        sb.append(description());
+        registered.entrySet().stream()
+                  .filter(e -> e.getValue() == CHILD)
+                  .map(e -> (Action) e.getKey())
+                  .forEach(a -> {
+                      sb.append('\n');
+                      a.describeCurrentState(sb, prefix + "   |");
+                  });
+
+        return sb;
+    }
+
+}
\ No newline at end of file
diff --git a/test/simulator/main/org/apache/cassandra/simulator/ActionList.java b/test/simulator/main/org/apache/cassandra/simulator/ActionList.java
new file mode 100644
index 0000000..5052e43
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/ActionList.java
@@ -0,0 +1,142 @@
+/*
+ * 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.simulator;
+
+import java.util.AbstractCollection;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.simulator.OrderOn.StrictSequential;
+import org.apache.cassandra.utils.Throwables;
+
+import static java.util.Arrays.copyOf;
+
+public class ActionList extends AbstractCollection<Action>
+{
+    private static final ActionList EMPTY = new ActionList(new Action[0]);
+    public static ActionList empty() { return EMPTY; }
+    public static ActionList of(Action action) { return new ActionList(new Action[] { action }); }
+    public static ActionList of(Stream<Action> action) { return new ActionList(action.toArray(Action[]::new)); }
+    public static ActionList of(Collection<Action> actions) { return actions.isEmpty() ? EMPTY : new ActionList(actions.toArray(new Action[0])); }
+    public static ActionList of(Action ... actions) { return new ActionList(actions); }
+
+    private final Action[] actions;
+
+    ActionList(Action[] actions)
+    {
+        this.actions = actions;
+    }
+
+    public int size()
+    {
+        return actions.length;
+    }
+
+    public boolean isEmpty()
+    {
+        return 0 == actions.length;
+    }
+
+    public Action get(int i)
+    {
+        return actions[i];
+    }
+
+    public Iterator<Action> iterator()
+    {
+        return Iterators.forArray(actions);
+    }
+
+    public Stream<Action> stream()
+    {
+        return Stream.of(actions);
+    }
+
+    public ActionList transform(Function<Action, Action> apply)
+    {
+        return ActionList.of(stream().map(apply));
+    }
+
+    public ActionList filter(Predicate<Action> apply)
+    {
+        return ActionList.of(stream().filter(apply));
+    }
+
+    public boolean anyMatch(Predicate<Action> test)
+    {
+        for (int i = 0 ; i < actions.length ; ++i)
+            if (test.test(actions[i])) return true;
+        return false;
+    }
+
+    public ActionList andThen(Action andThen)
+    {
+        return andThen(ActionList.of(andThen));
+    }
+
+    public ActionList andThen(ActionList andThen)
+    {
+        Action[] result = copyOf(actions, size() + andThen.size());
+        System.arraycopy(andThen.actions, 0, result, size(), andThen.size());
+        return new ActionList(result);
+    }
+
+    public ActionList setStrictlySequential()
+    {
+        return setStrictlySequentialOn(this);
+    }
+
+    public ActionList setStrictlySequentialOn(Object on)
+    {
+        if (isEmpty()) return this;
+        StrictSequential orderOn = new StrictSequential(on);
+        forEach(a -> a.orderOn(orderOn));
+        return this;
+    }
+
+    public Throwable safeForEach(Consumer<Action> forEach)
+    {
+        Throwable result = null;
+        for (Action action : actions)
+        {
+            try
+            {
+                forEach.accept(action);
+            }
+            catch (Throwable t)
+            {
+                result = Throwables.merge(result, t);
+            }
+        }
+        return result;
+    }
+
+    public String toString()
+    {
+        return Arrays.toString(actions);
+    }
+}
+
diff --git a/test/simulator/main/org/apache/cassandra/simulator/ActionListener.java b/test/simulator/main/org/apache/cassandra/simulator/ActionListener.java
new file mode 100644
index 0000000..2bc8ae8
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/ActionListener.java
@@ -0,0 +1,201 @@
+/*
+ * 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.simulator;
+
+import java.util.List;
+import java.util.function.Consumer;
+
+public interface ActionListener
+{
+    enum Before { EXECUTE, DROP, INVALIDATE }
+
+    /**
+     * Immediately before the action is first executed
+     * @param action the action we are about to perform
+     * @param before if the action is to be performed (rather than dropped)
+     */
+    default void before(Action action, Before before) {}
+
+    /**
+     * Immediately after the action is first executed (or dropped)
+     * @param consequences the actions that result from the execution
+     */
+    default void consequences(ActionList consequences) {}
+
+    /**
+     * If an ActionThread, after termination; otherwise immediately after invoked
+     * @param finished the action that has finished
+     */
+    default void after(Action finished) {}
+
+    /**
+     * After the action and all its consequent terminate (excluding the initiation of an infinite loop execution)
+     * @param finished the action that has finished
+     */
+    default void transitivelyAfter(Action finished) {}
+
+    static ActionListener runAfter(Consumer<Action> after)
+    {
+        return new ActionListener()
+        {
+            @Override
+            public void after(Action performed)
+            {
+                after.accept(performed);
+            }
+        };
+    }
+
+    static ActionListener runAfterAndTransitivelyAfter(Consumer<Action> after)
+    {
+        return new ActionListener()
+        {
+            @Override
+            public void after(Action performed)
+            {
+                after.accept(performed);
+            }
+
+            @Override
+            public void transitivelyAfter(Action performed)
+            {
+                after.accept(performed);
+            }
+        };
+    }
+
+    static ActionListener runAfterTransitiveClosure(Consumer<Action> transitivelyAfter)
+    {
+        return new ActionListener()
+        {
+            @Override
+            public void transitivelyAfter(Action performed)
+            {
+                transitivelyAfter.accept(performed);
+            }
+        };
+    }
+
+    static ActionListener recursive(ActionListener runOnAll)
+    {
+        return new WrappedRecursiveActionListener(runOnAll);
+    }
+
+    public interface SelfAddingActionListener extends ActionListener, Consumer<Action>
+    {
+        @Override
+        default public void accept(Action action)
+        {
+            action.register(this);
+        }
+    }
+
+    public static class RecursiveActionListener implements SelfAddingActionListener
+    {
+        @Override
+        public void consequences(ActionList consequences)
+        {
+            consequences.forEach(this);
+        }
+    }
+
+    public static class WrappedRecursiveActionListener extends Wrapped implements SelfAddingActionListener
+    {
+        public WrappedRecursiveActionListener(ActionListener wrap)
+        {
+            super(wrap);
+        }
+
+        @Override
+        public void consequences(ActionList consequences)
+        {
+            consequences.forEach(this);
+            super.consequences(consequences);
+        }
+    }
+
+    public static class Wrapped implements ActionListener
+    {
+        final ActionListener wrap;
+
+        public Wrapped(ActionListener wrap)
+        {
+            this.wrap = wrap;
+        }
+
+        @Override
+        public void before(Action action, Before before)
+        {
+            wrap.before(action, before);
+        }
+
+        @Override
+        public void consequences(ActionList consequences)
+        {
+            wrap.consequences(consequences);
+        }
+
+        @Override
+        public void after(Action finished)
+        {
+            wrap.after(finished);
+        }
+
+        @Override
+        public void transitivelyAfter(Action finished)
+        {
+            wrap.transitivelyAfter(finished);
+        }
+    }
+
+    public static class Combined implements ActionListener
+    {
+        final List<ActionListener> combined;
+
+        public Combined(List<ActionListener> combined)
+        {
+            this.combined = combined;
+        }
+
+        @Override
+        public void before(Action action, Before before)
+        {
+            combined.forEach(listener -> listener.before(action, before));
+        }
+
+        @Override
+        public void consequences(ActionList consequences)
+        {
+            combined.forEach(listener -> listener.consequences(consequences));
+        }
+
+        @Override
+        public void after(Action finished)
+        {
+            combined.forEach(listener -> listener.after(finished));
+        }
+
+        @Override
+        public void transitivelyAfter(Action finished)
+        {
+            combined.forEach(listener -> listener.transitivelyAfter(finished));
+        }
+    }
+}
+
diff --git a/test/simulator/main/org/apache/cassandra/simulator/ActionPlan.java b/test/simulator/main/org/apache/cassandra/simulator/ActionPlan.java
new file mode 100644
index 0000000..b9e9d2d
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/ActionPlan.java
@@ -0,0 +1,91 @@
+/*
+ * 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.simulator;
+
+import java.util.List;
+import java.util.function.LongSupplier;
+
+import com.google.common.collect.ImmutableList;
+
+import org.apache.cassandra.simulator.ActionSchedule.Work;
+import org.apache.cassandra.simulator.systems.SimulatedTime;
+import org.apache.cassandra.utils.CloseableIterator;
+
+import static java.util.Collections.emptyList;
+import static java.util.Collections.singletonList;
+import static org.apache.cassandra.simulator.ActionSchedule.Mode.STREAM_LIMITED;
+import static org.apache.cassandra.simulator.ActionSchedule.Mode.TIME_LIMITED;
+import static org.apache.cassandra.simulator.ActionSchedule.Mode.UNLIMITED;
+
+public class ActionPlan
+{
+    /**
+     * Actions to perform (reliably, and in strict order) before starting the proper simulation
+     */
+    public final ActionList pre;
+
+    /**
+     * List of action sequences, each representing the actions planned by a given actor, that will
+     * be performed in the provided sequence but otherwise randomly interleaved with the other action sequences.
+     * These planned actions may initiate other actions, that will all complete before the next planned action
+     * for that action sequence is started.
+     */
+    public final List<ActionList> interleave;
+
+    /**
+     * Actions to perform (reliably, and in strict order) after finishing the proper simulation.
+     *
+     * This is only run if the simulation was successful, so no cleanup should be performed here unless optional.
+     */
+    public final ActionList post;
+
+    public ActionPlan(ActionList pre, List<ActionList> interleave, ActionList post)
+    {
+        this.pre = pre;
+        this.interleave = interleave;
+        this.post = post;
+    }
+
+    public CloseableIterator<?> iterator(long runForNanos, LongSupplier schedulerJitter, SimulatedTime time, RunnableActionScheduler preAndPostScheduler, RunnableActionScheduler mainScheduler, FutureActionScheduler futureScheduler)
+    {
+        return new ActionSchedule(time, futureScheduler, schedulerJitter,
+                                  new Work(UNLIMITED, preAndPostScheduler, singletonList(pre.setStrictlySequential())),
+                                  new Work(runForNanos > 0 ? TIME_LIMITED : STREAM_LIMITED, runForNanos, mainScheduler, interleave),
+                                  new Work(UNLIMITED, preAndPostScheduler, singletonList(post.setStrictlySequential())));
+    }
+
+    public static ActionPlan interleave(List<ActionList> interleave)
+    {
+        return new ActionPlan(ActionList.empty(), interleave, ActionList.empty());
+    }
+
+    public static ActionPlan setUpTearDown(ActionList pre, ActionList post)
+    {
+        return new ActionPlan(pre, emptyList(), post);
+    }
+
+    public ActionPlan encapsulate(ActionPlan that)
+    {
+        return new ActionPlan(
+                this.pre.andThen(that.pre),
+                ImmutableList.<ActionList>builder().addAll(this.interleave).addAll(that.interleave).build(),
+                that.post.andThen(this.post));
+    }
+}
+
diff --git a/test/simulator/main/org/apache/cassandra/simulator/ActionSchedule.java b/test/simulator/main/org/apache/cassandra/simulator/ActionSchedule.java
new file mode 100644
index 0000000..59d0180
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/ActionSchedule.java
@@ -0,0 +1,390 @@
+/*
+ * 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.simulator;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.function.LongSupplier;
+import java.util.stream.Stream;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.util.internal.DefaultPriorityQueue;
+import io.netty.util.internal.PriorityQueue;
+import org.apache.cassandra.simulator.OrderOn.OrderOnId;
+import org.apache.cassandra.simulator.Ordered.Sequence;
+import org.apache.cassandra.simulator.systems.SimulatedTime;
+import org.apache.cassandra.simulator.utils.SafeCollections;
+import org.apache.cassandra.utils.CloseableIterator;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.simulator.Action.Modifier.DAEMON;
+import static org.apache.cassandra.simulator.Action.Modifier.STREAM;
+import static org.apache.cassandra.simulator.Action.Phase.CONSEQUENCE;
+import static org.apache.cassandra.simulator.Action.Phase.READY_TO_SCHEDULE;
+import static org.apache.cassandra.simulator.Action.Phase.RUNNABLE;
+import static org.apache.cassandra.simulator.Action.Phase.SCHEDULED;
+import static org.apache.cassandra.simulator.Action.Phase.SEQUENCED_POST_SCHEDULED;
+import static org.apache.cassandra.simulator.Action.Phase.SEQUENCED_PRE_SCHEDULED;
+import static org.apache.cassandra.simulator.ActionSchedule.Mode.TIME_LIMITED;
+import static org.apache.cassandra.simulator.ActionSchedule.Mode.UNLIMITED;
+import static org.apache.cassandra.simulator.SimulatorUtils.failWithOOM;
+import static org.apache.cassandra.simulator.SimulatorUtils.dumpStackTraces;
+
+/**
+ * TODO (feature): support total stalls on specific nodes
+ *
+ * This class coordinates the running of actions that have been planned by an ActionPlan, or are the consequences
+ * of actions that have been executed by such a plan. This coordination includes enforcing all {@link OrderOn}
+ * criteria, and running DAEMON (recurring scheduled) tasks.
+ *
+ * Note there is a distinct scheduling mechanism {@link org.apache.cassandra.simulator.Action.Modifier#WITHHOLD}
+ * that is coordinated by an Action and its parent, that is used to prevent certain actions from running unless
+ * all descendants have executed (with the aim of it ordinarily being invalidated before this happens), and this
+ * is not imposed here because it would be more complicated to manage.
+ */
+public class ActionSchedule implements CloseableIterator<Object>
+{
+    private static final Logger logger = LoggerFactory.getLogger(ActionList.class);
+
+    public enum Mode { TIME_LIMITED, STREAM_LIMITED, UNLIMITED }
+
+    public static class Work
+    {
+        final Mode mode;
+        final long runForNanos;
+        final RunnableActionScheduler runnableScheduler;
+        final List<ActionList> actors;
+
+        public Work(Mode mode, RunnableActionScheduler runnableScheduler, List<ActionList> actors)
+        {
+            this(mode, -1, runnableScheduler, actors);
+            Preconditions.checkArgument(mode != TIME_LIMITED);
+        }
+
+        public Work(long runForNanos, RunnableActionScheduler runnableScheduler, List<ActionList> actors)
+        {
+            this(TIME_LIMITED, runForNanos, runnableScheduler, actors);
+            Preconditions.checkArgument(runForNanos > 0);
+        }
+
+        public Work(Mode mode, long runForNanos, RunnableActionScheduler runnableScheduler, List<ActionList> actors)
+        {
+            this.mode = mode;
+            this.runForNanos = runForNanos;
+            this.actors = actors;
+            this.runnableScheduler = runnableScheduler;
+        }
+    }
+
+    final SimulatedTime time;
+    final FutureActionScheduler scheduler;
+    final LongSupplier schedulerJitter; // we will prioritise all actions scheduled to run within this period of the current oldest action
+    // Action flow is:
+    //    perform() -> [withheld]
+    //              -> consequences
+    //              -> [pendingDaemonWave | <invalidate daemon>]
+    //              -> [sequences (if ordered and SEQUENCE_EAGERLY)]
+    //              -> [scheduled]
+    //              -> [sequences (if ordered and !SEQUENCE_EAGERLY)]
+    //              -> runnable + [runnableByScheduledAt]
+    final Map<OrderOn, Sequence> sequences = new HashMap<>();
+    // queue of items that are not yet runnable sorted by deadline
+    final PriorityQueue<Action> scheduled = new DefaultPriorityQueue<>(Action::compareByDeadline, 128);
+    // queue of items that are runnable (i.e. within scheduler jitter of min deadline) sorted by their execution order (i.e. priority)
+    final PriorityQueue<Action> runnable = new DefaultPriorityQueue<>(Action::compareByPriority, 128);
+    // auxillary queue of items that are runnable so that we may track the time span covered by runnable items we are randomising execution of
+    final PriorityQueue<Action> runnableByDeadline = new DefaultPriorityQueue<>(Action::compareByDeadline, 128);
+
+    private Mode mode;
+
+    // if running in TIME_LIMITED mode, stop ALL streams (finite or infinite) and daemon tasks once we pass this point
+    private long runUntilNanos;
+
+    // if running in STREAM_LIMITED mode, stop infinite streams once we have no more finite streams to process
+    private int activeFiniteStreamCount;
+
+    // If running in UNLIMITED mode, release daemons (recurring tasks) in waves,
+    // so we can simplify checking if they're all that's running
+    // TODO (cleanup): we can do better than this, probably most straightforwardly by ensuring daemon actions
+    //                 have a consistent but unique id(), and managing the set of these.
+    private int activeDaemonWaveCount;
+    private int pendingDaemonWaveCountDown;
+    private DefaultPriorityQueue<Action> pendingDaemonWave;
+
+    private final Iterator<Work> moreWork;
+
+    public ActionSchedule(SimulatedTime time, FutureActionScheduler futureScheduler, LongSupplier schedulerJitter, Work ... moreWork)
+    {
+        this(time, futureScheduler, schedulerJitter, Arrays.asList(moreWork).iterator());
+    }
+
+    public ActionSchedule(SimulatedTime time, FutureActionScheduler futureScheduler, LongSupplier schedulerJitter, Iterator<Work> moreWork)
+    {
+        this.time = time;
+        this.scheduler = futureScheduler;
+        this.schedulerJitter = schedulerJitter;
+        this.moreWork = moreWork;
+        moreWork();
+    }
+
+    void add(Action action)
+    {
+        Preconditions.checkState(action.phase() == CONSEQUENCE);
+        action.schedule(time, scheduler);
+        action.setupOrdering(this);
+        if (action.is(STREAM) && !action.is(DAEMON))
+            ++activeFiniteStreamCount;
+
+        switch (mode)
+        {
+            default: throw new AssertionError();
+            case TIME_LIMITED:
+                if (time.nanoTime() >= runUntilNanos && (action.is(DAEMON) || action.is(STREAM)))
+                {
+                    action.cancel();
+                    return;
+                }
+                break;
+            case STREAM_LIMITED:
+                if (activeFiniteStreamCount == 0 && action.is(DAEMON))
+                {
+                    action.cancel();
+                    return;
+                }
+                break;
+            case UNLIMITED:
+                if (action.is(STREAM)) throw new IllegalStateException();
+                if (action.is(DAEMON))
+                {
+                    action.saveIn(pendingDaemonWave);
+                    action.advanceTo(READY_TO_SCHEDULE);
+                    return;
+                }
+                break;
+        }
+        action.advanceTo(READY_TO_SCHEDULE);
+        advance(action);
+    }
+
+    void advance(Action action)
+    {
+        switch (action.phase())
+        {
+            default:
+                throw new AssertionError();
+
+            case CONSEQUENCE:
+                    // this should only happen if we invalidate an Ordered action that tries to
+                    // enqueue one of the actions we are in the middle of scheduling for the first time
+                    return;
+
+            case READY_TO_SCHEDULE:
+                if (action.ordered != null && action.ordered.waitPreScheduled())
+                {
+                    action.advanceTo(SEQUENCED_PRE_SCHEDULED);
+                    return;
+                }
+
+            case SEQUENCED_PRE_SCHEDULED:
+                if (action.deadline() > time.nanoTime())
+                {
+                    action.addTo(scheduled);
+                    action.advanceTo(SCHEDULED);
+                    return;
+                }
+
+            case SCHEDULED:
+                if (action.ordered != null && action.ordered.waitPostScheduled())
+                {
+                    action.advanceTo(SEQUENCED_POST_SCHEDULED);
+                    return;
+                }
+
+            case SEQUENCED_POST_SCHEDULED:
+                action.addTo(runnable);
+                action.saveIn(runnableByDeadline);
+                action.advanceTo(RUNNABLE);
+        }
+    }
+
+    void add(ActionList add)
+    {
+        if (add.isEmpty())
+            return;
+
+        add.forEach(this::add);
+    }
+
+    public boolean hasNext()
+    {
+        if (!runnable.isEmpty() || !scheduled.isEmpty())
+            return true;
+
+        while (moreWork())
+        {
+            if (!runnable.isEmpty() || !scheduled.isEmpty())
+                return true;
+        }
+
+        if (!sequences.isEmpty())
+        {
+            // TODO (feature): detection of which action is blocking progress, and logging of its stack trace only
+            Stream<Action> actions;
+            if (Ordered.DEBUG)
+            {
+                logger.error("Simulation failed to make progress; blocked task graph:");
+                actions = sequences.values()
+                                   .stream()
+                                   .flatMap(s -> Stream.concat(s.maybeRunning.stream(), s.next.stream()))
+                                   .map(o -> o.ordered().action);
+            }
+            else
+            {
+                logger.error("Simulation failed to make progress. Run with assertions enabled to see the blocked task graph. Blocked tasks:");
+                actions = sequences.values()
+                                   .stream()
+                                   .filter(s -> s.on instanceof OrderOnId)
+                                   .map(s -> ((OrderOnId) s.on).id)
+                                   .flatMap(s -> s instanceof ActionList ? ((ActionList) s).stream() : Stream.empty());
+                logger.error("Run with assertions enabled to see the blocked task graph.");
+            }
+
+            actions.filter(Action::isStarted)
+                   .distinct()
+                   .sorted(Comparator.comparingLong(a -> ((long) ((a.isStarted() ? 1 : 0) + (a.isFinished() ? 2 : 0)) << 32) | a.childCount()))
+                   .forEach(a -> logger.error(a.describeCurrentState()));
+
+            logger.error("Thread stack traces:");
+            dumpStackTraces(logger);
+            throw failWithOOM();
+        }
+
+        return false;
+    }
+
+    private boolean moreWork()
+    {
+        if (!moreWork.hasNext())
+            return false;
+
+        Work work = moreWork.next();
+        this.runUntilNanos = work.runForNanos < 0 ? -1 : time.nanoTime() + work.runForNanos;
+        Mode oldMode = mode;
+        mode = work.mode;
+        if (oldMode != work.mode)
+        {
+            if (work.mode == UNLIMITED)
+            {
+                this.pendingDaemonWave = new DefaultPriorityQueue<>(Action::compareByPriority, 128);
+            }
+            else if (oldMode == UNLIMITED)
+            {
+                while (!pendingDaemonWave.isEmpty())
+                    advance(pendingDaemonWave.poll());
+                pendingDaemonWave = null;
+            }
+        }
+        work.actors.forEach(work.runnableScheduler::attachTo);
+        work.actors.forEach(a -> a.forEach(Action::setConsequence));
+        work.actors.forEach(this::add);
+        return true;
+    }
+
+    public Object next()
+    {
+        if (!scheduled.isEmpty())
+        {
+            long scheduleUntil = (runnableByDeadline.isEmpty() ? time.nanoTime() : runnableByDeadline.peek().deadline())
+                                 + schedulerJitter.getAsLong();
+
+            while (!scheduled.isEmpty() && (runnable.isEmpty() || scheduled.peek().deadline() <= scheduleUntil))
+                advance(scheduled.poll());
+        }
+
+        Action perform = runnable.poll();
+        if (perform == null)
+            throw new NoSuchElementException();
+
+        if (!runnableByDeadline.remove(perform) && perform.deadline() > 0)
+            throw new IllegalStateException();
+        time.tick(perform.deadline());
+        maybeScheduleDaemons(perform);
+
+        ActionList consequences = perform.perform();
+        add(consequences);
+        if (perform.is(STREAM) && !perform.is(DAEMON))
+            --activeFiniteStreamCount;
+
+        return Pair.of(perform, consequences);
+    }
+
+    private void maybeScheduleDaemons(Action perform)
+    {
+        if (pendingDaemonWave != null)
+        {
+            if (perform.is(DAEMON) && --activeDaemonWaveCount == 0)
+            {
+                pendingDaemonWaveCountDown = Math.max(128, 16 * (scheduled.size() + pendingDaemonWave.size()));
+            }
+            else if (activeDaemonWaveCount == 0 && --pendingDaemonWaveCountDown <= 0)
+            {
+                activeDaemonWaveCount = pendingDaemonWave.size();
+                while (!pendingDaemonWave.isEmpty())
+                    advance(pendingDaemonWave.poll());
+                if (activeDaemonWaveCount == 0) pendingDaemonWaveCountDown = Math.max(128, 16 * scheduled.size());
+            }
+        }
+    }
+
+    public void close()
+    {
+        if (sequences.isEmpty() && scheduled.isEmpty() && runnable.isEmpty()
+            && (pendingDaemonWave == null || pendingDaemonWave.isEmpty()) && !moreWork.hasNext())
+            return;
+
+        List<Sequence> invalidateSequences = new ArrayList<>(this.sequences.values());
+        List<Action> invalidateActions = new ArrayList<>(scheduled.size() + runnable.size() + (pendingDaemonWave == null ? 0 : pendingDaemonWave.size()));
+        invalidateActions.addAll(scheduled);
+        invalidateActions.addAll(runnable);
+        if (pendingDaemonWave != null)
+            invalidateActions.addAll(pendingDaemonWave);
+        while (moreWork.hasNext())
+            moreWork.next().actors.forEach(invalidateActions::addAll);
+
+        Throwable fail = SafeCollections.safeForEach(invalidateSequences, Sequence::invalidatePending);
+        fail = Throwables.merge(fail, SafeCollections.safeForEach(invalidateActions, Action::invalidate));
+        scheduled.clear();
+        runnable.clear();
+        runnableByDeadline.clear();
+        if (pendingDaemonWave != null)
+            pendingDaemonWave.clear();
+        sequences.clear();
+        Throwables.maybeFail(fail);
+    }
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/Actions.java b/test/simulator/main/org/apache/cassandra/simulator/Actions.java
new file mode 100644
index 0000000..eea1dd9
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/Actions.java
@@ -0,0 +1,158 @@
+/*
+ * 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.simulator;
+
+import java.util.function.Supplier;
+
+import org.apache.cassandra.simulator.Action.Modifiers;
+
+import static org.apache.cassandra.simulator.Action.Modifiers.INFINITE_STREAM;
+import static org.apache.cassandra.simulator.Action.Modifiers.INFINITE_STREAM_ITEM;
+import static org.apache.cassandra.simulator.Action.Modifiers.NONE;
+import static org.apache.cassandra.simulator.Action.Modifiers.RELIABLE;
+import static org.apache.cassandra.simulator.Action.Modifiers.STREAM;
+import static org.apache.cassandra.simulator.Action.Modifiers.STREAM_ITEM;
+import static org.apache.cassandra.simulator.Action.Modifiers.STRICT;
+import static org.apache.cassandra.utils.LazyToString.lazy;
+
+public class Actions
+{
+    public static class LambdaAction extends Action
+    {
+        private Supplier<ActionList> perform;
+
+        public LambdaAction(Object description, Supplier<ActionList> perform)
+        {
+            this(description, Modifiers.NONE, perform);
+        }
+
+        public LambdaAction(Object description, Modifiers self, Supplier<ActionList> perform)
+        {
+            this(description, self, Modifiers.NONE, perform);
+        }
+
+        public LambdaAction(Object description, Modifiers self, Modifiers children, Supplier<ActionList> perform)
+        {
+            this(description, OrderOn.NONE, self, children, perform);
+        }
+
+        public LambdaAction(Object description, OrderOn orderOn, Modifiers self, Modifiers children, Supplier<ActionList> perform)
+        {
+            super(description, orderOn, self, children);
+            this.perform = perform;
+        }
+
+        protected ActionList performSimple()
+        {
+            ActionList result = perform.get();
+            perform = null;
+            return result;
+        }
+    }
+
+    /**
+     * Should always be performed eventually.
+     */
+    public static class ReliableAction extends LambdaAction
+    {
+        public ReliableAction(Object description, Supplier<ActionList> perform, boolean transitive)
+        {
+            this(description, RELIABLE, transitive ? RELIABLE : NONE, perform);
+        }
+
+        public ReliableAction(Object description, Modifiers self, Modifiers children, Supplier<ActionList> perform)
+        {
+            this(description, OrderOn.NONE, self, children, perform);
+        }
+
+        public ReliableAction(Object description, OrderOn orderOn, Modifiers self, Modifiers children, Supplier<ActionList> perform)
+        {
+            super(description, orderOn, self, children, perform);
+            assert !is(Modifier.DROP);
+            assert children.is(Modifier.RELIABLE);
+        }
+
+        public static ReliableAction transitively(Object description, Supplier<ActionList> action)
+        {
+            return new ReliableAction(description, action, true);
+        }
+    }
+
+    /**
+     * Should always be performed in strict order, i.e. all of this action's child actions should complete before
+     * the next action scheduled by the same actor is invoked.
+     */
+    public static class StrictAction extends LambdaAction
+    {
+        public StrictAction(Object description, Supplier<ActionList> perform, boolean transitive)
+        {
+            super(description, STRICT, transitive ? STRICT : NONE, perform);
+        }
+
+        public static StrictAction of(Object description, Supplier<ActionList> action)
+        {
+            return new StrictAction(description, action, false);
+        }
+    }
+
+    public static Action of(Object description, Supplier<ActionList> action)
+    {
+        return new LambdaAction(description, action);
+    }
+
+    public static Action of(Modifiers self, Modifiers children, Object description, Supplier<ActionList> action)
+    {
+        return new LambdaAction(description, self, children, action);
+    }
+
+    public static Action of(OrderOn orderOn, Modifiers self, Modifiers children, Object description, Supplier<ActionList> action)
+    {
+        return new LambdaAction(description, orderOn, self, children, action);
+    }
+
+    public static Action empty(String message)
+    {
+        return of(message, ActionList::empty);
+    }
+
+    public static Action empty(Modifiers modifiers, Object message)
+    {
+        return of(modifiers, NONE, message, ActionList::empty);
+    }
+
+    public static Action stream(int concurrency, Supplier<Action> actions) { return stream(new OrderOn.Strict(actions, concurrency), actions); }
+    public static Action stream(OrderOn on, Supplier<Action> actions) { return of(OrderOn.NONE, STREAM, NONE, on, () -> ActionList.of(streamNextSupplier(STREAM, STREAM_ITEM, on, 0, on, actions))); }
+    public static Action infiniteStream(int concurrency, Supplier<Action> actions) { return infiniteStream(new OrderOn.Strict(actions, concurrency), actions); }
+    public static Action infiniteStream(OrderOn on, Supplier<Action> actions) { return of(OrderOn.NONE, INFINITE_STREAM, NONE, on, () -> ActionList.of(streamNextSupplier(INFINITE_STREAM, INFINITE_STREAM_ITEM, on, 0, on, actions))); }
+    private static ActionList next(Modifiers modifiers, Object description, int sequence, OrderOn on, Supplier<Action> actions)
+    {
+        Action next = actions.get();
+        if (next == null)
+            return ActionList.empty();
+        return ActionList.of(next, streamNextSupplier(modifiers, modifiers, description, sequence + 1, on, actions));
+    }
+
+    private static Action streamNextSupplier(Modifiers modifiers, Modifiers nextModifiers, Object description, int sequence, OrderOn on, Supplier<Action> actions)
+    {
+        return Actions.of(on, modifiers, NONE,
+                          lazy(() -> description + " " + sequence), () -> next(nextModifiers, description, sequence, on, actions));
+    }
+
+
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java b/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java
new file mode 100644
index 0000000..a4938f1
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java
@@ -0,0 +1,748 @@
+/*
+ * 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.simulator;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.nio.file.FileSystem;
+import java.util.EnumMap;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import java.util.function.Supplier;
+
+import com.google.common.jimfs.Configuration;
+import com.google.common.jimfs.Jimfs;
+import com.google.common.util.concurrent.AsyncFunction;
+import com.google.common.util.concurrent.FutureCallback;
+
+import org.apache.cassandra.concurrent.ExecutorFactory;
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.IInstance;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.IInstanceInitializer;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableBiConsumer;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableConsumer;
+import org.apache.cassandra.distributed.impl.DirectStreamingConnectionFactory;
+import org.apache.cassandra.distributed.impl.IsolatedExecutor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.service.paxos.BallotGenerator;
+import org.apache.cassandra.simulator.RandomSource.Choices;
+import org.apache.cassandra.simulator.asm.NemesisFieldSelectors;
+import org.apache.cassandra.simulator.cluster.ClusterActions;
+import org.apache.cassandra.simulator.cluster.ClusterActions.TopologyChange;
+import org.apache.cassandra.simulator.debug.Capture;
+import org.apache.cassandra.simulator.asm.InterceptClasses;
+import org.apache.cassandra.simulator.systems.Failures;
+import org.apache.cassandra.simulator.systems.InterceptibleThread;
+import org.apache.cassandra.simulator.systems.InterceptorOfGlobalMethods;
+import org.apache.cassandra.simulator.systems.InterceptingExecutorFactory;
+import org.apache.cassandra.simulator.systems.NetworkConfig;
+import org.apache.cassandra.simulator.systems.NetworkConfig.PhaseConfig;
+import org.apache.cassandra.simulator.systems.SchedulerConfig;
+import org.apache.cassandra.simulator.systems.SimulatedFutureActionScheduler;
+import org.apache.cassandra.simulator.systems.SimulatedSystems;
+import org.apache.cassandra.simulator.systems.SimulatedBallots;
+import org.apache.cassandra.simulator.systems.SimulatedExecution;
+import org.apache.cassandra.simulator.systems.SimulatedFailureDetector;
+import org.apache.cassandra.simulator.systems.SimulatedMessageDelivery;
+import org.apache.cassandra.simulator.systems.SimulatedSnitch;
+import org.apache.cassandra.simulator.systems.SimulatedTime;
+import org.apache.cassandra.simulator.systems.SimulatedWaits;
+import org.apache.cassandra.simulator.utils.ChanceRange;
+import org.apache.cassandra.simulator.utils.IntRange;
+import org.apache.cassandra.simulator.utils.KindOfSequence;
+import org.apache.cassandra.simulator.utils.LongRange;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.memory.BufferPool;
+import org.apache.cassandra.utils.memory.BufferPools;
+import org.apache.cassandra.utils.memory.HeapPool;
+
+import static java.lang.Integer.min;
+import static java.util.Collections.emptyMap;
+import static java.util.concurrent.TimeUnit.MICROSECONDS;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.distributed.impl.AbstractCluster.getSharedClassPredicate;
+import static org.apache.cassandra.simulator.SimulatorUtils.failWithOOM;
+import static org.apache.cassandra.utils.Shared.Scope.ANY;
+import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
+
+/**
+ * Wraps a Cluster and a Simulation to run upon it
+ */
+@SuppressWarnings("RedundantCast")
+public class ClusterSimulation<S extends Simulation> implements AutoCloseable
+{
+    public static final Class<?>[] SHARE = new Class[]
+                                        {
+                                            AsyncFunction.class,
+                                            FutureCallback.class,
+                                            io.netty.util.concurrent.GenericFutureListener.class,
+                                            io.netty.channel.FileRegion.class,
+                                            io.netty.util.ReferenceCounted.class
+                                        };
+
+    public static final Class<?>[] ISOLATE = new Class<?>[0];
+
+    public interface SimulationFactory<S extends Simulation>
+    {
+        S create(SimulatedSystems simulated, RunnableActionScheduler scheduler, Cluster cluster, ClusterActions.Options options);
+    }
+
+    public interface SchedulerFactory
+    {
+        RunnableActionScheduler create(RandomSource random);
+    }
+
+    @SuppressWarnings("UnusedReturnValue")
+    public static abstract class Builder<S extends Simulation>
+    {
+        protected Supplier<RandomSource> randomSupplier = RandomSource.Default::new;
+        protected int uniqueNum = 0;
+        protected int threadCount;
+
+        protected int concurrency = 10;
+        protected IntRange nodeCount = new IntRange(4, 16), dcCount = new IntRange(1, 2),
+                        primaryKeySeconds = new IntRange(5, 30), withinKeyConcurrency = new IntRange(2, 5);
+        protected TopologyChange[] topologyChanges = TopologyChange.values();
+
+        protected int primaryKeyCount;
+        protected int secondsToSimulate;
+
+        protected ChanceRange normalNetworkDropChance  = new ChanceRange(randomSource -> randomSource.qlog2uniformFloat(4), 0f, 0.001f),
+                              normalNetworkDelayChance = new ChanceRange(randomSource -> randomSource.qlog2uniformFloat(4), 0.01f, 0.1f),
+                                flakyNetworkDropChance = new ChanceRange(randomSource -> randomSource.qlog2uniformFloat(4), 0.01f, 0.1f),
+                               flakyNetworkDelayChance = new ChanceRange(randomSource -> randomSource.qlog2uniformFloat(4), 0.01f, 0.1f),
+                                networkPartitionChance = new ChanceRange(randomSource -> randomSource.qlog2uniformFloat(4), 0.0f, 0.1f),
+                                    networkFlakyChance = new ChanceRange(randomSource -> randomSource.qlog2uniformFloat(4), 0.0f, 0.1f),
+                                    monitorDelayChance = new ChanceRange(randomSource -> randomSource.qlog2uniformFloat(4), 0.01f, 0.1f),
+                                  schedulerDelayChance = new ChanceRange(randomSource -> randomSource.qlog2uniformFloat(4), 0.01f, 0.1f),
+                                         timeoutChance = new ChanceRange(randomSource -> randomSource.qlog2uniformFloat(4),  0.01f, 0.1f),
+                                            readChance = new ChanceRange(RandomSource::uniformFloat,                         0.05f, 0.95f),
+                                         nemesisChance = new ChanceRange(randomSource -> randomSource.qlog2uniformFloat(4), 0.001f, 0.01f);
+
+        protected LongRange normalNetworkLatencyNanos = new LongRange(1, 2, MILLISECONDS, NANOSECONDS),
+                              normalNetworkDelayNanos = new LongRange(2, 100, MILLISECONDS, NANOSECONDS),
+                             flakyNetworkLatencyNanos = new LongRange(2, 100, MILLISECONDS, NANOSECONDS),
+                               flakyNetworkDelayNanos = new LongRange(2, 100, MILLISECONDS, NANOSECONDS),
+                           networkReconfigureInterval = new LongRange(50, 5000, MICROSECONDS, NANOSECONDS),
+                                 schedulerJitterNanos = new LongRange(100, 2000, MICROSECONDS, NANOSECONDS),
+                                  schedulerDelayNanos = new LongRange(0, 50, MICROSECONDS, NANOSECONDS),
+                              schedulerLongDelayNanos = new LongRange(50, 5000, MICROSECONDS, NANOSECONDS),
+                                      clockDriftNanos = new LongRange(1, 5000, MILLISECONDS, NANOSECONDS),
+                       clockDiscontinuitIntervalNanos = new LongRange(10, 60, SECONDS, NANOSECONDS),
+                          topologyChangeIntervalNanos = new LongRange(5, 15, SECONDS, NANOSECONDS);
+
+
+
+        protected long contentionTimeoutNanos = MILLISECONDS.toNanos(500L),
+                            writeTimeoutNanos = SECONDS.toNanos(1L),
+                             readTimeoutNanos = SECONDS.toNanos(2L),
+                          requestTimeoutNanos = SECONDS.toNanos(2L);
+
+        protected SchedulerFactory schedulerFactory = schedulerFactory(RunnableActionScheduler.Kind.values());
+
+        protected Debug debug = new Debug();
+        protected Capture capture = new Capture(false, false, false);
+        protected HeapPool.Logged.Listener memoryListener;
+
+        public Debug debug()
+        {
+            return debug;
+        }
+
+        public Builder<S> debug(EnumMap<Debug.Info, Debug.Levels> debug, int[] primaryKeys)
+        {
+            this.debug = new Debug(debug, primaryKeys);
+            return this;
+        }
+
+        public Builder<S> unique(int num)
+        {
+            this.uniqueNum = num;
+            return this;
+        }
+
+        public Builder<S> threadCount(int count)
+        {
+            this.threadCount = count;
+            return this;
+        }
+
+        public Builder<S> nodes(IntRange range)
+        {
+            this.nodeCount = range;
+            return this;
+        }
+
+        public Builder<S> nodes(int min, int max)
+        {
+            this.nodeCount = new IntRange(min, max);
+            return this;
+        }
+
+        public Builder<S> dcs(IntRange range)
+        {
+            this.dcCount = range;
+            return this;
+        }
+
+        public Builder<S> dcs(int min, int max)
+        {
+            this.dcCount = new IntRange(min, max);
+            return this;
+        }
+
+        public Builder<S> concurrency(int concurrency)
+        {
+            this.concurrency = concurrency;
+            return this;
+        }
+
+        public IntRange primaryKeySeconds()
+        {
+            return primaryKeySeconds;
+        }
+
+        public Builder<S> primaryKeySeconds(IntRange range)
+        {
+            this.primaryKeySeconds = range;
+            return this;
+        }
+
+        public Builder<S> withinKeyConcurrency(IntRange range)
+        {
+            this.withinKeyConcurrency = range;
+            return this;
+        }
+
+        public Builder<S> withinKeyConcurrency(int min, int max)
+        {
+            this.withinKeyConcurrency = new IntRange(min, max);
+            return this;
+        }
+
+        public Builder<S> topologyChanges(TopologyChange[] topologyChanges)
+        {
+            this.topologyChanges = topologyChanges;
+            return this;
+        }
+
+        public Builder<S> topologyChangeIntervalNanos(LongRange topologyChangeIntervalNanos)
+        {
+            this.topologyChangeIntervalNanos = topologyChangeIntervalNanos;
+            return this;
+        }
+
+        public int primaryKeyCount()
+        {
+            return primaryKeyCount;
+        }
+
+        public Builder<S> primaryKeyCount(int count)
+        {
+            this.primaryKeyCount = count;
+            return this;
+        }
+
+        public int secondsToSimulate()
+        {
+            return secondsToSimulate;
+        }
+
+        public Builder<S> secondsToSimulate(int seconds)
+        {
+            this.secondsToSimulate = seconds;
+            return this;
+        }
+
+        public Builder<S> networkPartitionChance(ChanceRange partitionChance)
+        {
+            this.networkPartitionChance = partitionChance;
+            return this;
+        }
+
+        public Builder<S> networkFlakyChance(ChanceRange flakyChance)
+        {
+            this.networkFlakyChance = flakyChance;
+            return this;
+        }
+
+        public Builder<S> networkReconfigureInterval(LongRange reconfigureIntervalNanos)
+        {
+            this.networkReconfigureInterval = reconfigureIntervalNanos;
+            return this;
+        }
+
+        public Builder<S> networkDropChance(ChanceRange dropChance)
+        {
+            this.normalNetworkDropChance = dropChance;
+            return this;
+        }
+
+        public Builder<S> networkDelayChance(ChanceRange delayChance)
+        {
+            this.normalNetworkDelayChance = delayChance;
+            return this;
+        }
+
+        public Builder<S> networkLatencyNanos(LongRange networkLatencyNanos)
+        {
+            this.normalNetworkLatencyNanos = networkLatencyNanos;
+            return this;
+        }
+
+        public Builder<S> networkDelayNanos(LongRange networkDelayNanos)
+        {
+            this.normalNetworkDelayNanos = networkDelayNanos;
+            return this;
+        }
+
+        public Builder<S> flakyNetworkDropChance(ChanceRange dropChance)
+        {
+            this.flakyNetworkDropChance = dropChance;
+            return this;
+        }
+
+        public Builder<S> flakyNetworkDelayChance(ChanceRange delayChance)
+        {
+            this.flakyNetworkDelayChance = delayChance;
+            return this;
+        }
+
+        public Builder<S> flakyNetworkLatencyNanos(LongRange networkLatencyNanos)
+        {
+            this.flakyNetworkLatencyNanos = networkLatencyNanos;
+            return this;
+        }
+
+        public Builder<S> flakyNetworkDelayNanos(LongRange networkDelayNanos)
+        {
+            this.flakyNetworkDelayNanos = networkDelayNanos;
+            return this;
+        }
+
+        public Builder<S> clockDriftNanos(LongRange clockDriftNanos)
+        {
+            this.clockDriftNanos = clockDriftNanos;
+            return this;
+        }
+
+        public Builder<S> clockDiscontinuityIntervalNanos(LongRange clockDiscontinuityIntervalNanos)
+        {
+            this.clockDiscontinuitIntervalNanos = clockDiscontinuityIntervalNanos;
+            return this;
+        }
+
+        public Builder<S> schedulerDelayChance(ChanceRange delayChance)
+        {
+            this.schedulerDelayChance = delayChance;
+            return this;
+        }
+
+        public Builder<S> schedulerJitterNanos(LongRange schedulerJitterNanos)
+        {
+            this.schedulerJitterNanos = schedulerJitterNanos;
+            return this;
+        }
+
+        public LongRange schedulerJitterNanos()
+        {
+            return schedulerJitterNanos;
+        }
+
+        public Builder<S> schedulerDelayNanos(LongRange schedulerDelayNanos)
+        {
+            this.schedulerDelayNanos = schedulerDelayNanos;
+            return this;
+        }
+
+        public Builder<S> schedulerLongDelayNanos(LongRange schedulerLongDelayNanos)
+        {
+            this.schedulerLongDelayNanos = schedulerLongDelayNanos;
+            return this;
+        }
+
+        public Builder<S> timeoutChance(ChanceRange timeoutChance)
+        {
+            this.timeoutChance = timeoutChance;
+            return this;
+        }
+
+        public ChanceRange readChance()
+        {
+            return readChance;
+        }
+
+        public IntRange withinKeyConcurrency()
+        {
+            return withinKeyConcurrency;
+        }
+
+        public int concurrency()
+        {
+            return concurrency;
+        }
+
+        public Builder<S> readChance(ChanceRange readChance)
+        {
+            this.readChance = readChance;
+            return this;
+        }
+
+        public Builder<S> nemesisChance(ChanceRange nemesisChance)
+        {
+            this.nemesisChance = nemesisChance;
+            return this;
+        }
+
+        public Builder<S> scheduler(RunnableActionScheduler.Kind... kinds)
+        {
+            this.schedulerFactory = schedulerFactory(kinds);
+            return this;
+        }
+
+        public SimulatedFutureActionScheduler futureActionScheduler(int nodeCount, SimulatedTime time, RandomSource random)
+        {
+            KindOfSequence kind = Choices.random(random, KindOfSequence.values())
+                                         .choose(random);
+            return new SimulatedFutureActionScheduler(kind, nodeCount, random, time,
+                                                      new NetworkConfig(new PhaseConfig(normalNetworkDropChance, normalNetworkDelayChance, normalNetworkLatencyNanos, normalNetworkDelayNanos),
+                                                                        new PhaseConfig(flakyNetworkDropChance, flakyNetworkDelayChance, flakyNetworkLatencyNanos, flakyNetworkDelayNanos),
+                                                                        networkPartitionChance, networkFlakyChance, networkReconfigureInterval),
+                                                      new SchedulerConfig(schedulerDelayChance, schedulerDelayNanos, schedulerLongDelayNanos));
+        }
+
+        static SchedulerFactory schedulerFactory(RunnableActionScheduler.Kind... kinds)
+        {
+            return (random) -> {
+                switch (Choices.random(random, kinds).choose(random))
+                {
+                    default: throw new AssertionError();
+                    case SEQUENTIAL: return new RunnableActionScheduler.Sequential();
+                    case UNIFORM: return new RunnableActionScheduler.RandomUniform(random);
+                    case RANDOM_WALK: return new RunnableActionScheduler.RandomWalk(random);
+                }
+            };
+        }
+
+        public Builder<S> scheduler(SchedulerFactory schedulerFactory)
+        {
+            this.schedulerFactory = schedulerFactory;
+            return this;
+        }
+
+        public Builder<S> random(Supplier<RandomSource> randomSupplier)
+        {
+            this.randomSupplier = randomSupplier;
+            return this;
+        }
+
+        public Builder<S> memoryListener(HeapPool.Logged.Listener memoryListener)
+        {
+            this.memoryListener = memoryListener;
+            return this;
+        }
+
+        public Builder<S> capture(Capture capture)
+        {
+            this.capture = capture;
+            return this;
+        }
+
+        public Capture capture()
+        {
+            return capture;
+        }
+
+        public abstract ClusterSimulation<S> create(long seed) throws IOException;
+    }
+
+    static class ThreadAllocator
+    {
+        final RandomSource random;
+        int clusterPool; // number of threads we have left for the whole cluster
+        int remainingNodes; // number of nodes we still need to allocate them to
+        int allocationPool; //  threads to allocate for the node we're currently processing
+        int remainingAllocations; // number of _remaining_ allocations take() assumes we want to evenly allocate threads over
+
+        public ThreadAllocator(RandomSource random, int threadsToAllocate, int betweenNodes)
+        {
+            this.random = random;
+            this.clusterPool = threadsToAllocate;
+            this.remainingNodes = betweenNodes;
+        }
+
+        // randomly set the number of threads in various thread pools
+        IInstanceConfig update(IInstanceConfig config)
+        {
+            cycle();
+            // allocate in ascending order of max, for take() correctness
+            return config
+                   .set("memtable_flush_writers", take(1, 1, 2))
+                   .set("concurrent_compactors", take(1, 1, 4))
+                   .set("concurrent_writes", take(1, 4))
+                   .set("concurrent_counter_writes", take(1, 4))
+                   .set("concurrent_materialized_view_writes", take(1, 4))
+                   .set("concurrent_reads", take(1, 4))
+                   .forceSet("available_processors", take(3, 4));
+        }
+
+        // begin allocating for a new node
+        void cycle()
+        {
+            assert remainingNodes > 0;
+            // return unallocated items to the outerPool
+            clusterPool += allocationPool;
+            // set the curPool to allocate the next allocationPool size
+            allocationPool = clusterPool;
+            remainingAllocations = remainingNodes;
+            // randomly select the next pool size, subtracting it from the outer pool
+            allocationPool = take(1, 1);
+            clusterPool -= allocationPool;
+            // this is hard-coded to match the sum of the first arguments above
+            remainingAllocations = 9;
+            --remainingNodes;
+        }
+
+        /**
+         * See {@link #take(int, int, int)}
+         */
+        int take(int times, int min)
+        {
+            return take(times, min, allocationPool);
+        }
+
+        /**
+         * Allocate a random number of threads between [min..max)
+         * The allocation is suitable for multiple users of the value, i.e.
+         * {@code times} multiple of the result are deducted from the pool.
+         *
+         * If there are adequate supplies we aim to allocate threads "equally" between pools,
+         * selecting a uniform value between 0.5x and 2x the fair split of the remaining pool
+         * on each allocation. If the min/max bounds override that, they are preferred.
+         *
+         * The minimum is always honoured, regardless of available pool size.
+         */
+        int take(int times, int min, int max)
+        {
+            int remaining = remainingAllocations;
+            assert remaining >= times;
+            remainingAllocations -= times;
+            if (remaining * min <= allocationPool)
+                return min;
+            if (times == remaining)
+                return allocationPool / remaining;
+            if (times + 1 == remaining)
+                return random.uniform(Math.max(min, (allocationPool - max) / times), Math.min(max, (allocationPool - min) / times));
+
+            int median = allocationPool / remaining;
+            min = Math.max(min, Math.min(max, median) / 2);
+            max = Math.min(max, median * 2);
+            return min >= max ? min : random.uniform(min, max);
+        }
+    }
+
+
+    public final RandomSource random;
+    public final SimulatedSystems simulated;
+    public final Cluster cluster;
+    public final S simulation;
+    private final FileSystem jimfs;
+    protected final Map<Integer, InterceptingExecutorFactory> factories = new TreeMap<>();
+
+    public ClusterSimulation(RandomSource random, long seed, int uniqueNum,
+                             Builder<?> builder,
+                             Consumer<IInstanceConfig> configUpdater,
+                             SimulationFactory<S> factory) throws IOException
+    {
+        this.random = random;
+        this.jimfs  = Jimfs.newFileSystem(Long.toHexString(seed) + 'x' + uniqueNum, Configuration.unix().toBuilder()
+                                                                               .setMaxSize(4L << 30).setBlockSize(512)
+                                                                               .build());
+
+        final SimulatedMessageDelivery delivery;
+        final SimulatedWaits waits;
+        final SimulatedExecution execution;
+        final SimulatedBallots ballots;
+        final SimulatedSnitch snitch;
+        final SimulatedTime time;
+        final SimulatedFailureDetector failureDetector;
+
+        int numOfNodes = builder.nodeCount.select(random);
+        int numOfDcs = builder.dcCount.select(random, 0, numOfNodes / 3);
+        int[] numInDcs = new int[numOfDcs];
+        int[] nodeToDc = new int[numOfNodes];
+
+        int[] minRf = new int[numOfDcs], initialRf = new int[numOfDcs], maxRf = new int[numOfDcs];
+        {
+            // TODO (feature): split unevenly
+            int n = 0, nc = 0;
+            for (int i = 0; i < numOfDcs; ++i)
+            {
+                int numInDc = (numOfNodes / numOfDcs) + (numOfNodes % numOfDcs > i ? 1 : 0);
+                numInDcs[i] = numInDc;
+                minRf[i] = 3;
+                maxRf[i] = min(numInDc, 9);
+                initialRf[i] = random.uniform(minRf[i], 1 + maxRf[i]);
+                nc += numInDc;
+                while (n < nc)
+                    nodeToDc[n++] = i;
+            }
+        }
+        snitch = new SimulatedSnitch(nodeToDc, numInDcs);
+
+        waits = new SimulatedWaits(random);
+        waits.captureStackTraces(builder.capture.waitSites, builder.capture.wakeSites, builder.capture.nowSites);
+        execution = new SimulatedExecution();
+
+        KindOfSequence kindOfDriftSequence = Choices.uniform(KindOfSequence.values()).choose(random);
+        KindOfSequence kindOfDiscontinuitySequence = Choices.uniform(KindOfSequence.values()).choose(random);
+        time = new SimulatedTime(random, 1577836800000L /*Jan 1st UTC*/, builder.clockDriftNanos, kindOfDriftSequence,
+                                 kindOfDiscontinuitySequence.period(builder.clockDiscontinuitIntervalNanos, random));
+        ballots = new SimulatedBallots(random, () -> {
+            long max = random.uniform(2, 16);
+            return () -> random.uniform(1, max);
+        });
+
+        Failures failures = new Failures();
+        ThreadAllocator threadAllocator = new ThreadAllocator(random, builder.threadCount, numOfNodes);
+        cluster = snitch.setup(Cluster.build(numOfNodes)
+                         .withRoot(jimfs.getPath("/cassandra"))
+                         .withSharedClasses(getSharedClassPredicate(ISOLATE, SHARE, ANY, SIMULATION))
+                         .withConfig(config -> configUpdater.accept(threadAllocator.update(config
+                             .with(Feature.BLANK_GOSSIP)
+                             .set("read_request_timeout_in_ms", NANOSECONDS.toMillis(builder.readTimeoutNanos))
+                             .set("write_request_timeout_in_ms", NANOSECONDS.toMillis(builder.writeTimeoutNanos))
+                             .set("cas_contention_timeout_in_ms", NANOSECONDS.toMillis(builder.contentionTimeoutNanos))
+                             .set("request_timeout_in_ms", NANOSECONDS.toMillis(builder.requestTimeoutNanos))
+                             .set("memtable_heap_space_in_mb", 1)
+                             .set("memtable_allocation_type", builder.memoryListener != null ? "unslabbed_heap_buffers_logged" : "heap_buffers")
+                             .set("file_cache_size_in_mb", 16)
+                             .set("use_deterministic_table_id", true)
+                             .set("disk_access_mode", "standard")
+                             .set("failure_detector", SimulatedFailureDetector.Instance.class.getName())
+                             .set("commitlog_compression", new ParameterizedClass(LZ4Compressor.class.getName(), emptyMap()))
+                         )))
+                         .withInstanceInitializer(new IInstanceInitializer()
+                         {
+                             @Override
+                             public void initialise(ClassLoader classLoader, ThreadGroup threadGroup, int num, int generation)
+                             {
+                                 InterceptorOfGlobalMethods interceptorOfGlobalMethods = waits.interceptGlobalMethods(classLoader);
+                                 InterceptingExecutorFactory factory = execution.factory(interceptorOfGlobalMethods, classLoader, threadGroup);
+                                 IsolatedExecutor.transferAdhoc((SerializableConsumer<ExecutorFactory>) ExecutorFactory.Global::unsafeSet, classLoader)
+                                                 .accept(factory);
+                                 IsolatedExecutor.transferAdhoc((SerializableBiConsumer<InterceptorOfGlobalMethods, IntSupplier>) InterceptorOfGlobalMethods.Global::unsafeSet, classLoader)
+                                                 .accept(interceptorOfGlobalMethods, () -> {
+                                                     if (InterceptibleThread.isDeterministic())
+                                                         throw failWithOOM();
+                                                     return random.uniform(Integer.MIN_VALUE, Integer.MAX_VALUE);
+                                                 });
+                                 time.setup(classLoader);
+                                 factories.put(num, factory);
+                             }
+
+                             @Override
+                             public void beforeStartup(IInstance i)
+                             {
+                                 if (builder.memoryListener != null)
+                                    ((IInvokableInstance) i).unsafeAcceptOnThisThread(HeapPool.Logged::setListener, builder.memoryListener);
+                                 ((IInvokableInstance) i).unsafeAcceptOnThisThread(FBUtilities::setAvailableProcessors, i.config().getInt("available_processors"));
+                             }
+
+                             @Override
+                             public void afterStartup(IInstance i)
+                             {
+                                 ((IInvokableInstance) i).unsafeAcceptOnThisThread(BallotGenerator.Global::unsafeSet, (BallotGenerator) ballots.get());
+                                 ((IInvokableInstance) i).unsafeAcceptOnThisThread((SerializableConsumer<BufferPool.DebugLeaks>) debug -> BufferPools.forChunkCache().debug(null, debug), failures);
+                                 ((IInvokableInstance) i).unsafeAcceptOnThisThread((SerializableConsumer<BufferPool.DebugLeaks>) debug -> BufferPools.forNetworking().debug(null, debug), failures);
+                                 ((IInvokableInstance) i).unsafeAcceptOnThisThread((SerializableConsumer<Ref.OnLeak>) Ref::setOnLeak, failures);
+                             }
+
+                         }).withClassTransformer(new InterceptClasses(builder.monitorDelayChance.asSupplier(random), builder.nemesisChance.asSupplier(random), NemesisFieldSelectors.get())::apply)
+        ).createWithoutStarting();
+
+        snitch.setup(cluster);
+        DirectStreamingConnectionFactory.setup(cluster);
+        delivery = new SimulatedMessageDelivery(cluster);
+        failureDetector = new SimulatedFailureDetector(cluster);
+        SimulatedFutureActionScheduler futureActionScheduler = builder.futureActionScheduler(numOfNodes, time, random);
+        simulated = new SimulatedSystems(random, time, waits, delivery, execution, ballots, failureDetector, snitch, futureActionScheduler, builder.debug, failures);
+        simulated.register(futureActionScheduler);
+
+        RunnableActionScheduler scheduler = builder.schedulerFactory.create(random);
+        ClusterActions.Options options = new ClusterActions.Options(Choices.uniform(KindOfSequence.values()).choose(random).period(builder.topologyChangeIntervalNanos, random),
+                                                                    Choices.random(random, builder.topologyChanges),
+                                                                    minRf, initialRf, maxRf, null);
+        simulation = factory.create(simulated, scheduler, cluster, options);
+    }
+
+    public void close() throws IOException
+    {
+        // Re-enable time on shutdown
+        try
+        {
+            Field field = Clock.Global.class.getDeclaredField("instance");
+            field.setAccessible(true);
+
+            Field modifiersField = Field.class.getDeclaredField("modifiers");
+            modifiersField.setAccessible(true);
+            modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
+
+            field.set(null, new Clock.Default());
+        }
+        catch (NoSuchFieldException|IllegalAccessException e)
+        {
+            throw new RuntimeException(e);
+        }
+
+        simulated.waits.stop();
+        simulated.execution.forceStop();
+        factories.values().forEach(InterceptingExecutorFactory::close);
+        Throwable fail = null;
+        try
+        {
+            simulation.close();
+        }
+        catch (Throwable t)
+        {
+            fail = t;
+        }
+        try
+        {
+            cluster.close();
+        }
+        catch (Throwable t)
+        {
+            fail = Throwables.merge(fail, t);
+        }
+        Throwables.maybeFail(fail, IOException.class);
+    }
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/Debug.java b/test/simulator/main/org/apache/cassandra/simulator/Debug.java
new file mode 100644
index 0000000..a03d304
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/Debug.java
@@ -0,0 +1,356 @@
+/*
+ * 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.simulator;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import javax.annotation.Nullable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.BufferDecoratedKey;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.TriFunction;
+import org.apache.cassandra.gms.EndpointState;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.simulator.systems.SimulatedTime;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static java.util.function.Function.identity;
+import static org.apache.cassandra.simulator.Action.Modifier.INFO;
+import static org.apache.cassandra.simulator.Action.Modifier.WAKEUP;
+import static org.apache.cassandra.simulator.ActionListener.runAfter;
+import static org.apache.cassandra.simulator.ActionListener.runAfterAndTransitivelyAfter;
+import static org.apache.cassandra.simulator.ActionListener.recursive;
+import static org.apache.cassandra.simulator.Debug.EventType.*;
+import static org.apache.cassandra.simulator.Debug.Info.LOG;
+import static org.apache.cassandra.simulator.Debug.Level.*;
+import static org.apache.cassandra.simulator.paxos.Ballots.paxosDebugInfo;
+
+// TODO (feature): move logging to a depth parameter
+// TODO (feature): log only deltas for schema/cluster data
+public class Debug
+{
+    private static final Logger logger = LoggerFactory.getLogger(Debug.class);
+
+    public enum EventType { PARTITION, CLUSTER }
+    public enum Level
+    {
+        PLANNED,
+        CONSEQUENCES,
+        ALL;
+
+        private static final Level[] LEVELS = values();
+    }
+    public enum Info
+    {
+        LOG(EventType.values()),
+        PAXOS(PARTITION),
+        OWNERSHIP(CLUSTER),
+        GOSSIP(CLUSTER),
+        RF(CLUSTER),
+        RING(CLUSTER);
+
+        public final EventType[] defaultEventTypes;
+
+        Info(EventType ... defaultEventTypes)
+        {
+            this.defaultEventTypes = defaultEventTypes;
+        }
+    }
+
+    public static class Levels
+    {
+        private final EnumMap<EventType, Level> levels;
+
+        public Levels(EnumMap<EventType, Level> levels)
+        {
+            this.levels = levels;
+        }
+
+        public Levels(Level level, EventType ... types)
+        {
+            this.levels = new EnumMap<>(EventType.class);
+            for (EventType type : types)
+                this.levels.put(type, level);
+        }
+
+        public Levels(int partition, int cluster)
+        {
+            this.levels = new EnumMap<>(EventType.class);
+            if (partition > 0) this.levels.put(PARTITION, Level.LEVELS[partition - 1]);
+            if (cluster > 0) this.levels.put(CLUSTER, Level.LEVELS[cluster - 1]);
+        }
+
+        Level get(EventType type)
+        {
+            return levels.get(type);
+        }
+
+        boolean anyMatch(Predicate<Level> test)
+        {
+            return levels.values().stream().anyMatch(test);
+        }
+    }
+
+    private final EnumMap<Info, Levels> levels;
+    public final int[] primaryKeys;
+
+    public Debug()
+    {
+        this(new EnumMap<>(Info.class), null);
+    }
+
+    public Debug(Map<Info, Levels> levels, int[] primaryKeys)
+    {
+        this.levels = new EnumMap<>(levels);
+        this.primaryKeys = primaryKeys;
+    }
+
+    public ActionListener debug(EventType type, SimulatedTime time, Cluster cluster, String keyspace, Integer primaryKey)
+    {
+        List<ActionListener> listeners = new ArrayList<>();
+        for (Map.Entry<Info, Levels> e : levels.entrySet())
+        {
+            Info info = e.getKey();
+            Level level = e.getValue().get(type);
+            if (level == null) continue;
+
+            ActionListener listener;
+            if (info == LOG)
+            {
+                Function<ActionListener, ActionListener> adapt = type == CLUSTER ? LogTermination::new : identity();
+                switch (level)
+                {
+                    default: throw new AssertionError();
+                    case PLANNED: listener = adapt.apply(new LogOne(time, false)); break;
+                    case CONSEQUENCES: case ALL: listener = adapt.apply(recursive(new LogOne(time, true))); break;
+                }
+            }
+            else
+            {
+                Consumer<Action> debug;
+                switch (info)
+                {
+                    default: throw new AssertionError();
+                    case GOSSIP: debug = debugGossip(cluster); break;
+                    case RF: debug = debugRf(cluster, keyspace); break;
+                    case RING: debug = debugRing(cluster, keyspace); break;
+                    case PAXOS: debug = forKeys(cluster, keyspace, primaryKey, Debug::debugPaxos); break;
+                    case OWNERSHIP: debug = forKeys(cluster, keyspace, primaryKey, Debug::debugOwnership); break;
+                }
+                switch (level)
+                {
+                    default: throw new AssertionError();
+                    case PLANNED: listener = type == CLUSTER ? runAfterAndTransitivelyAfter(debug) : runAfter(debug); break;
+                    case CONSEQUENCES: listener = recursive(runAfter(ignoreWakeupAndLogEvents(debug))); break;
+                    case ALL: listener = recursive(runAfter(ignoreLogEvents(debug))); break;
+                }
+            }
+
+            listeners.add(listener);
+        }
+
+        if (listeners.isEmpty())
+            return null;
+        return new ActionListener.Combined(listeners);
+    }
+
+    public boolean isOn(Info info)
+    {
+        return isOn(info, PLANNED);
+    }
+
+    public boolean isOn(Info info, Level level)
+    {
+        Levels levels = this.levels.get(info);
+        if (levels == null) return false;
+        return levels.anyMatch(test -> level.compareTo(test) >= 0);
+    }
+
+    @SuppressWarnings("UnnecessaryToStringCall")
+    private static class LogOne implements ActionListener
+    {
+        final SimulatedTime time;
+        final boolean logConsequences;
+        private LogOne(SimulatedTime time, boolean logConsequences)
+        {
+            this.time = time;
+            this.logConsequences = logConsequences;
+        }
+
+        @Override
+        public void before(Action action, Before before)
+        {
+            if (logger.isWarnEnabled()) // invoke toString() eagerly to ensure we have the task's descriptin
+                logger.warn(String.format("%6ds %s %s", TimeUnit.NANOSECONDS.toSeconds(time.nanoTime()), before, action));
+        }
+
+        @Override
+        public void consequences(ActionList consequences)
+        {
+            if (logConsequences && !consequences.isEmpty() && logger.isWarnEnabled())
+                logger.warn(String.format("%6ds Next: %s", TimeUnit.NANOSECONDS.toSeconds(time.nanoTime()), consequences));
+        }
+    }
+
+    private static class LogTermination extends ActionListener.Wrapped
+    {
+        public LogTermination(ActionListener wrap)
+        {
+            super(wrap);
+        }
+
+        @Override
+        public void transitivelyAfter(Action finished)
+        {
+            logger.warn("Terminated {}", finished);
+        }
+    }
+
+    private static Consumer<Action> ignoreWakeupAndLogEvents(Consumer<Action> consumer)
+    {
+        return action -> {
+            if (!action.is(WAKEUP) && !action.is(INFO))
+                consumer.accept(action);
+        };
+    }
+
+    private static Consumer<Action> ignoreLogEvents(Consumer<Action> consumer)
+    {
+        return action -> {
+            if (!action.is(INFO))
+                consumer.accept(action);
+        };
+    }
+
+    private Consumer<Action> debugGossip(Cluster cluster)
+    {
+        return ignore -> {
+            cluster.forEach(i -> i.unsafeRunOnThisThread(() -> {
+                for (InetAddressAndPort ep : Gossiper.instance.getLiveMembers())
+                {
+                    EndpointState epState = Gossiper.instance.getEndpointStateForEndpoint(ep);
+                    logger.warn("Gossip {}: {} {}", ep, epState.isAlive(), epState.states().stream()
+                                                                                   .map(e -> e.getKey().toString() + "=(" + e.getValue().value + ',' + e.getValue().version + ')')
+                                                                                   .collect(Collectors.joining(", ", "[", "]")));
+                }
+            }));
+        };
+    }
+
+    private Consumer<Action> forKeys(Cluster cluster, String keyspace, @Nullable Integer specificPrimaryKey, TriFunction<Cluster, String, Integer, Consumer<Action>> factory)
+    {
+        if (specificPrimaryKey != null) return factory.apply(cluster, keyspace, specificPrimaryKey);
+        else return forEachKey(cluster, keyspace, primaryKeys, Debug::debugPaxos);
+    }
+
+    public static Consumer<Action> forEachKey(Cluster cluster, String keyspace, int[] primaryKeys, TriFunction<Cluster, String, Integer, Consumer<Action>> factory)
+    {
+        Consumer<Action>[] eachKey = new Consumer[primaryKeys.length];
+        for (int i = 0 ; i < primaryKeys.length ; ++i)
+            eachKey[i] = factory.apply(cluster, keyspace, primaryKeys[i]);
+
+        return action -> {
+            for (Consumer<Action> run : eachKey)
+                run.accept(action);
+        };
+    }
+
+    public static Consumer<Action> debugPaxos(Cluster cluster, String keyspace, int primaryKey)
+    {
+        return ignore -> {
+            for (int node = 1 ; node <= cluster.size() ; ++node)
+            {
+                cluster.get(node).unsafeAcceptOnThisThread((num, pkint) -> {
+                    try
+                    {
+                        TableMetadata metadata = Keyspace.open(keyspace).getColumnFamilyStore("tbl").metadata.get();
+                        ByteBuffer pkbb = Int32Type.instance.decompose(pkint);
+                        DecoratedKey key = new BufferDecoratedKey(DatabaseDescriptor.getPartitioner().getToken(pkbb), pkbb);
+                        logger.warn("node{}({}): {}", num, primaryKey, paxosDebugInfo(key, metadata, FBUtilities.nowInSeconds()));
+                    }
+                    catch (Throwable t)
+                    {
+                        logger.warn("node{}({})", num, primaryKey, t);
+                    }
+                }, node, primaryKey);
+            }
+        };
+    }
+
+    public static Consumer<Action> debugRf(Cluster cluster, String keyspace)
+    {
+        return ignore -> {
+            cluster.forEach(i -> i.unsafeRunOnThisThread(() -> {
+                logger.warn("{} {}",
+                        Schema.instance.getKeyspaceMetadata(keyspace) == null ? "" : Schema.instance.getKeyspaceMetadata(keyspace).params.replication.toString(),
+                        Schema.instance.getKeyspaceMetadata(keyspace) == null ? "" : Keyspace.open(keyspace).getReplicationStrategy().configOptions.toString());
+            }));
+        };
+    }
+
+    public static Consumer<Action> debugOwnership(Cluster cluster, String keyspace, int primaryKey)
+    {
+        return ignore -> {
+            for (int node = 1 ; node <= cluster.size() ; ++node)
+            {
+                logger.warn("node{}({}): {}", node, primaryKey, cluster.get(node).unsafeApplyOnThisThread(v -> {
+                    try
+                    {
+                        return ReplicaLayout.forTokenWriteLiveAndDown(Keyspace.open(keyspace), Murmur3Partitioner.instance.getToken(Int32Type.instance.decompose(v))).all().endpointList().toString();
+                    }
+                    catch (Throwable t)
+                    {
+                        return "Error";
+                    }
+                }, primaryKey));
+            }
+        };
+    }
+
+    public static Consumer<Action> debugRing(Cluster cluster, String keyspace)
+    {
+        return ignore -> cluster.forEach(i -> i.unsafeRunOnThisThread(() -> {
+            if (Schema.instance.getKeyspaceMetadata(keyspace) != null)
+                logger.warn("{}", StorageService.instance.getTokenMetadata().toString());
+        }));
+    }
+
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/FutureActionScheduler.java b/test/simulator/main/org/apache/cassandra/simulator/FutureActionScheduler.java
new file mode 100644
index 0000000..e0ccc56
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/FutureActionScheduler.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.simulator;
+
+/**
+ * Makes decisions about when in the simulated scheduled, in terms of the global simulated nanoTime,
+ * events should occur.
+ */
+public interface FutureActionScheduler
+{
+    enum Deliver { DELIVER, TIMEOUT, FAILURE }
+
+    /**
+     * Make a decision about the result of some attempt to deliver a message.
+     * Note that this includes responses, so for any given message the chance
+     * of a successful reply depends on two of these calls succeeding.
+     */
+    Deliver shouldDeliver(int from, int to);
+
+    /**
+     * The simulated global nanoTime arrival of a message
+     */
+    long messageDeadlineNanos(int from, int to);
+
+    /**
+     * The simulated global nanoTime at which a timeout should be reported for a message
+     * with {@code expiresAfterNanos} timeout
+     */
+    long messageTimeoutNanos(long expiresAfterNanos);
+
+    /**
+     * The simulated global nanoTime at which a failure should be reported for a message
+     */
+    long messageFailureNanos(int from, int to);
+
+    /**
+     * The additional time in nanos that should elapse for some thread signal event to occur
+     * to simulate scheduler latency
+     */
+    long schedulerDelayNanos();
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/OrderOn.java b/test/simulator/main/org/apache/cassandra/simulator/OrderOn.java
new file mode 100644
index 0000000..382bd8c
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/OrderOn.java
@@ -0,0 +1,180 @@
+/*
+ * 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.simulator;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.utils.Shared;
+
+import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
+
+/**
+ * A token representing some ordering property of the system.
+ * Most notably this is used to implement executor services where a number of tasks may be run concurrently,
+ * particularly single threaded executors where causality is a strict requirement of correctness.
+ *
+ * This is also used to denote "strict" ordering on any suitably annotated {@link Action}.
+ *
+ * For convenience, efficiency and simplicity we have OrderOn represent a singleton collection of OrderOns.
+ */
+@Shared(scope = SIMULATION)
+public interface OrderOn extends OrderOns
+{
+    /**
+     * The number of {@link Action} in the provided sequence that may be executed at once,
+     * before any ordering takes effect.
+     */
+    int concurrency();
+
+    /**
+     * If true then all child actions (and their children, etc) must be ordered together, i.e. the next
+     * {@link Action} ordered by this sequence may not run until the present {@link Action} and all actions
+     * started by it, directly or indirectly, have completed.
+     */
+    default boolean isStrict() { return false; }
+
+    /**
+     * Whether the ordering is imposed immediately, occupying a slot in the sequence prior to any Action being scheduled
+     * (e.g. in the case of {@code executor.execute()}), or if it applies only after the scheduled time elapsed
+     * (e.g. in the case of {@code executor.schedule()}).
+     *
+     * This may be modified and still refer to the same {@code OrderOn} as another {@code OrderOn} by overriding
+     * the {@code unwrap()} method
+     */
+    default boolean appliesBeforeScheduling() { return true; }
+
+    /**
+     * {@code this} may be a thin wrapper around another {@code OrderOn} with a different {@code appliesBeforeScheduling()}.
+     * In this case this method returns the underlying {@code OrderOn} to impose the order upon.
+     */
+    default OrderOn unwrap() { return this; }
+
+    /**
+     * A convenience method to indicate if this {@code OrderOn} imposes any ordering
+     */
+    @Override
+    default boolean isOrdered() { return concurrency() < Integer.MAX_VALUE; }
+
+    @Override
+    default OrderOns with(OrderOn add)
+    {
+        return new TwoOrderOns(this, add);
+    }
+
+    @Override
+    default int size()
+    {
+        return 1;
+    }
+
+    @Override
+    default OrderOn get(int i)
+    {
+        Preconditions.checkArgument(i == 0);
+        return this;
+    }
+
+    abstract class OrderOnId implements OrderOn
+    {
+        public final Object id;
+
+        public OrderOnId(Object id)
+        {
+            this.id = id;
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return id.hashCode();
+        }
+
+        @Override
+        public boolean equals(Object that)
+        {
+            return that instanceof OrderOnId && id.equals(((OrderOnId) that).id);
+        }
+
+        public String toString()
+        {
+            return id.toString();
+        }
+    }
+
+    public class Sequential extends OrderOnId
+    {
+        public Sequential(Object id)
+        {
+            super(id);
+        }
+
+        public int concurrency() { return 1; }
+    }
+
+    public class StrictSequential extends Sequential
+    {
+        public StrictSequential(Object id)
+        {
+            super(id);
+        }
+
+        @Override
+        public boolean isStrict()
+        {
+            return true;
+        }
+    }
+
+    public class Strict extends Sequential
+    {
+        final int concurrency;
+
+        public Strict(Object id, int concurrency)
+        {
+            super(id);
+            this.concurrency = concurrency;
+        }
+
+        @Override
+        public int concurrency()
+        {
+            return concurrency;
+        }
+
+        @Override
+        public boolean isStrict()
+        {
+            return true;
+        }
+    }
+
+    public class OrderAppliesAfterScheduling implements OrderOn
+    {
+        final OrderOn inner;
+
+        public OrderAppliesAfterScheduling(OrderOn inner) { this.inner = inner; }
+        @Override public int concurrency() { return inner.concurrency(); }
+        @Override public boolean isStrict() { return inner.isStrict(); }
+        @Override public boolean isOrdered() { return inner.isOrdered(); }
+
+        @Override public boolean appliesBeforeScheduling() { return false; }
+        @Override public OrderOn unwrap() { return inner; }
+        @Override public String toString() { return inner.toString(); }
+    }
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/OrderOns.java b/test/simulator/main/org/apache/cassandra/simulator/OrderOns.java
new file mode 100644
index 0000000..19a2f4b
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/OrderOns.java
@@ -0,0 +1,160 @@
+/*
+ * 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.simulator;
+
+import java.util.ArrayList;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A (possibly empty) collection of OrderOn
+ */
+public interface OrderOns
+{
+    /**
+     * Equivalent to !isEmpty()
+     */
+    boolean isOrdered();
+
+    /**
+     * Equivalent to anyMatch(OrderOn::isOrdered)
+     */
+    boolean isStrict();
+
+    /**
+     * Return an {@code OrderOns} (possibly this one) also containing {@code add}
+     */
+    OrderOns with(OrderOn add);
+
+    /**
+     * The number of {@link OrderOn} contained in this collection
+     */
+    int size();
+
+    /**
+     * The i'th {@link OrderOn} contained in this collection
+     */
+    OrderOn get(int i);
+
+    public static final OrderOn NONE = new OrderOn()
+    {
+        @Override
+        public OrderOns with(OrderOn add)
+        {
+            return add;
+        }
+
+        @Override
+        public int size()
+        {
+            return 0;
+        }
+
+        @Override
+        public OrderOn get(int i)
+        {
+            throw new IndexOutOfBoundsException();
+        }
+
+        @Override
+        public int concurrency()
+        {
+            return Integer.MAX_VALUE;
+        }
+
+        @Override
+        public String toString()
+        {
+            return "Unordered";
+        }
+    };
+
+    public class TwoOrderOns implements OrderOns
+    {
+        final OrderOn one;
+        final OrderOn two;
+
+        public TwoOrderOns(OrderOn one, OrderOn two)
+        {
+            this.one = one;
+            this.two = two;
+        }
+
+        @Override
+        public boolean isOrdered()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean isStrict()
+        {
+            return one.isStrict() || two.isStrict();
+        }
+
+        @Override
+        public OrderOns with(OrderOn three)
+        {
+            OrderOnsList result = new OrderOnsList();
+            result.add(one);
+            result.add(two);
+            result.add(three);
+            return result;
+        }
+
+        @Override
+        public int size()
+        {
+            return 2;
+        }
+
+        @Override
+        public OrderOn get(int i)
+        {
+            Preconditions.checkArgument((i & 1) == i);
+            return i == 0 ? one : two;
+        }
+    }
+
+    public class OrderOnsList extends ArrayList<OrderOn> implements OrderOns
+    {
+        @Override
+        public boolean isOrdered()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean isStrict()
+        {
+            for (int i = 0 ; i < size() ; ++i)
+            {
+                if (get(i).isStrict())
+                    return true;
+            }
+            return false;
+        }
+
+        public OrderOns with(OrderOn add)
+        {
+            add(add);
+            return this;
+        }
+    }
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/Ordered.java b/test/simulator/main/org/apache/cassandra/simulator/Ordered.java
new file mode 100644
index 0000000..0281c04
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/Ordered.java
@@ -0,0 +1,331 @@
+/*
+ * 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.simulator;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.function.Function;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.simulator.utils.CountingCollection;
+import org.apache.cassandra.simulator.utils.IntrusiveLinkedList;
+import org.apache.cassandra.simulator.utils.IntrusiveLinkedListNode;
+
+import static java.util.Collections.newSetFromMap;
+import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_SIMULATOR_DEBUG;
+
+/**
+ * Represents an action that may not run before certain other actions
+ * have been executed, excluding child tasks that are not continuations
+ * (i.e. required threads/tasks to terminate their execution, but not
+ * any other child or transitive child actions)
+ */
+class Ordered extends OrderedLink implements ActionListener
+{
+    static final boolean DEBUG = TEST_SIMULATOR_DEBUG.getBoolean();
+
+    /**
+     * A sequence is used to model STRICT execution order imposed on certain actions that are not able
+     * to reliably complete if their actions are re-ordered, and to implement thread executor order,
+     * both for sequential executors and for ensuring executors with a given concurrency level do not
+     * exceed that concurrency level.
+     */
+    static class Sequence
+    {
+        final OrderOn on;
+        final int concurrency;
+        /** The tasks we are currently permitting to run (but may not be running due to membership of other sequences) */
+        final Collection<Ordered> maybeRunning;
+        /** The tasks we have pending */
+        final IntrusiveLinkedList<OrderedLink> next = new IntrusiveLinkedList<>();
+
+        Sequence(OrderOn on)
+        {
+            this.on = on;
+            this.concurrency = on.concurrency();
+            this.maybeRunning = !DEBUG ? new CountingCollection<>()
+                                       : concurrency == 1
+                                         ? new ArrayList<>(1)
+                                         : new LinkedHashSet<>();
+        }
+
+        <O extends Ordered> void add(O add, Function<O, List<Sequence>> memberOf)
+        {
+            memberOf.apply(add).add(this);
+            if (maybeRunning.size() < concurrency)
+            {
+                maybeRunning.add(add);
+            }
+            else
+            {
+                if (add.isFree())
+                {
+                    next.add(add);
+                }
+                else
+                {
+                    Preconditions.checkState(add.additionalLink == null);
+                    add.additionalLink = new AdditionalOrderedLink(add);
+                    next.add(add.additionalLink);
+                }
+
+                add.predecessors.add(this); // we don't submit, as we may yet be added to other sequences that prohibit our execution
+            }
+        }
+
+        /**
+         * Mark a task complete, and maybe schedule another from {@link #next}
+         */
+        void complete(Ordered completed, ActionSchedule schedule)
+        {
+            if (!maybeRunning.remove(completed))
+                throw new IllegalStateException();
+
+            complete(schedule);
+        }
+
+        void invalidate(Ordered completed, ActionSchedule schedule)
+        {
+            if (maybeRunning.remove(completed))
+                complete(schedule);
+        }
+
+        void invalidatePending()
+        {
+            if (next.isEmpty())
+                return;
+
+            List<Ordered> invalidate = new ArrayList<>();
+            for (OrderedLink link = next.poll() ; link != null ; link = next.poll())
+                invalidate.add(link.ordered());
+            invalidate.forEach(Ordered::invalidate);
+        }
+
+        void complete(ActionSchedule schedule)
+        {
+            if (next.isEmpty() && maybeRunning.isEmpty())
+            {
+                schedule.sequences.remove(on);
+            }
+            else
+            {
+                OrderedLink nextLink = this.next.poll();
+                if (nextLink != null)
+                {
+                    Ordered next = nextLink.ordered();
+                    if (!next.predecessors.remove(this))
+                        throw new IllegalStateException();
+                    maybeRunning.add(next);
+                    next.maybeAdvance();
+                }
+            }
+        }
+
+        public String toString()
+        {
+            return on.toString();
+        }
+    }
+
+    /**
+     * Represents an action that may not run before all child actions
+     * have been executed, transitively (i.e. child of child, ad infinitum).
+     */
+    static class StrictlyOrdered extends Ordered implements ActionListener
+    {
+        /** The sequences we participate in, in a strict fashion */
+        final List<Sequence> strictMemberOf = new ArrayList<>(1);
+        boolean isCompleteStrict;
+
+        StrictlyOrdered(Action action, ActionSchedule schedule)
+        {
+            super(action, schedule);
+        }
+
+        @Override
+        public void transitivelyAfter(Action finished)
+        {
+            assert !isCompleteStrict;
+            isCompleteStrict = true;
+            strictMemberOf.forEach(m -> m.complete(this, schedule));
+        }
+
+        @Override
+        void invalidate(boolean isCancellation)
+        {
+            super.invalidate(isCancellation);
+            strictMemberOf.forEach(m -> m.invalidate(this, schedule));
+        }
+
+        @Override
+        void joinNow(OrderOn orderOn)
+        {
+            schedule.sequences.computeIfAbsent(orderOn.unwrap(), Sequence::new)
+                              .add(this, orderOn.isStrict() ? o -> o.strictMemberOf : o -> o.memberOf);
+        }
+    }
+
+    final ActionSchedule schedule;
+    /** Those sequences that contain tasks that must complete before we can execute */
+    final Collection<Sequence> predecessors = !DEBUG ? new CountingCollection<>() : newSetFromMap(new IdentityHashMap<>());
+
+    /** The sequences we participate in, in a non-strict fashion */
+    final List<Sequence> memberOf = new ArrayList<>(1);
+    /** The underlying action waiting to execute */
+    final Action action;
+    /** State tracking to assert correct behaviour */
+    boolean isStarted, isComplete;
+    List<OrderOn> joinPostScheduling;
+    OrderedLink additionalLink;
+
+    Ordered(Action action, ActionSchedule schedule)
+    {
+        this.schedule = schedule;
+        this.action = action;
+        action.register(this);
+    }
+
+    public String toString()
+    {
+        return action.toString();
+    }
+
+    public void before(Action performed, Before before)
+    {
+        switch (before)
+        {
+            default: throw new AssertionError();
+            case INVALIDATE: // will be handled by invalidate()
+                return;
+            case DROP:
+            case EXECUTE:
+                assert performed == action;
+                assert !isStarted;
+                isStarted = true;
+        }
+    }
+
+    void join(OrderOn orderOn)
+    {
+        if (!orderOn.isOrdered())
+            return;
+
+        if (orderOn.appliesBeforeScheduling()) joinNow(orderOn);
+        else joinPostScheduling(orderOn);
+    }
+
+    void joinNow(OrderOn orderOn)
+    {
+        schedule.sequences.computeIfAbsent(orderOn.unwrap(), Sequence::new)
+                          .add(this, o -> o.memberOf);
+    }
+
+    void joinPostScheduling(OrderOn orderOn)
+    {
+        if (joinPostScheduling == null)
+        {
+            joinPostScheduling = Collections.singletonList(orderOn);
+        }
+        else
+        {
+            if (joinPostScheduling.size() == 1)
+            {
+                List<OrderOn> tmp = new ArrayList<>(2);
+                tmp.addAll(joinPostScheduling);
+                joinPostScheduling = tmp;
+            }
+            joinPostScheduling.add(orderOn);
+        }
+    }
+
+    boolean waitPreScheduled()
+    {
+        return !predecessors.isEmpty();
+    }
+
+    boolean waitPostScheduled()
+    {
+        Preconditions.checkState(predecessors.isEmpty());
+        if (joinPostScheduling == null)
+            return false;
+        joinPostScheduling.forEach(this::joinNow);
+        joinPostScheduling = null;
+        return !predecessors.isEmpty();
+    }
+
+    void invalidate()
+    {
+        invalidate(false);
+    }
+
+    void invalidate(boolean isCancellation)
+    {
+        Preconditions.checkState(!isCancellation || !isStarted);
+        isStarted = isComplete = true;
+        action.deregister(this);
+        remove();
+        if (additionalLink != null)
+        {
+            additionalLink.remove();
+            additionalLink = null;
+        }
+        memberOf.forEach(m -> m.invalidate(this, schedule));
+    }
+
+    void maybeAdvance()
+    {
+        if (predecessors.isEmpty())
+            schedule.advance(action);
+    }
+
+    @Override
+    public void after(Action performed)
+    {
+        assert isStarted;
+        assert !isComplete;
+        isComplete = true;
+        memberOf.forEach(m -> m.complete(this, schedule));
+    }
+
+    @Override
+    Ordered ordered()
+    {
+        return this;
+    }
+}
+
+abstract class OrderedLink extends IntrusiveLinkedListNode
+{
+    abstract Ordered ordered();
+    public void remove() { super.remove(); }
+    public boolean isFree() { return super.isFree(); }
+}
+
+class AdditionalOrderedLink extends OrderedLink
+{
+    final Ordered ordered;
+
+    AdditionalOrderedLink(Ordered ordered) { this.ordered = ordered; }
+    Ordered ordered() { return ordered; }
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/OrderedOn.java b/test/simulator/main/org/apache/cassandra/simulator/OrderedOn.java
new file mode 100644
index 0000000..b8dc81f
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/OrderedOn.java
@@ -0,0 +1,32 @@
+/*
+ * 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.simulator;
+
+import org.apache.cassandra.utils.Shared;
+
+import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
+
+@Shared(scope = SIMULATION)
+public interface OrderedOn
+{
+    OrderOn on();
+    default boolean appliesBeforeScheduling() { return true; }
+    default boolean isOrdered() { return on().isOrdered(); }
+    default boolean isStrict() { return on().isStrict(); }
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/RandomSource.java b/test/simulator/main/org/apache/cassandra/simulator/RandomSource.java
new file mode 100644
index 0000000..14d7ad9
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/RandomSource.java
@@ -0,0 +1,320 @@
+/*
+ * 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.simulator;
+
+import java.lang.reflect.Array;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Random;
+import java.util.function.IntSupplier;
+import java.util.function.LongSupplier;
+import java.util.stream.IntStream;
+import java.util.stream.LongStream;
+
+import org.apache.cassandra.utils.Shared;
+
+import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
+
+@Shared(scope = SIMULATION)
+public interface RandomSource
+{
+    public static class Choices<T>
+    {
+        final float[] cumulativeProbabilities;
+        public final T[] options;
+
+        private Choices(float[] cumulativeProbabilities, T[] options)
+        {
+            this.cumulativeProbabilities = cumulativeProbabilities;
+            this.options = options;
+        }
+
+        public T choose(RandomSource random)
+        {
+            if (options.length == 0)
+                return null;
+
+            float choose = random.uniformFloat();
+            int i = Arrays.binarySearch(cumulativeProbabilities, choose);
+
+            if (i < 0) i = -1 - i;
+            return options[i];
+        }
+
+        public Choices<T> without(T option)
+        {
+            for (int i = 0 ; i < options.length ; ++i)
+            {
+                if (option.equals(options[i]))
+                {
+                    float[] prob = new float[cumulativeProbabilities.length - 1];
+                    T[] opts = (T[]) Array.newInstance(options.getClass().getComponentType(), options.length - 1);
+                    System.arraycopy(cumulativeProbabilities, 0, prob, 0, i);
+                    System.arraycopy(cumulativeProbabilities, i + 1, prob, i, this.options.length - (i + 1));
+                    System.arraycopy(options, 0, opts, 0, i);
+                    System.arraycopy(options, i + 1, opts, i, options.length - (i + 1));
+                    for (int j = prob.length - 1 ; j > 1 ; --j)
+                        prob[j] -= prob[j - 1];
+                    return build(prob, opts);
+                }
+            }
+            return this;
+        }
+
+        private static float[] randomCumulativeProbabilities(RandomSource random, int count)
+        {
+            float[] nonCumulativeProbabilities = new float[count];
+            for (int i = 0 ; i < count ; ++i)
+                nonCumulativeProbabilities[i] = random.uniformFloat();
+            return cumulativeProbabilities(nonCumulativeProbabilities);
+        }
+
+        private static float[] cumulativeProbabilities(float[] nonCumulativeProbabilities)
+        {
+            int count = nonCumulativeProbabilities.length;
+            if (count == 0)
+                return new float[0];
+
+            float[] result = new float[nonCumulativeProbabilities.length];
+            float sum = 0;
+            for (int i = 0 ; i < count ; ++i)
+                result[i] = sum += nonCumulativeProbabilities[i];
+            result[result.length - 1] = 1.0f;
+            for (int i = 0 ; i < count - 1 ; ++i)
+                result[i] = result[i] /= sum;
+            return result;
+        }
+
+        public static <T> Choices<T> random(RandomSource random, T[] options)
+        {
+            return new Choices<>(randomCumulativeProbabilities(random, options.length), options);
+        }
+
+        public static <T> Choices<T> random(RandomSource random, T[] options, Map<T, float[]> bounds)
+        {
+            float[] nonCumulativeProbabilities = new float[options.length];
+            for (int i = 0 ; i < options.length ; ++i)
+            {
+                float[] minmax = bounds.get(options[i]);
+                float uniform = random.uniformFloat();
+                nonCumulativeProbabilities[i] = minmax == null ? uniform : minmax[0] + (uniform * (minmax[1] - minmax[0]));
+            }
+            return new Choices<>(cumulativeProbabilities(nonCumulativeProbabilities), options);
+        }
+
+        public static <T> Choices<T> build(float[] nonCumulativeProbabilities, T[] options)
+        {
+            if (nonCumulativeProbabilities.length != options.length)
+                throw new IllegalArgumentException();
+            return new Choices<>(cumulativeProbabilities(nonCumulativeProbabilities), options);
+        }
+
+        public static <T> Choices<T> uniform(T ... options)
+        {
+            float[] nonCumulativeProbabilities = new float[options.length];
+            Arrays.fill(nonCumulativeProbabilities, 1f / options.length);
+            return new Choices<>(cumulativeProbabilities(nonCumulativeProbabilities), options);
+        }
+    }
+
+    public static abstract class Abstract implements RandomSource
+    {
+        public abstract float uniformFloat();
+        public abstract int uniform(int min, int max);
+        public abstract long uniform(long min, long max);
+
+        public LongSupplier uniqueUniformSupplier(long min, long max)
+        {
+            return uniqueUniformStream(min, max).iterator()::nextLong;
+        }
+
+        public LongStream uniqueUniformStream(long min, long max)
+        {
+            return uniformStream(min, max).distinct();
+        }
+
+        public LongStream uniformStream(long min, long max)
+        {
+            return LongStream.generate(() -> uniform(min, max));
+        }
+
+        public LongSupplier uniformSupplier(long min, long max)
+        {
+            return () -> uniform(min, max);
+        }
+
+        public IntSupplier uniqueUniformSupplier(int min, int max)
+        {
+            return uniqueUniformStream(min, max).iterator()::nextInt;
+        }
+
+        public IntStream uniqueUniformStream(int min, int max)
+        {
+            return uniformStream(min, max).distinct();
+        }
+
+        public IntStream uniformStream(int min, int max)
+        {
+            return IntStream.generate(() -> uniform(min, max));
+        }
+
+        public boolean decide(float chance)
+        {
+            return uniformFloat() < chance;
+        }
+
+        public int log2uniform(int min, int max)
+        {
+            return (int) log2uniform((long) min, max);
+        }
+
+        public long log2uniform(long min, long max)
+        {
+            return qlog2uniform(min, max, 64);
+        }
+
+        public long qlog2uniform(long min, long max, int quantizations)
+        {
+            return min + log2uniform(max - min, quantizations);
+        }
+
+        private long log2uniform(long max, int quantizations)
+        {
+            int maxBits = 64 - Long.numberOfLeadingZeros(max - 1);
+            if (maxBits == 0)
+                return 0;
+
+            long min;
+            if (maxBits <= quantizations)
+            {
+                int bits = uniform(0, maxBits);
+                min = 1L << (bits - 1);
+                max = Math.min(max, min * 2);
+            }
+            else
+            {
+                int bitsPerRange = (maxBits / quantizations);
+                int i = uniform(0, quantizations);
+                min = 1L << (i * bitsPerRange);
+                max = Math.min(max, 1L << ((i + 1) * bitsPerRange));
+            }
+
+            return uniform(min, max);
+        }
+
+        public float qlog2uniformFloat(int quantizations)
+        {
+            return qlog2uniform(0, 1 << 24, quantizations) / (float)(1 << 24);
+        }
+    }
+
+    public static class Default extends Abstract
+    {
+        private final Random random = new Random(0);
+
+        public float uniformFloat() { return random.nextFloat(); }
+
+        @Override
+        public double uniformDouble()
+        {
+            return random.nextDouble();
+        }
+
+        public int uniform(int min, int max)
+        {
+            int delta = max - min;
+            if (delta > 1) return min + random.nextInt(max - min);
+            if (delta == 1) return min;
+            if (min >= max)
+                throw new IllegalArgumentException(String.format("Min (%s) should be less than max (%d).", min, max));
+            return (int)uniform(min, (long)max);
+        }
+
+        public long uniform(long min, long max)
+        {
+            if (min >= max) throw new IllegalArgumentException();
+
+            long delta = max - min;
+            if (delta == 1) return min;
+            if (delta == Long.MIN_VALUE && max == Long.MAX_VALUE) return random.nextLong();
+            if (delta < 0) return random.longs(min, max).iterator().nextLong();
+            if (delta <= Integer.MAX_VALUE) return min + uniform(0, (int) delta);
+
+            long result = min + 1 == max ? min : min + ((random.nextLong() & 0x7fffffff) % (max - min));
+            assert result >= min && result < max;
+            return result;
+        }
+
+        public void reset(long seed)
+        {
+            random.setSeed(seed);
+        }
+
+        public long reset()
+        {
+            long seed = random.nextLong();
+            reset(seed);
+            return seed;
+        }
+    }
+
+    IntStream uniqueUniformStream(int min, int max);
+
+    LongSupplier uniqueUniformSupplier(long min, long max);
+    LongStream uniqueUniformStream(long min, long max);
+    LongStream uniformStream(long min, long max);
+
+    // [min...max)
+    int uniform(int min, int max);
+    // [min...max)
+    long uniform(long min, long max);
+
+    /**
+     * Select a number in the range [min, max), with a power of two in the range [0, max-min)
+     * selected uniformly and a uniform value less than this power of two added to it
+     */
+    int log2uniform(int min, int max);
+    long log2uniform(long min, long max);
+
+    /**
+     * Select a number in the range [min, max), with the range being split into
+     * {@code quantizations} adjacent powers of two, a range being select from these
+     * with uniform probability, and the value within that range being selected uniformly
+     */
+    long qlog2uniform(long min, long max, int quantizations);
+
+    float uniformFloat();
+
+    /**
+     * Select a number in the range [0, 1), with the range being split into
+     * {@code quantizations} adjacent powers of two; a range being select from these
+     * with uniform probability, and the value within that range being selected uniformly
+     *
+     * This is used to distribute behavioural toggles more extremely between different runs of the simulator.
+     */
+    float qlog2uniformFloat(int quantizations);
+    double uniformDouble();
+
+    // options should be cumulative probability in range [0..1]
+    boolean decide(float chance);
+
+    void reset(long seed);
+    long reset();
+}
+
diff --git a/test/simulator/main/org/apache/cassandra/simulator/RunnableActionScheduler.java b/test/simulator/main/org/apache/cassandra/simulator/RunnableActionScheduler.java
new file mode 100644
index 0000000..9609e66
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/RunnableActionScheduler.java
@@ -0,0 +1,141 @@
+/*
+ * 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.simulator;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+
+import org.apache.cassandra.simulator.utils.KindOfSequence;
+
+public abstract class RunnableActionScheduler implements Consumer<Action>
+{
+    public enum Kind { RANDOM_WALK, UNIFORM, SEQUENTIAL }
+
+    public static class Immediate extends RunnableActionScheduler
+    {
+        private final AtomicLong id = new AtomicLong();
+
+        public Immediate() { }
+
+        @Override
+        public double priority()
+        {
+            return id.incrementAndGet();
+        }
+    }
+
+    public static class Sequential extends RunnableActionScheduler
+    {
+        final AtomicInteger next = new AtomicInteger();
+
+        @Override
+        public double priority()
+        {
+            return next.incrementAndGet();
+        }
+
+        public Sequential() { }
+    }
+
+    public abstract static class AbstractRandom extends RunnableActionScheduler
+    {
+        protected final RandomSource random;
+
+        public AbstractRandom(RandomSource random)
+        {
+            this.random = random;
+        }
+    }
+
+    public static class RandomUniform extends AbstractRandom
+    {
+        final double min, range;
+
+        RandomUniform(RandomSource random, double min, double range)
+        {
+            super(random);
+            this.min = min;
+            this.range = range;
+        }
+
+        public RandomUniform(RandomSource random)
+        {
+            this(random, 0d, 1d);
+        }
+
+        @Override
+        public double priority()
+        {
+            return min + random.uniformDouble() * range;
+        }
+    }
+
+    static class RandomWalk extends AbstractRandom
+    {
+        final double maxStepSize;
+        double cur;
+
+        @Override
+        public double priority()
+        {
+            double result = cur;
+            double step = (2*random.uniformDouble() - 1f) * maxStepSize;
+            this.cur = step > 0 ? Math.min(1d, cur + step)
+                                : Math.max(0d, cur + step);
+            return result;
+        }
+
+        @Override
+        protected RunnableActionScheduler next()
+        {
+            return new RunnableActionScheduler.RandomWalk(random, cur);
+        }
+
+        RandomWalk(RandomSource random, double cur)
+        {
+            super(random);
+            this.maxStepSize = KindOfSequence.maxStepSize(0f, 1f, random);
+            this.cur = cur;
+        }
+
+        RandomWalk(RandomSource random)
+        {
+            this(random, 0.5d);
+        }
+    }
+
+    public abstract double priority();
+
+    protected RunnableActionScheduler next()
+    {
+        return this;
+    }
+
+    public void attachTo(ActionList actions)
+    {
+        actions.forEach(next());
+    }
+
+    @Override
+    public void accept(Action action)
+    {
+        action.setScheduler(this);
+    }
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/Simulation.java b/test/simulator/main/org/apache/cassandra/simulator/Simulation.java
new file mode 100644
index 0000000..8fd1532
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/Simulation.java
@@ -0,0 +1,27 @@
+/*
+ * 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.simulator;
+
+import org.apache.cassandra.utils.CloseableIterator;
+
+public interface Simulation extends AutoCloseable
+{
+    CloseableIterator<?> iterator();
+    void run();
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/SimulationRunner.java b/test/simulator/main/org/apache/cassandra/simulator/SimulationRunner.java
new file mode 100644
index 0000000..9371844
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/SimulationRunner.java
@@ -0,0 +1,456 @@
+/*
+ * 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.simulator;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.ToDoubleFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.airlift.airline.Command;
+import io.airlift.airline.Help;
+import io.airlift.airline.Option;
+import io.netty.util.concurrent.FastThreadLocal;
+import org.apache.cassandra.config.CassandraRelevantProperties;
+import org.apache.cassandra.simulator.Debug.Info;
+import org.apache.cassandra.simulator.Debug.Levels;
+import org.apache.cassandra.simulator.cluster.ClusterActions.TopologyChange;
+import org.apache.cassandra.simulator.debug.Capture;
+import org.apache.cassandra.simulator.debug.SelfReconcile;
+import org.apache.cassandra.simulator.systems.InterceptedWait;
+import org.apache.cassandra.simulator.systems.InterceptibleThread;
+import org.apache.cassandra.simulator.systems.InterceptorOfGlobalMethods;
+import org.apache.cassandra.simulator.utils.ChanceRange;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.Hex;
+import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
+
+import static java.util.Arrays.stream;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.cassandra.config.CassandraRelevantProperties.BATCH_COMMIT_LOG_SYNC_INTERVAL;
+import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_REMOTE_PORT;
+import static org.apache.cassandra.config.CassandraRelevantProperties.CLOCK_GLOBAL;
+import static org.apache.cassandra.config.CassandraRelevantProperties.CLOCK_MONOTONIC_APPROX;
+import static org.apache.cassandra.config.CassandraRelevantProperties.CLOCK_MONOTONIC_PRECISE;
+import static org.apache.cassandra.config.CassandraRelevantProperties.DETERMINISM_CONSISTENT_DIRECTORY_LISTINGS;
+import static org.apache.cassandra.config.CassandraRelevantProperties.DETERMINISM_UNSAFE_UUID_NODE;
+import static org.apache.cassandra.config.CassandraRelevantProperties.DISABLE_SSTABLE_ACTIVITY_TRACKING;
+import static org.apache.cassandra.config.CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT;
+import static org.apache.cassandra.config.CassandraRelevantProperties.GOSSIPER_SKIP_WAITING_TO_SETTLE;
+import static org.apache.cassandra.config.CassandraRelevantProperties.IGNORE_MISSING_NATIVE_FILE_HINTS;
+import static org.apache.cassandra.config.CassandraRelevantProperties.IS_DISABLED_MBEAN_REGISTRATION;
+import static org.apache.cassandra.config.CassandraRelevantProperties.MEMTABLE_OVERHEAD_SIZE;
+import static org.apache.cassandra.config.CassandraRelevantProperties.MIGRATION_DELAY;
+import static org.apache.cassandra.config.CassandraRelevantProperties.PAXOS_REPAIR_RETRY_TIMEOUT_IN_MS;
+import static org.apache.cassandra.config.CassandraRelevantProperties.RING_DELAY;
+import static org.apache.cassandra.config.CassandraRelevantProperties.SHUTDOWN_ANNOUNCE_DELAY_IN_MS;
+import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_AUTH_DEFAULT_RF;
+import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_IGNORE_SIGAR;
+import static org.apache.cassandra.config.CassandraRelevantProperties.DISABLE_GOSSIP_ENDPOINT_REMOVAL;
+import static org.apache.cassandra.simulator.debug.Reconcile.reconcileWith;
+import static org.apache.cassandra.simulator.debug.Record.record;
+import static org.apache.cassandra.simulator.debug.SelfReconcile.reconcileWithSelf;
+import static org.apache.cassandra.simulator.utils.IntRange.parseRange;
+import static org.apache.cassandra.simulator.utils.LongRange.parseNanosRange;
+
+@SuppressWarnings({ "ZeroLengthArrayAllocation", "CodeBlock2Expr", "SameParameterValue", "DynamicRegexReplaceableByCompiledPattern", "CallToSystemGC" })
+public class SimulationRunner
+{
+    private static final Logger logger = LoggerFactory.getLogger(SimulationRunner.class);
+
+    @BeforeClass
+    public static void beforeAll()
+    {
+        // setup system properties for our instances to behave correctly and consistently/deterministically
+
+        // Disallow time on the bootstrap classloader
+        for (CassandraRelevantProperties property : Arrays.asList(CLOCK_GLOBAL, CLOCK_MONOTONIC_APPROX, CLOCK_MONOTONIC_PRECISE))
+            property.setString("org.apache.cassandra.simulator.systems.SimulatedTime$Throwing");
+        try { Clock.Global.nanoTime(); } catch (IllegalStateException e) {} // make sure static initializer gets called
+
+        // TODO (cleanup): disable unnecessary things like compaction logger threads etc
+        System.setProperty("cassandra.libjemalloc", "-");
+        System.setProperty("cassandra.dtest.api.log.topology", "false");
+
+        // this property is used to allow non-members of the ring to exist in gossip without breaking RF changes
+        // it would be nice not to rely on this, but hopefully we'll have consistent range movements before it matters
+        System.setProperty("cassandra.allow_alter_rf_during_range_movement", "true");
+
+        for (CassandraRelevantProperties property : Arrays.asList(CLOCK_GLOBAL, CLOCK_MONOTONIC_APPROX, CLOCK_MONOTONIC_PRECISE))
+            property.setString("org.apache.cassandra.simulator.systems.SimulatedTime$Global");
+
+        CASSANDRA_JMX_REMOTE_PORT.setString("");
+        RING_DELAY.setInt(0);
+        PAXOS_REPAIR_RETRY_TIMEOUT_IN_MS.setLong(NANOSECONDS.toMillis(Long.MAX_VALUE));
+        SHUTDOWN_ANNOUNCE_DELAY_IN_MS.setInt(0);
+        DETERMINISM_UNSAFE_UUID_NODE.setBoolean(true);
+        GOSSIPER_SKIP_WAITING_TO_SETTLE.setInt(0);
+        BATCH_COMMIT_LOG_SYNC_INTERVAL.setInt(-1);
+        DISABLE_SSTABLE_ACTIVITY_TRACKING.setBoolean(false);
+        DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.setBoolean(false); // compression causes variation in file size for e.g. UUIDs, IP addresses, random file paths
+        DETERMINISM_CONSISTENT_DIRECTORY_LISTINGS.setBoolean(true);
+        TEST_IGNORE_SIGAR.setBoolean(true);
+        SYSTEM_AUTH_DEFAULT_RF.setInt(3);
+        MIGRATION_DELAY.setInt(Integer.MAX_VALUE);
+        DISABLE_GOSSIP_ENDPOINT_REMOVAL.setBoolean(true);
+        MEMTABLE_OVERHEAD_SIZE.setInt(100);
+        IGNORE_MISSING_NATIVE_FILE_HINTS.setBoolean(true);
+        IS_DISABLED_MBEAN_REGISTRATION.setBoolean(true);
+
+        if (Thread.currentThread() instanceof InterceptibleThread); // load InterceptibleThread class to avoid infinite loop in InterceptorOfGlobalMethods
+        new InterceptedWait.CaptureSites(Thread.currentThread(), false)
+        .toString(ste -> !ste.getClassName().equals(SelfReconcile.class.getName())); // ensure self reconcile verify can work without infinite looping
+        InterceptorOfGlobalMethods.Global.unsafeReset();
+        ThreadLocalRandom.current();
+    }
+
+    protected interface ICommand<B extends ClusterSimulation.Builder<?>>
+    {
+        public void run(B builder) throws IOException;
+    }
+
+    protected abstract static class BasicCommand<B extends ClusterSimulation.Builder<?>> implements ICommand<B>
+    {
+        @Option(name = { "-s", "--seed"} , title = "0x|int", description = "Specify the first seed to test (each simulation will increment the seed by 1)")
+        protected String seed;
+
+        @Option(name = { "--simulations"} , title = "int", description = "The number of simulations to run")
+        protected int simulationCount = 1;
+
+        @Option(name = { "-t", "--threads" }, title = "int", description = "The number of threads to split between node thread pools. Each ongoing action also requires its own thread.")
+        protected int threadCount = 1000;
+
+        @Option(name = { "-n", "--nodes" } , title = "int...int", description = "Cluster size range, lb..ub (default 4..16)")
+        protected String nodeCount = "4..16";
+
+        @Option(name = { "--dcs" }, title = "int...int", description = "Cluster DC count, lb..ub (default 1..2)")
+        protected String dcCount = "1..2";
+
+        @Option(name = { "-o", "--within-key-concurrency" }, title = "int..int", description = "Number of simultaneous paxos operations per key, lb..ub (default 2..5)")
+        protected String withinKeyConcurrency = "2..5";
+
+        @Option(name = { "-c", "--concurrency" }, title = "int", description = "Number of keys to operate on simultaneously (default 10)")
+        protected int concurrency = 10;
+        @Option(name = { "-k", "--keys" }, title = "int", description = "Number of unique partition keys to operate over (default to 2* concurrency)")
+        protected int primaryKeyCount = -1;
+        @Option(name = { "--key-seconds" }, title = "int...int", description = "Number of seconds to simulate a partition key for before selecting another (default 5..30)")
+        protected String primaryKeySeconds = "5..30";
+
+        @Option(name = { "--cluster-actions" }, title = "JOIN,LEAVE,REPLACE,CHANGE_RF", description = "Cluster actions to select from, comma delimited (JOIN, LEAVE, REPLACE, CHANGE_RF)")
+        protected String topologyChanges = stream(TopologyChange.values()).map(Object::toString).collect(Collectors.joining(","));
+        @Option(name = { "--cluster-action-interval" }, title = "int...int(s|ms|us|ns)", description = "The period of time between two cluster actions (default 5..15s)")
+        protected String topologyChangeInterval = "5..15s";
+
+
+        @Option(name = {"--run-time"}, title = "int", description = "Length of simulated time to run in seconds (default -1)")
+        protected int secondsToSimulate = -1;
+
+        @Option(name = { "--reads" }, title = "[distribution:]float...float", description = "Proportion of actions that are reads (default: 0.05..0.95)")
+        protected String readChance;
+        @Option(name = { "--nemesis" }, title = "[distribution:]float...float", description = "Proportion of nemesis points that are intercepted (default: 0..0.01)")
+        protected String nemesisChance;
+
+        @Option(name = { "--priority" }, title = "uniform|randomwalk|seq", description = "Priority assignment for actions that may overlap their execution", allowedValues = { "uniform", "randomwalk", "seq" })
+        protected String priority;
+
+        // TODO (feature): simulate GC pauses
+
+        @Option(name = { "--network-flaky-chance" }, title = "[distribution:]float...float", description = "Chance of some minority of nodes experiencing flaky connections (default: qlog:0.01..0.1)")
+        protected String networkFlakyChance = "qlog:0.01..0.1";
+        @Option(name = { "--network-partition-chance" }, title = "[distribution:]float...float", description = "Chance of some minority of nodes being isolated (default: qlog:0.01..0.1)")
+        protected String networkPartitionChance = "qlog:0.01..0.1";
+        @Option(name = { "--network-reconfigure-interval" }, title = "int...int(s|ms|us|ns)", description = "Period of time for which a flaky or catastrophic network partition may be in force")
+        protected String networkReconfigureInterval = "1..10s";
+        @Option(name = { "--network-drop-chance" }, title = "[distribution:]float...float", description = "Chance of dropping a message under normal circumstances (default: qlog:0..0.001)")
+        protected String networkDropChance = "qlog:0..0.001";
+        // TODO (feature): TDP vs UDP simulation (right now we have no head of line blocking so we deliver in a UDP fashion which is not how the cluster operates)
+        @Option(name = { "--network-delay-chance" }, title = "[distribution:]float...float", description = "Chance of delaying a message under normal circumstances (default: qlog:0..0.1)")
+        protected String networkDelayChance = "qlog:0..0.01";
+        @Option(name = { "--network-latency" }, title = "int...int(s|ms|us|ns)", description = "Range of possible latencies messages can be simulated to experience (default 1..2ms)")
+        protected String networkLatency = "1..2ms";
+        @Option(name = { "--network-delay" }, title = "int...int(s|ms|us|ns)", description = "Range of possible latencies messages can be simulated to experience when delayed (default 2..20ms)")
+        protected String networkDelay = "2..20ms";
+        @Option(name = { "--network-flaky-drop-chance" }, title = "[distribution:]float...float", description = "Chance of dropping a message on a flaky connection (default: qlog:0.01..0.1)")
+        protected String flakyNetworkDropChance = "qlog:0.01..0.1";
+        @Option(name = { "--network-flaky-delay-chance" }, title = "[distribution:]float...float", description = "Chance of delaying a message on a flaky connection (default: qlog:0.01..0.2)")
+        protected String flakyNetworkDelayChance = "qlog:0.01..0.2";
+        @Option(name = { "--network-flaky-latency" }, title = "int...int(s|ms|us|ns)", description = "Range of possible latencies messages can be simulated to experience on a flaky connection (default 2..4ms)")
+        protected String flakyNetworkLatency = "2..4ms";
+        @Option(name = { "--network-flaky-delay" }, title = "int...int(s|ms|us|ns)", description = "Range of possible latencies messages can be simulated to experience when delayed on a flaky connection (default 4..100ms)")
+        protected String flakyNetworkDelay = "4..100ms";
+
+        @Option(name = { "--clock-drift" }, title = "int...int(s|ms|us|ns)", description = "The range of clock skews to experience (default 10..1000ms)")
+        protected String clockDrift = "10..1000ms";
+        @Option(name = { "--clock-discontinuity-interval" }, title = "int...int(s|ms|us|ns)", description = "The period of clock continuity (a discontinuity is a large jump of the global clock to introduce additional chaos for event scheduling) (default 10..60s)")
+        protected String clockDiscontinuityInterval = "10..60s";
+
+        @Option(name = { "--scheduler-jitter" }, title = "int...int(s|ms|us|ns)", description = "The scheduler will randomly prioritise all tasks scheduled to run within this interval (default 10..1500us)")
+        protected String schedulerJitter = "10..1500us";
+        @Option(name = { "--scheduler-delay-chance" }, title = "[distribution:]float...float", description = "Chance of delaying the consequence of an action (default: 0..0.1)")
+        protected String schedulerDelayChance = "qlog:0..0.1";
+        @Option(name = { "--scheduler-delay" }, title = "int...int(s|ms|us|ns)", description = "Range of possible additional latencies thread execution can be simulated to experience when delayed (default 1..10000us)")
+        protected String schedulerDelayMicros = "1..10000us";
+        @Option(name = { "--scheduler-long-delay" }, title = "int...int(s|ms|us|ns)", description = "Range of possible additional latencies thread execution can be simulated to experience when delayed (default 1..10000us)")
+        protected String schedulerLongDelayMicros = "1..10000us";
+
+        @Option(name = { "--log" }, title = "level", description = "<partition> <cluster> level events, between 0 and 2", arity = 2)
+        protected List<Integer> log;
+
+        @Option(name = { "--debug-keys" }, title = "level", description = "Print debug info only for these keys (comma delimited)")
+        protected String debugKeys;
+
+        @Option(name = { "--debug-rf" }, title = "level", description = "Print RF on <partition> <cluster> events; level 0 to 2", arity = 2, allowedValues = { "0", "1", "2" })
+        protected List<Integer> debugRf;
+
+        @Option(name = { "--debug-ownership" }, title = "level", description = "Print ownership on <partition> <cluster> events; level 0 to 2", arity = 2, allowedValues = { "0", "1", "2" })
+        protected List<Integer> debugOwnership;
+
+        @Option(name = { "--debug-ring" }, title = "level", description = "Print ring state on <partition> <cluster> events; level 0 to 2", arity = 2, allowedValues = { "0", "1", "2" })
+        protected List<Integer> debugRing;
+
+        @Option(name = { "--debug-gossip" }, title = "level", description = "Debug gossip at <partition> <cluster> events; level 0 to 2", arity = 2, allowedValues = { "0", "1", "2" })
+        protected List<Integer> debugGossip;
+
+        @Option(name = { "--debug-paxos" }, title = "level", description = "Print paxos state on <partition> <cluster> events; level 0 to 2", arity = 2, allowedValues = { "0", "1", "2" })
+        protected List<Integer> debugPaxos;
+
+        @Option(name = { "--capture" }, title = "wait,wake,now", description = "Capture thread stack traces alongside events, choose from (wait,wake,now)")
+        protected String capture;
+
+        protected void propagate(B builder)
+        {
+            builder.threadCount(threadCount);
+            builder.concurrency(concurrency);
+            if (primaryKeyCount >= 0) builder.primaryKeyCount(primaryKeyCount);
+            else builder.primaryKeyCount(2 * concurrency);
+            builder.secondsToSimulate(secondsToSimulate);
+            parseChanceRange(Optional.ofNullable(networkPartitionChance)).ifPresent(builder::networkPartitionChance);
+            parseChanceRange(Optional.ofNullable(networkFlakyChance)).ifPresent(builder::networkFlakyChance);
+            parseNanosRange(Optional.ofNullable(networkReconfigureInterval)).ifPresent(builder::networkReconfigureInterval);
+            parseChanceRange(Optional.ofNullable(networkDropChance)).ifPresent(builder::networkDropChance);
+            parseChanceRange(Optional.ofNullable(networkDelayChance)).ifPresent(builder::networkDelayChance);
+            parseNanosRange(Optional.ofNullable(networkLatency)).ifPresent(builder::networkLatencyNanos);
+            parseNanosRange(Optional.ofNullable(networkDelay)).ifPresent(builder::networkDelayNanos);
+            parseChanceRange(Optional.ofNullable(flakyNetworkDropChance)).ifPresent(builder::flakyNetworkDropChance);
+            parseChanceRange(Optional.ofNullable(flakyNetworkDelayChance)).ifPresent(builder::flakyNetworkDelayChance);
+            parseNanosRange(Optional.ofNullable(flakyNetworkLatency)).ifPresent(builder::flakyNetworkLatencyNanos);
+            parseNanosRange(Optional.ofNullable(flakyNetworkDelay)).ifPresent(builder::flakyNetworkDelayNanos);
+            parseChanceRange(Optional.ofNullable(schedulerDelayChance)).ifPresent(builder::schedulerDelayChance);
+            parseNanosRange(Optional.ofNullable(clockDrift)).ifPresent(builder::clockDriftNanos);
+            parseNanosRange(Optional.ofNullable(clockDiscontinuityInterval)).ifPresent(builder::clockDiscontinuityIntervalNanos);
+            parseNanosRange(Optional.ofNullable(schedulerJitter)).ifPresent(builder::schedulerJitterNanos);
+            parseNanosRange(Optional.ofNullable(schedulerDelayMicros)).ifPresent(builder::schedulerDelayNanos);
+            parseNanosRange(Optional.ofNullable(schedulerLongDelayMicros)).ifPresent(builder::schedulerLongDelayNanos);
+            parseChanceRange(Optional.ofNullable(readChance)).ifPresent(builder::readChance);
+            parseChanceRange(Optional.ofNullable(nemesisChance)).ifPresent(builder::nemesisChance);
+            parseRange(Optional.ofNullable(nodeCount)).ifPresent(builder::nodes);
+            parseRange(Optional.ofNullable(dcCount)).ifPresent(builder::dcs);
+            parseRange(Optional.ofNullable(primaryKeySeconds)).ifPresent(builder::primaryKeySeconds);
+            parseRange(Optional.ofNullable(withinKeyConcurrency)).ifPresent(builder::withinKeyConcurrency);
+            Optional.ofNullable(topologyChanges).ifPresent(topologyChanges -> {
+                builder.topologyChanges(stream(topologyChanges.split(","))
+                                        .filter(v -> !v.isEmpty())
+                                        .map(v -> TopologyChange.valueOf(v.toUpperCase()))
+                                        .toArray(TopologyChange[]::new));
+            });
+            parseNanosRange(Optional.ofNullable(topologyChangeInterval)).ifPresent(builder::topologyChangeIntervalNanos);
+            Optional.ofNullable(priority).ifPresent(kinds -> {
+                builder.scheduler(stream(kinds.split(","))
+                                  .filter(v -> !v.isEmpty())
+                                  .map(v -> RunnableActionScheduler.Kind.valueOf(v.toUpperCase()))
+                                  .toArray(RunnableActionScheduler.Kind[]::new));
+            });
+
+            Optional.ofNullable(this.capture)
+                    .map(s -> s.split(","))
+                    .map(s -> new Capture(
+                        stream(s).anyMatch(s2 -> s2.equalsIgnoreCase("wait")),
+                        stream(s).anyMatch(s2 -> s2.equalsIgnoreCase("wake")),
+                        stream(s).anyMatch(s2 -> s2.equalsIgnoreCase("now"))
+                    ))
+                    .ifPresent(builder::capture);
+
+            EnumMap<Info, Levels> debugLevels = new EnumMap<>(Info.class);
+            Optional.ofNullable(log).ifPresent(list -> debugLevels.put(Info.LOG, new Levels(list.get(0), list.get(1))));
+            Optional.ofNullable(debugRf).ifPresent(list -> debugLevels.put(Info.RF, new Levels(list.get(0), list.get(1))));
+            Optional.ofNullable(debugOwnership).ifPresent(list -> debugLevels.put(Info.OWNERSHIP, new Levels(list.get(0), list.get(1))));
+            Optional.ofNullable(debugRing).ifPresent(list -> debugLevels.put(Info.RING, new Levels(list.get(0), list.get(1))));
+            Optional.ofNullable(debugGossip).ifPresent(list -> debugLevels.put(Info.GOSSIP, new Levels(list.get(0), list.get(1))));
+            Optional.ofNullable(debugPaxos).ifPresent(list -> debugLevels.put(Info.PAXOS, new Levels(list.get(0), list.get(1))));
+            if (!debugLevels.isEmpty())
+            {
+                int[] debugPrimaryKeys = Optional.ofNullable(debugKeys)
+                                                 .map(pks -> stream(pks.split(",")).mapToInt(Integer::parseInt).sorted().toArray())
+                                                 .orElse(new int[0]);
+                builder.debug(debugLevels, debugPrimaryKeys);
+            }
+        }
+
+        public void run(B builder) throws IOException
+        {
+            beforeAll();
+            Thread.setDefaultUncaughtExceptionHandler((th, e) -> {
+                boolean isInterrupt = false;
+                Throwable t = e;
+                while (!isInterrupt && t != null)
+                {
+                    isInterrupt = t instanceof InterruptedException || t instanceof UncheckedInterruptedException;
+                    t = t.getCause();
+                }
+                if (!isInterrupt)
+                    logger.error("Uncaught exception on {}", th, e);
+                if (e instanceof Error)
+                    throw (Error) e;
+            });
+
+            propagate(builder);
+
+            long seed = parseLong(Optional.ofNullable(this.seed)).orElse(new Random(System.nanoTime()).nextLong());
+            for (int i = 0 ; i < simulationCount ; ++i)
+            {
+                cleanup();
+                run(seed, builder);
+                ++seed;
+            }
+        }
+
+        protected abstract void run(long seed, B builder) throws IOException;
+    }
+
+    @Command(name = "run")
+    protected static class Run<B extends ClusterSimulation.Builder<?>> extends BasicCommand<B>
+    {
+        protected void run(long seed, B builder) throws IOException
+        {
+            logger.error("Seed 0x{}", Long.toHexString(seed));
+
+            try (ClusterSimulation<?> cluster = builder.create(seed))
+            {
+                try
+                {
+                    cluster.simulation.run();
+                }
+                catch (Throwable t)
+                {
+                    logger.error("Failed on seed {}", Long.toHexString(seed), t);
+                }
+            }
+        }
+    }
+
+    @Command(name = "record")
+    protected static class Record<B extends ClusterSimulation.Builder<?>> extends BasicCommand<B>
+    {
+        @Option(name = {"--to"}, description = "Directory of recordings to reconcile with for the seed", required = true)
+        private String dir;
+
+        @Option(name = {"--with-rng"}, description = "Record RNG values", arity = 0)
+        private boolean rng;
+
+        @Option(name = {"--with-rng-callsites"}, description = "Record RNG call sites", arity = 0)
+        private boolean rngCallSites;
+
+        @Override
+        protected void run(long seed, B builder) throws IOException
+        {
+            record(dir, seed, rng, rngCallSites, builder);
+        }
+    }
+
+    @Command(name = "reconcile")
+    protected static class Reconcile<B extends ClusterSimulation.Builder<?>> extends BasicCommand<B>
+    {
+        @Option(name = {"--with"}, description = "Directory of recordings to reconcile with for the seed")
+        private String dir;
+
+        @Option(name = {"--with-rng"}, description = "Reconcile RNG values (if present in source)", arity = 0)
+        private boolean rng;
+
+        @Option(name = {"--with-rng-callsites"}, description = "Reconcile RNG call sites (if present in source)", arity = 0)
+        private boolean rngCallSites;
+
+        @Option(name = {"--with-allocations"}, description = "Reconcile memtable allocations (only with --with-self)", arity = 0)
+        private boolean allocations;
+
+        @Option(name = {"--with-self"}, description = "Reconcile with self", arity = 0)
+        private boolean withSelf;
+
+        @Override
+        protected void run(long seed, B builder) throws IOException
+        {
+            if (withSelf) reconcileWithSelf(seed, rng, rngCallSites, allocations, builder);
+            else if (allocations) throw new IllegalArgumentException("--with-allocations is only compatible with --with-self");
+            else reconcileWith(dir, seed, rng, rngCallSites, builder);
+        }
+    }
+
+    protected static class HelpCommand<B extends ClusterSimulation.Builder<?>> extends Help implements ICommand<B>
+    {
+        @Override
+        public void run(B builder) throws IOException
+        {
+            super.run();
+        }
+    }
+
+
+    private static Optional<Long> parseLong(Optional<String> value)
+    {
+        return value.map(s -> s.startsWith("0x")
+                              ? Hex.parseLong(s, 2, s.length())
+                              : Long.parseLong(s));
+    }
+
+    private static final Pattern CHANCE_PATTERN = Pattern.compile("(uniform|(?<qlog>qlog(\\((?<quantizations>[0-9]+)\\))?):)?(?<min>0(\\.[0-9]+)?)(..(?<max>0\\.[0-9]+))?", Pattern.CASE_INSENSITIVE);
+    private static Optional<ChanceRange> parseChanceRange(Optional<String> chance)
+    {
+        return chance.map(s -> {
+            ToDoubleFunction<RandomSource> chanceSelector = RandomSource::uniformFloat;
+            Matcher m = CHANCE_PATTERN.matcher(s);
+            if (!m.matches()) throw new IllegalArgumentException("Invalid chance specification: " + s);
+            if (m.group("qlog") != null)
+            {
+                int quantizations = m.group("quantizations") == null ? 4 : Integer.parseInt(m.group("quantizations"));
+                chanceSelector = randomSource -> randomSource.qlog2uniformFloat(quantizations);
+            }
+            float min = Float.parseFloat(m.group("min"));
+            float max = m.group("max") == null ? min : Float.parseFloat(m.group("max"));
+            return new ChanceRange(chanceSelector, min, max);
+        });
+    }
+
+    private static void cleanup()
+    {
+        FastThreadLocal.destroy();
+        for (int i = 0 ; i < 10 ; ++i)
+            System.gc();
+    }
+
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/SimulatorUtils.java b/test/simulator/main/org/apache/cassandra/simulator/SimulatorUtils.java
new file mode 100644
index 0000000..5be3384
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/SimulatorUtils.java
@@ -0,0 +1,48 @@
+/*
+ * 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.simulator;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.slf4j.Logger;
+
+import io.netty.util.concurrent.FastThreadLocal;
+import org.apache.cassandra.utils.concurrent.Threads;
+
+public class SimulatorUtils
+{
+    public static RuntimeException failWithOOM()
+    {
+        List<long[]> oom = new ArrayList<>();
+        for (int i = 0 ; i < 1024 ; ++i)
+            oom.add(new long[0x7fffffff]);
+        throw new AssertionError();
+    }
+
+    public static void dumpStackTraces(Logger logger)
+    {
+        Map<Thread, StackTraceElement[]> threadMap = Thread.getAllStackTraces();
+        threadMap.forEach((thread, ste) -> {
+            logger.error("{}:\n   {}", thread, Threads.prettyPrint(ste, false, "   ", "\n", ""));
+        });
+        FastThreadLocal.destroy();
+    }
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/asm/NemesisFieldSelectors.java b/test/simulator/main/org/apache/cassandra/simulator/asm/NemesisFieldSelectors.java
new file mode 100644
index 0000000..e8c420d
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/asm/NemesisFieldSelectors.java
@@ -0,0 +1,67 @@
+/*
+ * 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.simulator.asm;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+import java.util.concurrent.atomic.AtomicLongFieldUpdater;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.stream.Stream;
+
+import org.apache.cassandra.utils.Nemesis;
+import org.reflections.Reflections;
+import org.reflections.scanners.FieldAnnotationsScanner;
+import org.reflections.util.ConfigurationBuilder;
+
+import static java.util.Collections.emptyMap;
+import static org.apache.cassandra.simulator.asm.InterceptClasses.dotsToSlashes;
+import static org.apache.cassandra.simulator.asm.NemesisFieldKind.SIMPLE;
+
+/**
+ * Define classes that receive special handling.
+ * At present all instance methods invoked on such classes have nemesis points inserted either side of them.
+ */
+public class NemesisFieldSelectors
+{
+    public static final Map<String, Map<String, NemesisFieldKind>> classToFieldToNemesis;
+
+    static
+    {
+        Map<Class<?>, NemesisFieldKind> byClass = new HashMap<>();
+        for (NemesisFieldKind type : NemesisFieldKind.values())
+            type.classes.forEach(c -> byClass.put(c, type));
+
+        Stream.of(AtomicIntegerFieldUpdater.class, AtomicLongFieldUpdater.class, AtomicReferenceFieldUpdater.class)
+              .forEach(c -> byClass.put(c, NemesisFieldKind.ATOMICUPDATERX));
+
+        Map<String, Map<String, NemesisFieldKind>> byField = new HashMap<>();
+        new Reflections(ConfigurationBuilder.build("org.apache.cassandra").addScanners(new FieldAnnotationsScanner()))
+        .getFieldsAnnotatedWith(Nemesis.class)
+        .forEach(field -> byField.computeIfAbsent(dotsToSlashes(field.getDeclaringClass()), ignore -> new HashMap<>())
+                                 .put(field.getName(), byClass.getOrDefault(field.getType(), SIMPLE)));
+        classToFieldToNemesis = Collections.unmodifiableMap(byField);
+    }
+
+    public static NemesisFieldKind.Selector get()
+    {
+        return (name, field) -> classToFieldToNemesis.getOrDefault(name, emptyMap()).get(field);
+    }
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterAction.java b/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterAction.java
new file mode 100644
index 0000000..90289de
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterAction.java
@@ -0,0 +1,29 @@
+/*
+ * 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.simulator.cluster;
+
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableRunnable;
+import org.apache.cassandra.simulator.systems.SimulatedActionTask;
+
+class ClusterAction extends SimulatedActionTask
+{
+    ClusterAction(String description, Modifiers self, Modifiers children, ClusterActions actions, int on, SerializableRunnable invoke)
+    {
+        super(description, self.with(Modifier.DISPLAY_ORIGIN), children, actions, actions.cluster.get(on), invoke);
+    }
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterActionListener.java b/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterActionListener.java
new file mode 100644
index 0000000..d00f38c
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterActionListener.java
@@ -0,0 +1,37 @@
+/*
+ * 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.simulator.cluster;
+
+public interface ClusterActionListener
+{
+    interface TopologyChangeValidator
+    {
+        public void before(Topology before, int[] participatingKeys);
+        public void after(Topology after);
+    }
+
+    interface RepairValidator
+    {
+        public void before(Topology topology, boolean repairPaxos, boolean repairOnlyPaxos);
+        public void after();
+    }
+
+    TopologyChangeValidator newTopologyChangeValidator(Object id);
+    RepairValidator newRepairValidator(Object id);
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterActions.java b/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterActions.java
new file mode 100644
index 0000000..35fe882
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterActions.java
@@ -0,0 +1,267 @@
+/*
+ * 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.simulator.cluster;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config.PaxosVariant;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInstance;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.simulator.Action;
+import org.apache.cassandra.simulator.ActionList;
+import org.apache.cassandra.simulator.Actions;
+import org.apache.cassandra.simulator.Actions.ReliableAction;
+import org.apache.cassandra.simulator.Actions.StrictAction;
+import org.apache.cassandra.simulator.Debug;
+import org.apache.cassandra.simulator.RandomSource.Choices;
+import org.apache.cassandra.simulator.systems.InterceptedExecution;
+import org.apache.cassandra.simulator.systems.InterceptingExecutor;
+import org.apache.cassandra.simulator.systems.NonInterceptible;
+import org.apache.cassandra.simulator.systems.SimulatedSystems;
+import org.apache.cassandra.simulator.utils.KindOfSequence;
+
+import static org.apache.cassandra.distributed.impl.UnsafeGossipHelper.addToRingNormalRunner;
+import static org.apache.cassandra.simulator.Action.Modifiers.NO_TIMEOUTS;
+import static org.apache.cassandra.simulator.cluster.ClusterActions.TopologyChange.JOIN;
+import static org.apache.cassandra.simulator.cluster.ClusterActions.TopologyChange.LEAVE;
+import static org.apache.cassandra.simulator.cluster.ClusterActions.TopologyChange.REPLACE;
+
+
+// TODO (feature): add Gossip failures (up to some acceptable number)
+// TODO (feature): add node down/up (need to coordinate bootstrap/repair execution around this)
+// TODO (feature): add node stop/start (need to coordinate normal operation execution around this)
+// TODO (feature): permit multiple topology actions in parallel, e.g. REPLACE and CHANGE_RF
+// TODO (feature): support nodes rejoining cluster so we can leave running indefinitely
+@SuppressWarnings("unused")
+public class ClusterActions extends SimulatedSystems
+{
+    private static final Logger logger = LoggerFactory.getLogger(ClusterActions.class);
+
+    public enum TopologyChange
+    {
+        JOIN, LEAVE, REPLACE, CHANGE_RF
+    }
+
+    public static class Options
+    {
+        public final KindOfSequence.Period topologyChangeInterval;
+        public final Choices<TopologyChange> allChoices;
+        public final Choices<TopologyChange> choicesNoLeave;
+        public final Choices<TopologyChange> choicesNoJoin;
+
+        public final int[] minRf, initialRf, maxRf;
+        public final PaxosVariant changePaxosVariantTo;
+
+        public Options(Options copy)
+        {
+            this(copy, copy.changePaxosVariantTo);
+        }
+
+        public Options(Options copy, PaxosVariant changePaxosVariantTo)
+        {
+            this.topologyChangeInterval = copy.topologyChangeInterval;
+            this.allChoices = copy.allChoices;
+            this.choicesNoLeave = copy.choicesNoLeave;
+            this.choicesNoJoin = copy.choicesNoJoin;
+            this.minRf = copy.minRf;
+            this.initialRf = copy.initialRf;
+            this.maxRf = copy.maxRf;
+            this.changePaxosVariantTo = changePaxosVariantTo;
+        }
+
+        public Options(KindOfSequence.Period topologyChangeInterval, Choices<TopologyChange> choices, int[] minRf, int[] initialRf, int[] maxRf, PaxosVariant changePaxosVariantTo)
+        {
+            this.topologyChangeInterval = topologyChangeInterval;
+            this.minRf = minRf;
+            this.initialRf = initialRf;
+            this.maxRf = maxRf;
+            this.allChoices = choices;
+            this.choicesNoJoin = allChoices.without(JOIN).without(REPLACE);
+            this.choicesNoLeave = allChoices.without(LEAVE);
+            this.changePaxosVariantTo = changePaxosVariantTo;
+        }
+
+        public Options changePaxosVariantTo(PaxosVariant newVariant)
+        {
+            return new Options(this, newVariant);
+        }
+    }
+
+    final Cluster cluster;
+    final Options options;
+    final ClusterActionListener listener;
+    final Debug debug;
+
+    public ClusterActions(SimulatedSystems simulated,
+                          Cluster cluster,
+                          Options options,
+                          ClusterActionListener listener,
+                          Debug debug)
+    {
+        super(simulated);
+        this.cluster = cluster;
+        this.options = options;
+        this.listener = listener;
+        this.debug = debug;
+    }
+
+    public static class InitialConfiguration
+    {
+        public static final int[] EMPTY = {};
+        private final int[] joined;
+        private final int[] prejoin;
+
+        public InitialConfiguration(int[] joined, int[] prejoin)
+        {
+            this.joined = joined;
+            this.prejoin = prejoin;
+        }
+
+        public static InitialConfiguration initializeAll(int nodes)
+        {
+            int[] joined = new int[nodes];
+            for (int i = 0; i < nodes; i++)
+                joined[i] = i + 1;
+            return new InitialConfiguration(joined, EMPTY);
+        }
+    }
+
+    public Action initializeCluster(InitialConfiguration config)
+    {
+        return this.initializeCluster(config.joined, config.prejoin);
+    }
+
+    public Action initializeCluster(int[] joined, int[] prejoin)
+    {
+        return StrictAction.of("Initialise Cluster", () -> {
+            List<Action> actions = new ArrayList<>();
+
+            cluster.stream().forEach(i -> actions.add(invoke("Startup " + i.broadcastAddress(), NO_TIMEOUTS, NO_TIMEOUTS,
+                                                             new InterceptedExecution.InterceptedRunnableExecution((InterceptingExecutor) i.executor(), i::startup))));
+
+            List<InetSocketAddress> endpoints = cluster.stream().map(IInstance::broadcastAddress).collect(Collectors.toList());
+            cluster.forEach(i -> actions.add(resetGossipState(i, endpoints)));
+
+            for (int add : joined)
+            {
+                actions.add(transitivelyReliable("Add " + add + " to ring", cluster.get(add), addToRingNormalRunner(cluster.get(add))));
+                actions.addAll(sendLocalGossipStateToAll(add));
+            }
+
+            actions.add(ReliableAction.transitively("Sync Pending Ranges Executor", ClusterActions.this::syncPendingRanges));
+            return ActionList.of(actions);
+        });
+    }
+
+    Action resetGossipState(IInvokableInstance i, List<InetSocketAddress> endpoints)
+    {
+        return transitivelyReliable("Reset Gossip", i, () -> Gossiper.runInGossipStageBlocking(Gossiper.instance::unsafeSetEnabled));
+    }
+
+    @SuppressWarnings("unchecked")
+    void validateReplicasForKeys(IInvokableInstance on, String keyspace, String table, Topology topology)
+    {
+        int[] primaryKeys = topology.primaryKeys;
+        int[][] validate = NonInterceptible.apply(() -> {
+            Map<InetSocketAddress, Integer> lookup = Cluster.getUniqueAddressLookup(cluster, i -> i.config().num());
+            int[][] result = new int[primaryKeys.length][];
+            for (int i = 0 ; i < primaryKeys.length ; ++i)
+            {
+                int primaryKey = primaryKeys[i];
+                result[i] = on.unsafeApplyOnThisThread(ClusterActions::replicasForPrimaryKey, keyspace, table, primaryKey)
+                              .stream()
+                              .mapToInt(lookup::get)
+                              .filter(r -> Arrays.binarySearch(topology.membersOfQuorum, r) >= 0)
+                              .toArray();
+            }
+            return result;
+        });
+        for (int i = 0 ; i < primaryKeys.length ; ++i)
+        {
+            int[] vs1 = validate[i];
+            int[] vs2 = topology.replicasForKeys[i].clone();
+            Arrays.sort(vs1);
+            Arrays.sort(vs2);
+            if (!Arrays.equals(vs1, vs2))
+                throw new AssertionError();
+        }
+    }
+
+    // assumes every node knows the correct topology
+    static List<InetSocketAddress> replicasForPrimaryKey(String keyspaceName, String table, int primaryKey)
+    {
+        Keyspace keyspace = Keyspace.open(keyspaceName);
+        TableMetadata metadata = keyspace.getColumnFamilyStore(table).metadata.get();
+        DecoratedKey key = metadata.partitioner.decorateKey(Int32Type.instance.decompose(primaryKey));
+        // we return a Set because simulator can easily encounter point where nodes are both natural and pending
+        return ReplicaLayout.forTokenWriteLiveAndDown(keyspace, key.getToken()).all().asList(Replica::endpoint);
+    }
+
+    private ActionList to(BiFunction<Integer, Integer, Action> action, int from, IntStream to)
+    {
+        return ActionList.of(to.filter(i -> i != from)
+                .mapToObj(i -> action.apply(from, i)));
+    }
+    private ActionList toAll(BiFunction<Integer, Integer, Action> action, int from)
+    {
+        return to(action, from, IntStream.rangeClosed(1, cluster.size()));
+    }
+    private ActionList to(BiFunction<Integer, Integer, Action> action, int from, int[] to)
+    {
+        return to(action, from, IntStream.of(to));
+    }
+
+    ActionList on(Function<Integer, Action> action, IntStream on)
+    {
+        return ActionList.of(on.mapToObj(action::apply));
+    }
+    ActionList onAll(Function<Integer, Action> action)
+    {
+        return on(action, IntStream.rangeClosed(1, cluster.size()));
+    }
+    ActionList on(Function<Integer, Action> action, int[] on)
+    {
+        return on(action, IntStream.of(on));
+    }
+
+    ActionList syncPendingRanges() { return onAll(OnInstanceSyncPendingRanges.factory(this)); }
+    ActionList gossipWithAll(int from) { return toAll(OnInstanceGossipWith.factory(this), from); }
+    ActionList sendShutdownToAll(int from) { return toAll(OnInstanceSendShutdown.factory(this), from); }
+    ActionList sendLocalGossipStateToAll(int from) { return toAll(OnInstanceSendLocalGossipState.factory(this), from); }
+    ActionList flushAndCleanup(int[] on) { return on(OnInstanceFlushAndCleanup.factory(this), on); }
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterReliableAction.java b/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterReliableAction.java
new file mode 100644
index 0000000..591f2b9
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterReliableAction.java
@@ -0,0 +1,30 @@
+/*
+ * 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.simulator.cluster;
+
+import static org.apache.cassandra.distributed.api.IIsolatedExecutor.*;
+import static org.apache.cassandra.simulator.Action.Modifiers.RELIABLE_NO_TIMEOUTS;
+
+public class ClusterReliableAction extends ClusterAction
+{
+    public ClusterReliableAction(String description, ClusterActions actions, int on, SerializableRunnable runnable)
+    {
+        super(description, RELIABLE_NO_TIMEOUTS, RELIABLE_NO_TIMEOUTS, actions, on, runnable);
+    }
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterReliableQueryAction.java b/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterReliableQueryAction.java
new file mode 100644
index 0000000..ea22081
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterReliableQueryAction.java
@@ -0,0 +1,38 @@
+/*
+ * 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.simulator.cluster;
+
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.simulator.systems.SimulatedQuery;
+
+import static org.apache.cassandra.simulator.Action.Modifiers.RELIABLE_NO_TIMEOUTS;
+
+class ClusterReliableQueryAction extends SimulatedQuery
+{
+    ClusterReliableQueryAction(String id, ClusterActions actions, int on, String query, long timestamp, ConsistencyLevel consistencyLevel, Object... params)
+    {
+        super(id, RELIABLE_NO_TIMEOUTS, RELIABLE_NO_TIMEOUTS, actions, actions.cluster.get(on), query, timestamp, consistencyLevel, params);
+    }
+
+    public static ClusterReliableQueryAction schemaChange(String id, ClusterActions actions, int on, String query)
+    {
+        // this isn't used on 4.0+ nodes, but no harm in supplying it anyway
+        return new ClusterReliableQueryAction(id, actions, on, query, actions.time.nextGlobalMonotonicMicros(), ConsistencyLevel.ALL);
+    }
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterUnsafeAction.java b/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterUnsafeAction.java
new file mode 100644
index 0000000..3b61e92
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/cluster/ClusterUnsafeAction.java
@@ -0,0 +1,34 @@
+/*
+ * 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.simulator.cluster;
+
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.simulator.systems.SimulatedActionTask;
+
+class ClusterUnsafeAction extends SimulatedActionTask
+{
+    ClusterUnsafeAction(String id, Modifiers self, Modifiers children, ClusterActions actions, int on, Runnable run)
+    {
+        this(id, self, children, actions, actions.cluster.get(on), run);
+    }
+
+    ClusterUnsafeAction(String id, Modifiers self, Modifiers children, ClusterActions actions, IInvokableInstance on, Runnable run)
+    {
+        super(id, self, children, null, actions, unsafeAsTask(on, run, actions.failures));
+    }
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/cluster/KeyspaceActions.java b/test/simulator/main/org/apache/cassandra/simulator/cluster/KeyspaceActions.java
new file mode 100644
index 0000000..fa3e939
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/cluster/KeyspaceActions.java
@@ -0,0 +1,391 @@
+/*
+ * 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.simulator.cluster;
+
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.IntStream;
+
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner.LongToken;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.locator.EndpointsForToken;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.NetworkTopologyStrategy;
+import org.apache.cassandra.locator.PendingRangeMaps;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.simulator.Action;
+import org.apache.cassandra.simulator.ActionList;
+import org.apache.cassandra.simulator.ActionListener;
+import org.apache.cassandra.simulator.ActionPlan;
+import org.apache.cassandra.simulator.Actions;
+import org.apache.cassandra.simulator.Debug;
+import org.apache.cassandra.simulator.OrderOn.StrictSequential;
+import org.apache.cassandra.simulator.systems.SimulatedSystems;
+
+import static java.util.Collections.singleton;
+import static java.util.Collections.singletonList;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.LOCAL_SERIAL;
+import static org.apache.cassandra.simulator.Debug.EventType.CLUSTER;
+import static org.apache.cassandra.simulator.cluster.ClusterActions.TopologyChange.CHANGE_RF;
+import static org.apache.cassandra.simulator.cluster.ClusterActions.TopologyChange.JOIN;
+import static org.apache.cassandra.simulator.cluster.ClusterActions.TopologyChange.LEAVE;
+import static org.apache.cassandra.simulator.cluster.ClusterReliableQueryAction.schemaChange;
+
+public class KeyspaceActions extends ClusterActions
+{
+    final String keyspace;
+    final String table;
+    final String createTableCql;
+    final ConsistencyLevel serialConsistency;
+    final int[] primaryKeys;
+
+    final EnumSet<TopologyChange> ops = EnumSet.noneOf(TopologyChange.class);
+    final NodeLookup nodeLookup;
+    final int[] minRf, initialRf, maxRf;
+    final int[] membersOfQuorumDcs;
+
+    // working state
+    final NodesByDc all;
+    final NodesByDc prejoin;
+    final NodesByDc joined;
+    final NodesByDc left;
+
+    final int[] currentRf;
+    final TokenMetadata tokenMetadata = new TokenMetadata(snitch.get());
+    Topology topology;
+    boolean haveChangedVariant;
+
+    public KeyspaceActions(SimulatedSystems simulated,
+                           String keyspace, String table, String createTableCql,
+                           Cluster cluster,
+                           Options options,
+                           ConsistencyLevel serialConsistency,
+                           ClusterActionListener listener,
+                           int[] primaryKeys,
+                           Debug debug)
+    {
+        super(simulated, cluster, options, listener, debug);
+        this.keyspace = keyspace;
+        this.table = table;
+        this.createTableCql = createTableCql;
+        this.primaryKeys = primaryKeys;
+        this.serialConsistency = serialConsistency;
+
+        this.nodeLookup = simulated.snitch;
+
+        int[] dcSizes = new int[options.initialRf.length];
+        for (int dc : nodeLookup.nodeToDc)
+            ++dcSizes[dc];
+
+        this.all = new NodesByDc(nodeLookup, dcSizes);
+        this.prejoin = new NodesByDc(nodeLookup, dcSizes);
+        this.joined = new NodesByDc(nodeLookup, dcSizes);
+        this.left = new NodesByDc(nodeLookup, dcSizes);
+
+        for (int i = 1 ; i <= nodeLookup.nodeToDc.length ; ++i)
+        {
+            this.prejoin.add(i);
+            this.all.add(i);
+        }
+
+        minRf = options.minRf;
+        initialRf = options.initialRf;
+        maxRf = options.maxRf;
+        currentRf = initialRf.clone();
+        membersOfQuorumDcs = serialConsistency == LOCAL_SERIAL ? all.dcs[0] : all.toArray();
+        ops.addAll(Arrays.asList(options.allChoices.options));
+
+    }
+
+    public ActionPlan plan()
+    {
+        ActionList pre = ActionList.of(pre(createKeyspaceCql(keyspace), createTableCql));
+        ActionList interleave = stream();
+        ActionList post = ActionList.empty();
+        return new ActionPlan(pre, singletonList(interleave), post);
+    }
+
+    @SuppressWarnings("StringConcatenationInLoop")
+    private String createKeyspaceCql(String keyspace)
+    {
+        String createKeyspaceCql = "CREATE KEYSPACE " + keyspace  + " WITH replication = {'class': 'NetworkTopologyStrategy'";
+        for (int i = 0 ; i < options.initialRf.length ; ++i)
+            createKeyspaceCql += ", '" + snitch.nameOfDc(i) + "': " + options.initialRf[i];
+        createKeyspaceCql += "};";
+        return createKeyspaceCql;
+    }
+
+    private Action pre(String createKeyspaceCql, String createTableCql)
+    {
+        // randomise initial cluster, and return action to initialise it
+        for (int dc = 0 ; dc < options.initialRf.length ; ++dc)
+        {
+            for (int i = 0 ; i < options.initialRf[dc] ; ++i)
+            {
+                int join = prejoin.removeRandom(random, dc);
+                joined.add(join);
+                tokenMetadata.updateNormalToken(tokenOf(join), inet(join));
+            }
+        }
+
+        updateTopology(recomputeTopology());
+        int[] joined = this.joined.toArray();
+        int[] prejoin = this.prejoin.toArray();
+        return Actions.StrictAction.of("Initialize", () -> {
+            return ActionList.of(initializeCluster(joined, prejoin),
+                                 schemaChange("Create Keyspace", KeyspaceActions.this, 1, createKeyspaceCql),
+                                 schemaChange("Create Table", KeyspaceActions.this, 1, createTableCql));
+        });
+    }
+
+    @SuppressWarnings("StatementWithEmptyBody")
+    private ActionList stream()
+    {
+        ActionListener listener = debug.debug(CLUSTER, time, cluster, keyspace, null);
+        if (listener == null)
+            return ActionList.of(Actions.stream(new StrictSequential("Cluster Actions"), this::next));
+
+        return ActionList.of(Actions.stream(new StrictSequential("Cluster Actions"), () -> {
+            Action action = next();
+            if (action != null)
+                action.register(listener);
+            return action;
+        }));
+    }
+
+    private Action next()
+    {
+        while (!ops.isEmpty() && !prejoin.isEmpty() || (ops.contains(LEAVE) && joined.size() > sum(minRf)))
+        {
+            if (options.changePaxosVariantTo != null && !haveChangedVariant && random.decide(1f / (1 + prejoin.size())))
+            {
+                haveChangedVariant = true;
+                return schedule(new OnClusterSetPaxosVariant(KeyspaceActions.this, options.changePaxosVariantTo));
+            }
+
+            // pick a dc
+            int dc = random.uniform(0, currentRf.length);
+
+            // try to pick an action (and simply loop again if we cannot for this dc)
+            TopologyChange next;
+            if (prejoin.size(dc) > 0 && joined.size(dc) > currentRf[dc]) next = options.allChoices.choose(random);
+            else if (prejoin.size(dc) > 0 && ops.contains(JOIN)) next = options.choicesNoLeave.choose(random);
+            else if (joined.size(dc) > currentRf[dc] && ops.contains(LEAVE)) next = options.choicesNoJoin.choose(random);
+            else if (joined.size(dc) > minRf[dc] && ops.contains(LEAVE)) next = CHANGE_RF;
+            else continue;
+
+            // TODO (feature): introduce some time period between cluster actions
+            switch (next)
+            {
+                case REPLACE:
+                {
+                    Topology before = topology;
+                    int join = prejoin.removeRandom(random, dc);
+                    int leave = joined.selectRandom(random, dc);
+                    joined.add(join);
+                    joined.remove(leave);
+                    left.add(leave);
+                    nodeLookup.setTokenOf(join, nodeLookup.tokenOf(leave));
+                    Collection<Token> token = singleton(tokenOf(leave));
+                    tokenMetadata.addReplaceTokens(token, inet(join), inet(leave));
+                    tokenMetadata.unsafeCalculatePendingRanges(strategy(), keyspace);
+                    Topology during = recomputeTopology();
+                    updateTopology(during);
+                    tokenMetadata.updateNormalTokens(token, inet(join));
+                    tokenMetadata.unsafeCalculatePendingRanges(strategy(), keyspace);
+                    Topology after = recomputeTopology();
+                    Action action = new OnClusterReplace(KeyspaceActions.this, before, during, after, leave, join);
+                    return scheduleAndUpdateTopologyOnCompletion(action, after);
+                    // if replication factor is 2, cannot perform safe replacements
+                    // however can have operations that began earlier during RF=2
+                    // so need to introduce some concept of barriers/ordering/sync points
+                }
+                case JOIN:
+                {
+                    Topology before = topology;
+                    int join = prejoin.removeRandom(random, dc);
+                    joined.add(join);
+                    Collection<Token> token = singleton(tokenOf(join));
+                    tokenMetadata.addBootstrapTokens(token, inet(join));
+                    tokenMetadata.unsafeCalculatePendingRanges(strategy(), keyspace);
+                    Topology during = recomputeTopology();
+                    updateTopology(during);
+                    tokenMetadata.updateNormalTokens(token, inet(join));
+                    tokenMetadata.unsafeCalculatePendingRanges(strategy(), keyspace);
+                    Topology after = recomputeTopology();
+                    Action action = new OnClusterJoin(KeyspaceActions.this, before, during, after, join);
+                    return scheduleAndUpdateTopologyOnCompletion(action, after);
+                }
+                case LEAVE:
+                {
+                    Topology before = topology;
+                    int leave = joined.removeRandom(random, dc);
+                    left.add(leave);
+                    tokenMetadata.addLeavingEndpoint(inet(leave));
+                    tokenMetadata.unsafeCalculatePendingRanges(strategy(), keyspace);
+                    Topology during = recomputeTopology();
+                    updateTopology(during);
+                    tokenMetadata.removeEndpoint(inet(leave));
+                    tokenMetadata.unsafeCalculatePendingRanges(strategy(), keyspace);
+                    Topology after = recomputeTopology();
+                    Action action = new OnClusterLeave(KeyspaceActions.this, before, during, after, leave);
+                    return scheduleAndUpdateTopologyOnCompletion(action, after);
+                }
+                case CHANGE_RF:
+                    if (maxRf[dc] == minRf[dc]) {} // cannot perform RF changes at all
+                    if (currentRf[dc] == minRf[dc] && joined.size(dc) == currentRf[dc]) {} // can do nothing until joined grows
+                    else
+                    {
+                        boolean increase;
+                        if (currentRf[dc] == minRf[dc]) // can only grow
+                        { ++currentRf[dc]; increase = true;}
+                        else if (currentRf[dc] == joined.size(dc) || currentRf[dc] == maxRf[dc]) // can only shrink, and we know currentRf > minRf
+                        { --currentRf[dc]; increase = false; }
+                        else if (random.decide(0.5f)) // can do either
+                        { --currentRf[dc]; increase = false; }
+                        else
+                        { ++currentRf[dc]; increase = true; }
+
+                        // this isn't used on 4.0+ nodes, but no harm in supplying it anyway
+                        long timestamp = time.nextGlobalMonotonicMicros();
+                        int coordinator = joined.selectRandom(random, dc);
+                        Topology before = topology;
+                        Topology after = recomputeTopology();
+                        return scheduleAndUpdateTopologyOnCompletion(new OnClusterChangeRf(KeyspaceActions.this, timestamp, coordinator, before, after, increase), after);
+                    }
+            }
+        }
+
+        if (options.changePaxosVariantTo != null && !haveChangedVariant)
+        {
+            haveChangedVariant = true;
+            return schedule(new OnClusterSetPaxosVariant(KeyspaceActions.this, options.changePaxosVariantTo));
+        }
+
+        return null;
+    }
+
+    private Action schedule(Action action)
+    {
+        action.setDeadline(time.nanoTime() + options.topologyChangeInterval.get(random));
+        return action;
+    }
+
+    private Action scheduleAndUpdateTopologyOnCompletion(Action action, Topology newTopology)
+    {
+        action.register(new ActionListener()
+        {
+            @Override
+            public void before(Action action, Before before)
+            {
+                if (before == Before.EXECUTE)
+                    time.forbidDiscontinuities();
+            }
+
+            @Override
+            public void transitivelyAfter(Action finished)
+            {
+                updateTopology(newTopology);
+                time.permitDiscontinuities();
+            }
+        });
+        return schedule(action);
+    }
+
+    void updateTopology(Topology newTopology)
+    {
+        topology = newTopology;
+        announce(topology);
+    }
+
+    private Topology recomputeTopology()
+    {
+        AbstractReplicationStrategy strategy = strategy();
+        Map<InetSocketAddress, Integer> lookup = Cluster.getUniqueAddressLookup(cluster, i -> i.config().num());
+        int[][] replicasForKey = new int[primaryKeys.length][];
+        int[][] pendingReplicasForKey = new int[primaryKeys.length][];
+        for (int i = 0 ; i < primaryKeys.length ; ++i)
+        {
+            int primaryKey = primaryKeys[i];
+            Token token = new Murmur3Partitioner().getToken(Int32Type.instance.decompose(primaryKey));
+            replicasForKey[i] = strategy.calculateNaturalReplicas(token, tokenMetadata)
+                                        .endpointList().stream().mapToInt(lookup::get).toArray();
+            PendingRangeMaps pendingRanges = tokenMetadata.getPendingRanges(keyspace);
+            EndpointsForToken pendingEndpoints = pendingRanges == null ? null : pendingRanges.pendingEndpointsFor(token);
+            if (pendingEndpoints == null) pendingReplicasForKey[i] = new int[0];
+            else pendingReplicasForKey[i] = pendingEndpoints.endpointList().stream().mapToInt(lookup::get).toArray();
+        }
+        int[] membersOfRing = joined.toArray();
+        long[] membersOfRingTokens = IntStream.of(membersOfRing).mapToLong(nodeLookup::tokenOf).toArray();
+        return new Topology(primaryKeys, membersOfRing, membersOfRingTokens, membersOfQuorum(), currentRf.clone(),
+                            quorumRf(), replicasForKey, pendingReplicasForKey);
+    }
+
+    private int quorumRf()
+    {
+        if (serialConsistency == LOCAL_SERIAL)
+            return currentRf[0];
+
+        return sum(currentRf);
+    }
+
+    private int[] membersOfQuorum()
+    {
+        if (serialConsistency == LOCAL_SERIAL)
+            return joined.toArray(0);
+
+        return joined.toArray();
+    }
+
+    private static int sum(int[] vs)
+    {
+        int sum = 0;
+        for (int v : vs)
+            sum += v;
+        return sum;
+    }
+
+    private InetAddressAndPort inet(int node)
+    {
+        return InetAddressAndPort.getByAddress(cluster.get(node).config().broadcastAddress());
+    }
+
+    AbstractReplicationStrategy strategy()
+    {
+        Map<String, String> rf = new HashMap<>();
+        for (int i = 0 ; i < snitch.dcCount() ; ++i)
+            rf.put(snitch.nameOfDc(i), Integer.toString(currentRf[i]));
+        return new NetworkTopologyStrategy(keyspace, tokenMetadata, snitch.get(), rf);
+    }
+
+    private Token tokenOf(int node)
+    {
+        return new LongToken(Long.parseLong(cluster.get(nodeLookup.tokenOf(node)).config().getString("initial_token")));
+    }
+
+}
diff --git a/test/simulator/main/org/apache/cassandra/simulator/cluster/NodeLookup.java b/test/simulator/main/org/apache/cassandra/simulator/cluster/NodeLookup.java
new file mode 100644
index 0000000..e1e339f
--- /dev/null
+++ b/test/simulator/main/org/apache/cassandra/simulator/cluster/NodeLookup.java
@@ -0,0 +1,48 @@
+/*
+ * 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.simulator.cluster;
+
+public class NodeLookup
+{
+    protected final int[] nodeToDc;
+    protected final int[] nodeToToken;
+
+    protected NodeLookup(int[] nodeToDc)
+    {
+        this.nodeToDc = nodeToDc;
+        this.nodeToToken = new int[nodeToDc.length];
+        for (int i = 0; i < nodeToToken.length; ++i)
+            nodeToToken[i] = i + 1;
+    }
+
+    public int dcOf(int node)
+    {
+        return nodeToDc[node - 1];
+    }
+
+    public int tokenOf(int node)
... 14483 lines suppressed ...

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


[cassandra] 02/03: [CEP-10] Cluster and Code Simulations: Minor improvements

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit eae581a5f07c340594f6af47bb558693ef363611
Author: Benedict Elliott Smith <be...@apple.com>
AuthorDate: Wed Nov 17 14:34:23 2021 +0000

    [CEP-10] Cluster and Code Simulations: Minor improvements
    
    - Simplify Semaphore
    - Future improvements
    - ScheduledExecutorPlus improvements for simulator compatibility
    - Debug leaks in Ref or BufferPool
    - Support use of TokenMetadata without initialising Cassandra
    - Additional system properties and simulator flags
    - Permit Clock initialisation within separate ClassLoader
    - Introduce BallotGenerator
    
    patch by Benedict; reviewed by Sam Tunnicliffe for CASSANDRA-17008
---
 .../cassandra/auth/CassandraRoleManager.java       |   2 +-
 .../concurrent/ScheduledExecutorPlus.java          |  24 +++
 .../ScheduledThreadPoolExecutorPlus.java           |  27 +++
 .../cassandra/concurrent/SyncFutureTask.java       |   5 +-
 .../config/CassandraRelevantProperties.java        |   1 +
 .../cassandra/config/DatabaseDescriptor.java       |   4 +
 .../cql3/statements/BatchUpdatesCollector.java     |   2 +-
 src/java/org/apache/cassandra/db/Keyspace.java     |   2 +-
 src/java/org/apache/cassandra/db/Mutation.java     |   2 +-
 src/java/org/apache/cassandra/db/ReadCommand.java  |   2 +-
 .../cassandra/db/ReadExecutionController.java      |   2 +-
 .../db/commitlog/AbstractCommitLogService.java     |   3 +-
 .../cassandra/db/monitoring/MonitorableImpl.java   |   2 +-
 .../cassandra/db/monitoring/MonitoringTask.java    |   2 +-
 .../org/apache/cassandra/gms/FailureDetector.java  |   4 +-
 .../apache/cassandra/hints/HintsDispatcher.java    |   2 +-
 .../apache/cassandra/locator/TokenMetadata.java    |  76 ++++----
 src/java/org/apache/cassandra/metrics/Sampler.java |   2 +-
 .../cassandra/net/AbstractMessageHandler.java      |   2 +-
 .../cassandra/net/InboundMessageHandler.java       |   2 +-
 .../cassandra/net/InboundMessageHandlers.java      |   2 +-
 src/java/org/apache/cassandra/net/Message.java     |   2 +-
 .../apache/cassandra/net/OutboundConnection.java   |   2 +-
 .../org/apache/cassandra/net/RequestCallbacks.java |   2 +-
 .../apache/cassandra/net/ResponseVerbHandler.java  |   2 +-
 .../apache/cassandra/schema/MigrationManager.java  |   4 +
 .../apache/cassandra/schema/SchemaKeyspace.java    |   3 +
 .../cassandra/service/ActiveRepairService.java     |   3 +
 .../org/apache/cassandra/service/StorageProxy.java |  28 ++-
 .../cassandra/service/paxos/BallotGenerator.java   |  75 ++++++++
 .../cassandra/service/paxos/ProposeCallback.java   |   3 +-
 .../cassandra/transport/CQLMessageHandler.java     |   2 +-
 src/java/org/apache/cassandra/utils/Clock.java     |  31 +++-
 .../org/apache/cassandra/utils/MonotonicClock.java |  19 +-
 .../Nemesis.java}                                  |  27 ++-
 src/java/org/apache/cassandra/utils/Simulate.java  |  56 ++++++
 .../cassandra/utils/concurrent/AbstractFuture.java |  26 ++-
 .../cassandra/utils/concurrent/AsyncFuture.java    |   2 +-
 .../cassandra/utils/concurrent/Awaitable.java      | 135 +++++++-------
 .../apache/cassandra/utils/concurrent/Future.java  |  11 ++
 .../cassandra/utils/concurrent/ListenerList.java   |  18 +-
 .../org/apache/cassandra/utils/concurrent/Ref.java |  29 ++-
 .../cassandra/utils/concurrent/Semaphore.java      | 200 ++-------------------
 .../cassandra/utils/concurrent/SyncFuture.java     |   4 +-
 .../apache/cassandra/utils/memory/BufferPool.java  |  19 +-
 .../apache/cassandra/utils/memory/HeapPool.java    |   4 +
 .../cassandra/utils/memory/LongBufferPoolTest.java |   4 +-
 .../concurrent/AbstractExecutorPlusTest.java       |   8 +-
 .../cassandra/utils/concurrent/SemaphoreTest.java  |   8 +-
 49 files changed, 515 insertions(+), 382 deletions(-)

diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
index 0e49056..b813b55 100644
--- a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
+++ b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
@@ -385,7 +385,7 @@ public class CassandraRoleManager implements IRoleManager
     protected void scheduleSetupTask(final Callable<Void> setupTask)
     {
         // The delay is to give the node a chance to see its peers before attempting the operation
-        ScheduledExecutors.optionalTasks.schedule(() -> {
+        ScheduledExecutors.optionalTasks.scheduleSelfRecurring(() -> {
             isClusterReady = true;
             try
             {
diff --git a/src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java b/src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java
index ecf073d..a2b033a 100644
--- a/src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java
+++ b/src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java
@@ -19,6 +19,8 @@
 package org.apache.cassandra.concurrent;
 
 import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.cassandra.utils.Shared;
 
@@ -27,4 +29,26 @@ import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
 @Shared(scope = SIMULATION)
 public interface ScheduledExecutorPlus extends ExecutorPlus, ScheduledExecutorService
 {
+    /**
+     * Schedule an action that is recurring but self-administered.
+     */
+    ScheduledFuture<?> scheduleSelfRecurring(Runnable run, long delay, TimeUnit units);
+
+    /**
+     * Schedule a timeout action. This method is primarily used by the Simulator to modify its
+     * scheduling behaviour with respect to this operation.
+     */
+    ScheduledFuture<?> scheduleAt(Runnable run, long deadline);
+
+    /**
+     * Schedule a timeout action. This method is primarily used by the Simulator to modify its
+     * scheduling behaviour with respect to this operation.
+     */
+    ScheduledFuture<?> scheduleTimeoutAt(Runnable run, long deadline);
+
+    /**
+     * Schedule a timeout action. This method is primarily used by the Simulator to modify its
+     * scheduling behaviour with respect to this operation.
+     */
+    ScheduledFuture<?> scheduleTimeoutWithDelay(Runnable run, long delay, TimeUnit units);
 }
diff --git a/src/java/org/apache/cassandra/concurrent/ScheduledThreadPoolExecutorPlus.java b/src/java/org/apache/cassandra/concurrent/ScheduledThreadPoolExecutorPlus.java
index efd284f..0ab09a4 100644
--- a/src/java/org/apache/cassandra/concurrent/ScheduledThreadPoolExecutorPlus.java
+++ b/src/java/org/apache/cassandra/concurrent/ScheduledThreadPoolExecutorPlus.java
@@ -28,8 +28,11 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.service.StorageService;
 
+import static com.google.common.primitives.Longs.max;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static org.apache.cassandra.concurrent.ExecutionFailure.propagating;
 import static org.apache.cassandra.concurrent.ExecutionFailure.suppressing;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
 
 /**
  * Like ExecutorPlus, ScheduledThreadPoolExecutorPlus always
@@ -97,6 +100,30 @@ public class ScheduledThreadPoolExecutorPlus extends ScheduledThreadPoolExecutor
         return super.scheduleWithFixedDelay(suppressing(task), initialDelay, delay, unit);
     }
 
+    @Override
+    public ScheduledFuture<?> scheduleSelfRecurring(Runnable run, long delay, TimeUnit units)
+    {
+        return schedule(run, delay, units);
+    }
+
+    @Override
+    public ScheduledFuture<?> scheduleAt(Runnable run, long deadline)
+    {
+        return schedule(run, max(0, deadline - nanoTime()), NANOSECONDS);
+    }
+
+    @Override
+    public ScheduledFuture<?> scheduleTimeoutAt(Runnable run, long deadline)
+    {
+        return scheduleTimeoutWithDelay(run, max(0, deadline - nanoTime()), NANOSECONDS);
+    }
+
+    @Override
+    public ScheduledFuture<?> scheduleTimeoutWithDelay(Runnable run, long delay, TimeUnit units)
+    {
+        return schedule(run, delay, units);
+    }
+
     /*======== BEGIN DIRECT COPY OF ThreadPoolExecutorPlus ===============*/
 
     private <T extends Runnable> T addTask(T task)
diff --git a/src/java/org/apache/cassandra/concurrent/SyncFutureTask.java b/src/java/org/apache/cassandra/concurrent/SyncFutureTask.java
index 4f4aa67..4885821 100644
--- a/src/java/org/apache/cassandra/concurrent/SyncFutureTask.java
+++ b/src/java/org/apache/cassandra/concurrent/SyncFutureTask.java
@@ -60,7 +60,10 @@ public class SyncFutureTask<T> extends SyncFuture<T> implements RunnableFuture<T
         try
         {
             if (!setUncancellable())
-                throw new IllegalStateException();
+            {
+                if (isCancelled()) return;
+                else throw new IllegalStateException();
+            }
 
             if (!trySuccess(call.call()))
                 throw new IllegalStateException();
diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
index 43db1c3..807516c 100644
--- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
+++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
@@ -228,6 +228,7 @@ public enum CassandraRelevantProperties
     // properties for debugging simulator ASM output
     TEST_SIMULATOR_PRINT_ASM("cassandra.test.simulator.print_asm", "none"),
     TEST_SIMULATOR_PRINT_ASM_TYPES("cassandra.test.simulator.print_asm_types", ""),
+    TEST_SIMULATOR_LIVENESS_CHECK("cassandra.test.simulator.livenesscheck", "true"),
 
     // determinism properties for testing
     DETERMINISM_SSTABLE_COMPRESSION_DEFAULT("cassandra.sstable_compression_default", "true"),
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 8bb52d4..8265fd2 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -70,6 +70,7 @@ import org.apache.cassandra.security.EncryptionContext;
 import org.apache.cassandra.security.SSLFactory;
 import org.apache.cassandra.service.CacheService.CacheType;
 import org.apache.cassandra.service.paxos.Paxos;
+import org.apache.cassandra.utils.Clock;
 import org.apache.cassandra.utils.FBUtilities;
 
 import org.apache.commons.lang3.ArrayUtils;
@@ -80,6 +81,7 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.OS_ARCH;
 import static org.apache.cassandra.config.CassandraRelevantProperties.SUN_ARCH_DATA_MODEL;
 import static org.apache.cassandra.io.util.FileUtils.ONE_GB;
 import static org.apache.cassandra.io.util.FileUtils.ONE_MB;
+import static org.apache.cassandra.utils.Clock.Global.logInitializationOutcome;
 
 public class DatabaseDescriptor
 {
@@ -839,6 +841,8 @@ public class DatabaseDescriptor
         }
 
         Paxos.setPaxosVariant(conf.paxos_variant);
+
+        logInitializationOutcome(logger);
     }
 
     @VisibleForTesting
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchUpdatesCollector.java b/src/java/org/apache/cassandra/cql3/statements/BatchUpdatesCollector.java
index cb88bdd..e5136f4 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchUpdatesCollector.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchUpdatesCollector.java
@@ -30,7 +30,7 @@ import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 
-import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime;
 
 /**
  * Utility class to collect updates.
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 795230e..285d08c 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -78,7 +78,7 @@ import org.apache.cassandra.utils.concurrent.Promise;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
-import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime;
 
 /**
  * It represents a Keyspace.
diff --git a/src/java/org/apache/cassandra/db/Mutation.java b/src/java/org/apache/cassandra/db/Mutation.java
index a30b567..6350082 100644
--- a/src/java/org/apache/cassandra/db/Mutation.java
+++ b/src/java/org/apache/cassandra/db/Mutation.java
@@ -42,7 +42,7 @@ import org.apache.cassandra.utils.concurrent.Future;
 import static org.apache.cassandra.net.MessagingService.VERSION_30;
 import static org.apache.cassandra.net.MessagingService.VERSION_3014;
 import static org.apache.cassandra.net.MessagingService.VERSION_40;
-import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime;
 
 public class Mutation implements IMutation
 {
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index f14240b..3bf0d6d 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -72,7 +72,7 @@ import org.apache.cassandra.utils.ObjectSizes;
 import static com.google.common.collect.Iterables.any;
 import static com.google.common.collect.Iterables.filter;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
-import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime;
 import static org.apache.cassandra.db.partitions.UnfilteredPartitionIterators.MergeListener.NOOP;
 
 /**
diff --git a/src/java/org/apache/cassandra/db/ReadExecutionController.java b/src/java/org/apache/cassandra/db/ReadExecutionController.java
index 5bcd84b..2fbe3ac 100644
--- a/src/java/org/apache/cassandra/db/ReadExecutionController.java
+++ b/src/java/org/apache/cassandra/db/ReadExecutionController.java
@@ -28,7 +28,7 @@ import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.MonotonicClock;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
-import static org.apache.cassandra.utils.MonotonicClock.preciseTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.preciseTime;
 
 public class ReadExecutionController implements AutoCloseable
 {
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
index be3f8cd..6b5378f 100644
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
@@ -45,6 +45,7 @@ import static org.apache.cassandra.concurrent.Interruptible.State.NORMAL;
 import static org.apache.cassandra.concurrent.Interruptible.State.SHUTTING_DOWN;
 import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.preciseTime;
 import static org.apache.cassandra.utils.concurrent.Semaphore.newSemaphore;
 import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue;
 
@@ -151,7 +152,7 @@ public abstract class AbstractCommitLogService
             throw new IllegalArgumentException(String.format("Commit log flush interval must be positive: %fms",
                                                              syncIntervalNanos * 1e-6));
 
-        SyncRunnable sync = new SyncRunnable(MonotonicClock.preciseTime);
+        SyncRunnable sync = new SyncRunnable(preciseTime);
         executor = executorFactory().infiniteLoop(name, sync, SAFE, NON_DAEMON, SYNCHRONIZED);
     }
 
diff --git a/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java b/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java
index a6e7947..31b5404 100644
--- a/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java
+++ b/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java
@@ -18,7 +18,7 @@
 
 package org.apache.cassandra.db.monitoring;
 
-import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime;
 
 public abstract class MonitorableImpl implements Monitorable
 {
diff --git a/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java b/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
index 52d6160..d681e4b 100644
--- a/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
+++ b/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
@@ -38,7 +38,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.MonotonicClock.Global.approxTime;
 import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
 
 /**
diff --git a/src/java/org/apache/cassandra/gms/FailureDetector.java b/src/java/org/apache/cassandra/gms/FailureDetector.java
index 40a2de5..6db41de 100644
--- a/src/java/org/apache/cassandra/gms/FailureDetector.java
+++ b/src/java/org/apache/cassandra/gms/FailureDetector.java
@@ -28,7 +28,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
 import javax.management.openmbean.*;
-import org.apache.cassandra.io.util.File;
+
 import org.apache.cassandra.locator.Replica;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -41,7 +41,7 @@ import org.apache.cassandra.utils.MBeanWrapper;
 
 import static org.apache.cassandra.config.CassandraRelevantProperties.LINE_SEPARATOR;
 import static org.apache.cassandra.config.DatabaseDescriptor.newFailureDetector;
-import static org.apache.cassandra.utils.MonotonicClock.preciseTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.preciseTime;
 
 /**
  * This FailureDetector is an implementation of the paper titled
diff --git a/src/java/org/apache/cassandra/hints/HintsDispatcher.java b/src/java/org/apache/cassandra/hints/HintsDispatcher.java
index 2b6d9a3..b627338 100644
--- a/src/java/org/apache/cassandra/hints/HintsDispatcher.java
+++ b/src/java/org/apache/cassandra/hints/HintsDispatcher.java
@@ -39,7 +39,7 @@ import org.apache.cassandra.utils.concurrent.Condition;
 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.MonotonicClock.Global.approxTime;
 import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition;
 
 /**
diff --git a/src/java/org/apache/cassandra/locator/TokenMetadata.java b/src/java/org/apache/cassandra/locator/TokenMetadata.java
index fb9d43b..202bd6a 100644
--- a/src/java/org/apache/cassandra/locator/TokenMetadata.java
+++ b/src/java/org/apache/cassandra/locator/TokenMetadata.java
@@ -862,42 +862,8 @@ public class TokenMetadata
         {
             TokenMetadataDiagnostics.pendingRangeCalculationStarted(this, keyspaceName);
 
-            // create clone of current state
-            BiMultiValMap<Token, InetAddressAndPort> bootstrapTokensClone;
-            Set<InetAddressAndPort> leavingEndpointsClone;
-            Set<Pair<Token, InetAddressAndPort>> movingEndpointsClone;
-            TokenMetadata metadata;
-
-            lock.readLock().lock();
-            try
-            {
-
-                if (bootstrapTokens.isEmpty() && leavingEndpoints.isEmpty() && movingEndpoints.isEmpty())
-                {
-                    if (logger.isTraceEnabled())
-                        logger.trace("No bootstrapping, leaving or moving nodes -> empty pending ranges for {}", keyspaceName);
-                    if (bootstrapTokens.isEmpty() && leavingEndpoints.isEmpty() && movingEndpoints.isEmpty())
-                    {
-                        if (logger.isTraceEnabled())
-                            logger.trace("No bootstrapping, leaving or moving nodes -> empty pending ranges for {}", keyspaceName);
-                        pendingRanges.put(keyspaceName, new PendingRangeMaps());
-
-                        return;
-                    }
-                }
+            unsafeCalculatePendingRanges(strategy, keyspaceName);
 
-                bootstrapTokensClone  = new BiMultiValMap<>(this.bootstrapTokens);
-                leavingEndpointsClone = new HashSet<>(this.leavingEndpoints);
-                movingEndpointsClone = new HashSet<>(this.movingEndpoints);
-                metadata = this.cloneOnlyTokenMap();
-            }
-            finally
-            {
-                lock.readLock().unlock();
-            }
-
-            pendingRanges.put(keyspaceName, calculatePendingRanges(strategy, metadata, bootstrapTokensClone,
-                                                                   leavingEndpointsClone, movingEndpointsClone));
             if (logger.isDebugEnabled())
                 logger.debug("Starting pending range calculation for {}", keyspaceName);
 
@@ -910,6 +876,46 @@ public class TokenMetadata
         }
     }
 
+    public void unsafeCalculatePendingRanges(AbstractReplicationStrategy strategy, String keyspaceName)
+    {
+        // create clone of current state
+        BiMultiValMap<Token, InetAddressAndPort> bootstrapTokensClone;
+        Set<InetAddressAndPort> leavingEndpointsClone;
+        Set<Pair<Token, InetAddressAndPort>> movingEndpointsClone;
+        TokenMetadata metadata;
+
+        lock.readLock().lock();
+        try
+        {
+
+            if (bootstrapTokens.isEmpty() && leavingEndpoints.isEmpty() && movingEndpoints.isEmpty())
+            {
+                if (logger.isTraceEnabled())
+                    logger.trace("No bootstrapping, leaving or moving nodes -> empty pending ranges for {}", keyspaceName);
+                if (bootstrapTokens.isEmpty() && leavingEndpoints.isEmpty() && movingEndpoints.isEmpty())
+                {
+                    if (logger.isTraceEnabled())
+                        logger.trace("No bootstrapping, leaving or moving nodes -> empty pending ranges for {}", keyspaceName);
+                    pendingRanges.put(keyspaceName, new PendingRangeMaps());
+
+                    return;
+                }
+            }
+
+            bootstrapTokensClone  = new BiMultiValMap<>(this.bootstrapTokens);
+            leavingEndpointsClone = new HashSet<>(this.leavingEndpoints);
+            movingEndpointsClone = new HashSet<>(this.movingEndpoints);
+            metadata = this.cloneOnlyTokenMap();
+        }
+        finally
+        {
+            lock.readLock().unlock();
+        }
+
+        pendingRanges.put(keyspaceName, calculatePendingRanges(strategy, metadata, bootstrapTokensClone,
+                                                               leavingEndpointsClone, movingEndpointsClone));
+    }
+
     /**
      * @see TokenMetadata#calculatePendingRanges(AbstractReplicationStrategy, String)
      */
diff --git a/src/java/org/apache/cassandra/metrics/Sampler.java b/src/java/org/apache/cassandra/metrics/Sampler.java
index 90cc90c..b3d0f21 100644
--- a/src/java/org/apache/cassandra/metrics/Sampler.java
+++ b/src/java/org/apache/cassandra/metrics/Sampler.java
@@ -40,7 +40,7 @@ public abstract class Sampler<T>
     }
 
     @VisibleForTesting
-    MonotonicClock clock = MonotonicClock.approxTime;
+    MonotonicClock clock = MonotonicClock.Global.approxTime;
 
     @VisibleForTesting
     static final ExecutorPlus samplerExecutor = executorFactory()
diff --git a/src/java/org/apache/cassandra/net/AbstractMessageHandler.java b/src/java/org/apache/cassandra/net/AbstractMessageHandler.java
index 1045f28..e2cf68d 100644
--- a/src/java/org/apache/cassandra/net/AbstractMessageHandler.java
+++ b/src/java/org/apache/cassandra/net/AbstractMessageHandler.java
@@ -44,7 +44,7 @@ import org.apache.cassandra.net.ResourceLimits.Limit;
 import static java.lang.Math.max;
 import static java.lang.Math.min;
 import static org.apache.cassandra.net.Crc.InvalidCrc;
-import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime;
 
 /**
  * Core logic for handling inbound message deserialization and execution (in tandem with {@link FrameDecoder}).
diff --git a/src/java/org/apache/cassandra/net/InboundMessageHandler.java b/src/java/org/apache/cassandra/net/InboundMessageHandler.java
index c1b51be..e12fcec 100644
--- a/src/java/org/apache/cassandra/net/InboundMessageHandler.java
+++ b/src/java/org/apache/cassandra/net/InboundMessageHandler.java
@@ -43,7 +43,7 @@ import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.NoSpamLogger;
 
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
-import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime;
 
 /**
  * Implementation of {@link AbstractMessageHandler} for processing internode messages from peers.
diff --git a/src/java/org/apache/cassandra/net/InboundMessageHandlers.java b/src/java/org/apache/cassandra/net/InboundMessageHandlers.java
index a706557..c7b9463 100644
--- a/src/java/org/apache/cassandra/net/InboundMessageHandlers.java
+++ b/src/java/org/apache/cassandra/net/InboundMessageHandlers.java
@@ -32,7 +32,7 @@ import org.apache.cassandra.metrics.InternodeInboundMetrics;
 import org.apache.cassandra.net.Message.Header;
 
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
-import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime;
 
 /**
  * An aggregation of {@link InboundMessageHandler}s for all connections from a peer.
diff --git a/src/java/org/apache/cassandra/net/Message.java b/src/java/org/apache/cassandra/net/Message.java
index 802c79f..8fe8971 100644
--- a/src/java/org/apache/cassandra/net/Message.java
+++ b/src/java/org/apache/cassandra/net/Message.java
@@ -57,7 +57,7 @@ import static org.apache.cassandra.net.MessagingService.VERSION_3014;
 import static org.apache.cassandra.net.MessagingService.VERSION_30;
 import static org.apache.cassandra.net.MessagingService.VERSION_40;
 import static org.apache.cassandra.net.MessagingService.instance;
-import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime;
 import static org.apache.cassandra.utils.vint.VIntCoding.computeUnsignedVIntSize;
 import static org.apache.cassandra.utils.vint.VIntCoding.getUnsignedVInt;
 import static org.apache.cassandra.utils.vint.VIntCoding.skipUnsignedVInt;
diff --git a/src/java/org/apache/cassandra/net/OutboundConnection.java b/src/java/org/apache/cassandra/net/OutboundConnection.java
index fbf0c73..c2aecb0 100644
--- a/src/java/org/apache/cassandra/net/OutboundConnection.java
+++ b/src/java/org/apache/cassandra/net/OutboundConnection.java
@@ -68,7 +68,7 @@ import static org.apache.cassandra.net.ResourceLimits.*;
 import static org.apache.cassandra.net.ResourceLimits.Outcome.*;
 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.MonotonicClock.Global.approxTime;
 import static org.apache.cassandra.utils.Throwables.isCausedBy;
 import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownLatch;
 
diff --git a/src/java/org/apache/cassandra/net/RequestCallbacks.java b/src/java/org/apache/cassandra/net/RequestCallbacks.java
index 6275c15..fa1a03e 100644
--- a/src/java/org/apache/cassandra/net/RequestCallbacks.java
+++ b/src/java/org/apache/cassandra/net/RequestCallbacks.java
@@ -49,7 +49,7 @@ import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.concurrent.Stage.INTERNAL_RESPONSE;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
-import static org.apache.cassandra.utils.MonotonicClock.preciseTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.preciseTime;
 
 /**
  * An expiring map of request callbacks.
diff --git a/src/java/org/apache/cassandra/net/ResponseVerbHandler.java b/src/java/org/apache/cassandra/net/ResponseVerbHandler.java
index 369e5f4..1cee468 100644
--- a/src/java/org/apache/cassandra/net/ResponseVerbHandler.java
+++ b/src/java/org/apache/cassandra/net/ResponseVerbHandler.java
@@ -24,7 +24,7 @@ import org.apache.cassandra.exceptions.RequestFailureReason;
 import org.apache.cassandra.tracing.Tracing;
 
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
-import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime;
 
 class ResponseVerbHandler implements IVerbHandler
 {
diff --git a/src/java/org/apache/cassandra/schema/MigrationManager.java b/src/java/org/apache/cassandra/schema/MigrationManager.java
index 6fbfc5d..8e485e0 100644
--- a/src/java/org/apache/cassandra/schema/MigrationManager.java
+++ b/src/java/org/apache/cassandra/schema/MigrationManager.java
@@ -23,6 +23,8 @@ import java.lang.management.ManagementFactory;
 import java.util.function.LongSupplier;
 
 import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.utils.Simulate;
 import org.apache.cassandra.utils.concurrent.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -42,7 +44,9 @@ import org.apache.cassandra.utils.FBUtilities;
 
 import static org.apache.cassandra.concurrent.Stage.MIGRATION;
 import static org.apache.cassandra.net.Verb.SCHEMA_PUSH_REQ;
+import static org.apache.cassandra.utils.Simulate.With.GLOBAL_CLOCK;
 
+@Simulate(with = GLOBAL_CLOCK)
 public class MigrationManager
 {
     private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
index 6d5e331..90859ce 100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@ -49,6 +49,7 @@ import org.apache.cassandra.service.reads.repair.ReadRepairStrategy;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Simulate;
 
 import static java.lang.String.format;
 
@@ -58,6 +59,7 @@ import static java.util.stream.Collectors.toSet;
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal;
 import static org.apache.cassandra.schema.SchemaKeyspaceTables.*;
+import static org.apache.cassandra.utils.Simulate.With.GLOBAL_CLOCK;
 
 /**
  * system_schema.* tables and methods for manipulating them.
@@ -294,6 +296,7 @@ public final class SchemaKeyspace
     /**
      * Add entries to system_schema.* for the hardcoded system keyspaces
      */
+    @Simulate(with = GLOBAL_CLOCK)
     static void saveSystemKeyspacesSchema()
     {
         KeyspaceMetadata system = Schema.instance.getKeyspaceMetadata(SchemaConstants.SYSTEM_KEYSPACE_NAME);
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index 322fd18..7d0a290 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -43,6 +43,7 @@ import org.apache.cassandra.config.Config;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.locator.EndpointsByRange;
 import org.apache.cassandra.locator.EndpointsForRange;
+import org.apache.cassandra.utils.Simulate;
 import org.apache.cassandra.utils.concurrent.CountDownLatch;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -106,6 +107,7 @@ import static org.apache.cassandra.config.DatabaseDescriptor.*;
 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.Simulate.With.MONITORS;
 import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownLatch;
 
 /**
@@ -122,6 +124,7 @@ import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownL
  * The creation of a repair session is done through the submitRepairSession that
  * returns a future on the completion of that session.
  */
+@Simulate(with = MONITORS)
 public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFailureDetectionEventListener, ActiveRepairServiceMBean
 {
 
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index 62bd01a..3883062 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -141,6 +141,7 @@ 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.db.ConsistencyLevel.SERIAL;
 import static org.apache.cassandra.net.Message.out;
 import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.casReadMetrics;
 import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.casWriteMetrics;
@@ -152,6 +153,8 @@ import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.writeMetr
 import static org.apache.cassandra.net.NoPayload.noPayload;
 import static org.apache.cassandra.net.Verb.*;
 import static org.apache.cassandra.service.BatchlogResponseHandler.BatchlogCleanup;
+import static org.apache.cassandra.service.paxos.BallotGenerator.Global.nextBallotTimestampMicros;
+import static org.apache.cassandra.service.paxos.BallotGenerator.Global.randomBallot;
 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;
@@ -335,7 +338,6 @@ public class StorageProxy implements StorageProxyMBean
                            consistencyForPaxos,
                            consistencyForCommit,
                            consistencyForCommit,
-                           state,
                            queryStartNanoTime,
                            casWriteMetrics,
                            updateProposer);
@@ -430,7 +432,6 @@ public class StorageProxy implements StorageProxyMBean
                                        ConsistencyLevel consistencyForPaxos,
                                        ConsistencyLevel consistencyForReplayCommits,
                                        ConsistencyLevel consistencyForCommit,
-                                       ClientState state,
                                        long queryStartNanoTime,
                                        CASClientRequestMetrics casMetrics,
                                        Supplier<Pair<PartitionUpdate, RowIterator>> createUpdateProposal)
@@ -457,8 +458,7 @@ public class StorageProxy implements StorageProxyMBean
                                                                     replicaPlan,
                                                                     consistencyForPaxos,
                                                                     consistencyForReplayCommits,
-                                                                    casMetrics,
-                                                                    state);
+                                                                    casMetrics);
 
                 final UUID ballot = pair.ballot;
                 contentions += pair.contentions;
@@ -525,8 +525,7 @@ public class StorageProxy implements StorageProxyMBean
                                                                 ReplicaPlan.ForPaxosWrite paxosPlan,
                                                                 ConsistencyLevel consistencyForPaxos,
                                                                 ConsistencyLevel consistencyForCommit,
-                                                                CASClientRequestMetrics casMetrics,
-                                                                ClientState state)
+                                                                CASClientRequestMetrics casMetrics)
     throws WriteTimeoutException, WriteFailureException
     {
         long timeoutNanos = DatabaseDescriptor.getCasContentionTimeout(NANOSECONDS);
@@ -540,10 +539,10 @@ public class StorageProxy implements StorageProxyMBean
             // in progress (#5667). Lastly, we don't want to use a timestamp that is older than the last one assigned by ClientState or operations may appear
             // out-of-order (#7801).
             long minTimestampMicrosToUse = summary == null ? Long.MIN_VALUE : 1 + UUIDGen.microsTimestamp(summary.mostRecentInProgressCommit.ballot);
-            long ballotMicros = state.getTimestampForPaxos(minTimestampMicrosToUse);
+            long ballotMicros = nextBallotTimestampMicros(minTimestampMicrosToUse);
             // Note that ballotMicros is not guaranteed to be unique if two proposal are being handled concurrently by the same coordinator. But we still
             // need ballots to be unique for each proposal so we have to use getRandomTimeUUIDFromMicros.
-            UUID ballot = UUIDGen.getRandomTimeUUIDFromMicros(ballotMicros);
+            UUID ballot = randomBallot(ballotMicros, consistencyForPaxos == SERIAL);
 
             // prepare
             try
@@ -1810,7 +1809,6 @@ public class StorageProxy implements StorageProxyMBean
                         consistencyLevel,
                         consistencyForReplayCommitsOrFetch,
                         ConsistencyLevel.ANY,
-                        state,
                         start,
                         casReadMetrics,
                         updateProposer);
@@ -2077,12 +2075,12 @@ public class StorageProxy implements StorageProxyMBean
                 }
                 else
                 {
-                    MessagingService.instance().metrics.recordSelfDroppedMessage(verb, MonotonicClock.approxTime.now() - approxCreationTimeNanos, NANOSECONDS);
+                    MessagingService.instance().metrics.recordSelfDroppedMessage(verb, MonotonicClock.Global.approxTime.now() - approxCreationTimeNanos, NANOSECONDS);
                     handler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.UNKNOWN);
                 }
 
                 if (!readRejected)
-                    MessagingService.instance().latencySubscribers.add(FBUtilities.getBroadcastAddressAndPort(), MonotonicClock.approxTime.now() - approxCreationTimeNanos, NANOSECONDS);
+                    MessagingService.instance().latencySubscribers.add(FBUtilities.getBroadcastAddressAndPort(), MonotonicClock.Global.approxTime.now() - approxCreationTimeNanos, NANOSECONDS);
             }
             catch (Throwable t)
             {
@@ -2375,13 +2373,13 @@ public class StorageProxy implements StorageProxyMBean
 
         public DroppableRunnable(Verb verb)
         {
-            this.approxCreationTimeNanos = MonotonicClock.approxTime.now();
+            this.approxCreationTimeNanos = MonotonicClock.Global.approxTime.now();
             this.verb = verb;
         }
 
         public final void run()
         {
-            long approxCurrentTimeNanos = MonotonicClock.approxTime.now();
+            long approxCurrentTimeNanos = MonotonicClock.Global.approxTime.now();
             long expirationTimeNanos = verb.expiresAtNanos(approxCreationTimeNanos);
             if (approxCurrentTimeNanos > expirationTimeNanos)
             {
@@ -2408,7 +2406,7 @@ public class StorageProxy implements StorageProxyMBean
      */
     private static abstract class LocalMutationRunnable implements Runnable
     {
-        private final long approxCreationTimeNanos = MonotonicClock.approxTime.now();
+        private final long approxCreationTimeNanos = MonotonicClock.Global.approxTime.now();
 
         private final Replica localReplica;
 
@@ -2420,7 +2418,7 @@ public class StorageProxy implements StorageProxyMBean
         public final void run()
         {
             final Verb verb = verb();
-            long nowNanos = MonotonicClock.approxTime.now();
+            long nowNanos = MonotonicClock.Global.approxTime.now();
             long expirationTimeNanos = verb.expiresAtNanos(approxCreationTimeNanos);
             if (nowNanos > expirationTimeNanos)
             {
diff --git a/src/java/org/apache/cassandra/service/paxos/BallotGenerator.java b/src/java/org/apache/cassandra/service/paxos/BallotGenerator.java
new file mode 100644
index 0000000..d031f38
--- /dev/null
+++ b/src/java/org/apache/cassandra/service/paxos/BallotGenerator.java
@@ -0,0 +1,75 @@
+/*
+ * 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.service.paxos;
+
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.utils.Shared;
+import org.apache.cassandra.utils.UUIDGen;
+
+import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
+
+@Shared(scope = SIMULATION)
+public interface BallotGenerator
+{
+    static class Default implements BallotGenerator
+    {
+        public UUID randomBallot(long whenInMicros, boolean isSerial)
+        {
+            return UUIDGen.getRandomTimeUUIDFromMicros(whenInMicros, isSerial ? 2 : 1);
+        }
+
+        public UUID randomBallot(long fromInMicros, long toInMicros, boolean isSerial)
+        {
+            long timestampMicros = ThreadLocalRandom.current().nextLong(fromInMicros, toInMicros);
+            return randomBallot(timestampMicros, isSerial);
+        }
+
+        public long nextBallotTimestampMicros(long minTimestamp)
+        {
+            return ClientState.getTimestampForPaxos(minTimestamp);
+        }
+
+        public long prevBallotTimestampMicros()
+        {
+            return ClientState.getLastTimestampMicros();
+        }
+    }
+
+    static class Global
+    {
+        private static BallotGenerator instance = new Default();
+        public static UUID randomBallot(long whenInMicros, boolean isSerial) { return instance.randomBallot(whenInMicros, isSerial); }
+        public static UUID randomBallot(long fromInMicros, long toInMicros, boolean isSerial) { return instance.randomBallot(fromInMicros, toInMicros, isSerial); }
+        public static long nextBallotTimestampMicros(long minWhenInMicros) { return instance.nextBallotTimestampMicros(minWhenInMicros); }
+        public static long prevBallotTimestampMicros() { return instance.prevBallotTimestampMicros(); }
+
+        public static void unsafeSet(BallotGenerator newInstance)
+        {
+            instance = newInstance;
+        }
+    }
+
+    UUID randomBallot(long whenInMicros, boolean isSerial);
+    UUID randomBallot(long fromInMicros, long toInMicros, boolean isSerial);
+    long nextBallotTimestampMicros(long minWhenInMicros);
+    long prevBallotTimestampMicros();
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java b/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java
index dc2f9a7..64eca68 100644
--- a/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java
+++ b/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java
@@ -28,6 +28,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.net.Message;
+import org.apache.cassandra.utils.Nemesis;
 
 /**
  * ProposeCallback has two modes of operation, controlled by the failFast parameter.
@@ -46,7 +47,7 @@ public class ProposeCallback extends AbstractPaxosCallback<Boolean>
 {
     private static final Logger logger = LoggerFactory.getLogger(ProposeCallback.class);
 
-    private final AtomicInteger accepts = new AtomicInteger(0);
+    @Nemesis private final AtomicInteger accepts = new AtomicInteger(0);
     private final int requiredAccepts;
     private final boolean failFast;
 
diff --git a/src/java/org/apache/cassandra/transport/CQLMessageHandler.java b/src/java/org/apache/cassandra/transport/CQLMessageHandler.java
index bbb8cb5..09e9996 100644
--- a/src/java/org/apache/cassandra/transport/CQLMessageHandler.java
+++ b/src/java/org/apache/cassandra/transport/CQLMessageHandler.java
@@ -46,7 +46,7 @@ import org.apache.cassandra.transport.Flusher.FlushItem.Framed;
 import org.apache.cassandra.transport.messages.ErrorMessage;
 import org.apache.cassandra.utils.NoSpamLogger;
 
-import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime;
 
 /**
  * Implementation of {@link AbstractMessageHandler} for processing CQL messages which comprise a {@link Message} wrapped
diff --git a/src/java/org/apache/cassandra/utils/Clock.java b/src/java/org/apache/cassandra/utils/Clock.java
index d1a7337..acdfc82 100644
--- a/src/java/org/apache/cassandra/utils/Clock.java
+++ b/src/java/org/apache/cassandra/utils/Clock.java
@@ -37,10 +37,12 @@ import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
 @Shared(scope = SIMULATION)
 public interface Clock
 {
-    static final Logger logger = LoggerFactory.getLogger(Clock.class);
-
     public static class Global
     {
+        // something weird happens with class loading Logger that can cause a deadlock
+        private static Throwable FAILED_TO_INITIALISE;
+        private static String INITIALIZE_MESSAGE;
+
         /**
          * 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
@@ -52,19 +54,38 @@ public interface Clock
         {
             String classname = CLOCK_GLOBAL.getString();
             Clock clock = new Default();
+            Throwable errorOutcome = null;
+            String outcome = null;
             if (classname != null)
             {
                 try
                 {
-                    logger.debug("Using custom clock implementation: {}", classname);
+                    outcome = "Using custom clock implementation: " + classname;
                     clock = (Clock) Class.forName(classname).newInstance();
                 }
-                catch (Exception e)
+                catch (Throwable t)
                 {
-                    logger.error("Failed to load clock implementation {}", classname, e);
+                    outcome = "Failed to load clock implementation " + classname;
+                    errorOutcome = t;
                 }
             }
             instance = clock;
+            FAILED_TO_INITIALISE = errorOutcome;
+            INITIALIZE_MESSAGE = outcome;
+        }
+
+        public static void logInitializationOutcome(Logger logger)
+        {
+            if (FAILED_TO_INITIALISE != null)
+            {
+                logger.error(INITIALIZE_MESSAGE, FAILED_TO_INITIALISE);
+            }
+            else if (INITIALIZE_MESSAGE != null)
+            {
+                logger.debug(INITIALIZE_MESSAGE);
+            }
+            FAILED_TO_INITIALISE = null;
+            INITIALIZE_MESSAGE = null;
         }
 
         /**
diff --git a/src/java/org/apache/cassandra/utils/MonotonicClock.java b/src/java/org/apache/cassandra/utils/MonotonicClock.java
index e14fd45..d4590c9 100644
--- a/src/java/org/apache/cassandra/utils/MonotonicClock.java
+++ b/src/java/org/apache/cassandra/utils/MonotonicClock.java
@@ -39,7 +39,7 @@ import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
  * Wrapper around time related functions that are either implemented by using the default JVM calls
  * or by using a custom implementation for testing purposes.
  *
- * See {@link #preciseTime} for how to use a custom implementation.
+ * See {@link Global#preciseTime} for how to use a custom implementation.
  *
  * Please note that {@link java.time.Clock} wasn't used, as it would not be possible to provide an
  * implementation for {@link #now()} with the exact same properties of {@link System#nanoTime()}.
@@ -49,13 +49,6 @@ import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
 @Shared(scope = SIMULATION)
 public interface MonotonicClock
 {
-    /**
-     * 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
-     * different implementation instead.
-     */
-    public static final MonotonicClock preciseTime = Defaults.precise();
-    public static final MonotonicClock approxTime = Defaults.approx(preciseTime);
 
     /**
      * @see System#nanoTime()
@@ -77,10 +70,18 @@ public interface MonotonicClock
     public boolean isAfter(long instant);
     public boolean isAfter(long now, long instant);
 
-    static class Defaults
+    public static class Global
     {
         private static final Logger logger = LoggerFactory.getLogger(MonotonicClock.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
+         * different implementation instead.
+         */
+        public static final MonotonicClock preciseTime = precise();
+        public static final MonotonicClock approxTime = approx(preciseTime);
+
         private static MonotonicClock precise()
         {
             String sclock = CLOCK_MONOTONIC_PRECISE.getString();
diff --git a/src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java b/src/java/org/apache/cassandra/utils/Nemesis.java
similarity index 50%
copy from src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java
copy to src/java/org/apache/cassandra/utils/Nemesis.java
index ecf073d..b5110c4 100644
--- a/src/java/org/apache/cassandra/concurrent/ScheduledExecutorPlus.java
+++ b/src/java/org/apache/cassandra/utils/Nemesis.java
@@ -16,15 +16,28 @@
  * limitations under the License.
  */
 
-package org.apache.cassandra.concurrent;
+package org.apache.cassandra.utils;
 
-import java.util.concurrent.ScheduledExecutorService;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
 
-import org.apache.cassandra.utils.Shared;
+import static org.apache.cassandra.utils.Nemesis.Traffic.HIGH;
 
-import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
-
-@Shared(scope = SIMULATION)
-public interface ScheduledExecutorPlus extends ExecutorPlus, ScheduledExecutorService
+/**
+ * Annotate fields, particularly important volatile fields, where the system should adversarially schedule
+ * thread events around memory accesses (read or write).
+ *
+ * This can introduce significant simulation overhead, so should be used sparingly.
+ *
+ * TODO: Support @Nemesis on methods, to insert nemesis points either before or after invocations of the method
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ ElementType.FIELD })
+public @interface Nemesis
 {
+    enum Traffic { LOW, HIGH }
+
+    Traffic traffic() default HIGH;
 }
diff --git a/src/java/org/apache/cassandra/utils/Simulate.java b/src/java/org/apache/cassandra/utils/Simulate.java
new file mode 100644
index 0000000..dd0d230
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/Simulate.java
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/**
+ * Enable certain features for a specific method or class.
+ *
+ * Note that presently class level annotations are not inherited by inner classes.
+ *
+ * TODO: support package level, and apply to all nested classes
+ */
+public @interface Simulate
+{
+    enum With
+    {
+        /**
+         * Calls to FBUtilities.timestampMicros() will be guaranteed globally monotonically increasing.
+         *
+         * May be annotated at the method or class level.
+         */
+        GLOBAL_CLOCK,
+
+        /**
+         * synchronized methods and blocks, and wait/notify.
+         *
+         * May be annotated at the class level.
+         */
+        MONITORS,
+
+        /**
+         * Usages of LockSupport. This defaults to ON for all classes, including system classes.
+         *
+         * May be annotated at the method or class level.
+         */
+        LOCK_SUPPORT
+    }
+
+    With[] with() default {};
+    With[] without() default {};
+}
diff --git a/src/java/org/apache/cassandra/utils/concurrent/AbstractFuture.java b/src/java/org/apache/cassandra/utils/concurrent/AbstractFuture.java
index 86e3c12..026f9f2 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/AbstractFuture.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/AbstractFuture.java
@@ -316,6 +316,18 @@ public abstract class AbstractFuture<V> implements Future<V>
         return this;
     }
 
+
+    /**
+     * Support {@link com.google.common.util.concurrent.Futures#transformAsync(ListenableFuture, AsyncFunction, Executor)} natively
+     *
+     * See {@link #addListener(GenericFutureListener)} for ordering semantics.
+     */
+    @Override
+    public <T> Future<T> map(Function<? super V, ? extends T> mapper)
+    {
+        return map(mapper, null);
+    }
+
     /**
      * Support more fluid version of {@link com.google.common.util.concurrent.Futures#addCallback}
      *
@@ -435,31 +447,31 @@ public abstract class AbstractFuture<V> implements Future<V>
     @Override
     public boolean await(long timeout, TimeUnit unit) throws InterruptedException
     {
-        return Awaitable.await(this, timeout, unit);
+        return Defaults.await(this, timeout, unit);
     }
 
     @Override
     public boolean awaitThrowUncheckedOnInterrupt(long time, TimeUnit units) throws UncheckedInterruptedException
     {
-        return Awaitable.awaitThrowUncheckedOnInterrupt(this, time, units);
+        return Defaults.awaitThrowUncheckedOnInterrupt(this, time, units);
     }
 
     @Override
     public boolean awaitUninterruptibly(long timeout, TimeUnit unit)
     {
-        return Awaitable.awaitUninterruptibly(this, timeout, unit);
+        return Defaults.awaitUninterruptibly(this, timeout, unit);
     }
 
     @Override
     public boolean awaitUntilThrowUncheckedOnInterrupt(long nanoTimeDeadline) throws UncheckedInterruptedException
     {
-        return Awaitable.awaitUntilThrowUncheckedOnInterrupt(this, nanoTimeDeadline);
+        return Defaults.awaitUntilThrowUncheckedOnInterrupt(this, nanoTimeDeadline);
     }
 
     @Override
     public boolean awaitUntilUninterruptibly(long nanoTimeDeadline)
     {
-        return Awaitable.awaitUntilUninterruptibly(this, nanoTimeDeadline);
+        return Defaults.awaitUntilUninterruptibly(this, nanoTimeDeadline);
     }
 
     /**
@@ -468,7 +480,7 @@ public abstract class AbstractFuture<V> implements Future<V>
     @Override
     public Future<V> awaitUninterruptibly()
     {
-        return Awaitable.awaitUninterruptibly(this);
+        return Defaults.awaitUninterruptibly(this);
     }
 
     /**
@@ -477,7 +489,7 @@ public abstract class AbstractFuture<V> implements Future<V>
     @Override
     public Future<V> awaitThrowUncheckedOnInterrupt() throws UncheckedInterruptedException
     {
-        return Awaitable.awaitThrowUncheckedOnInterrupt(this);
+        return Defaults.awaitThrowUncheckedOnInterrupt(this);
     }
 
     public String toString()
diff --git a/src/java/org/apache/cassandra/utils/concurrent/AsyncFuture.java b/src/java/org/apache/cassandra/utils/concurrent/AsyncFuture.java
index b09eeb7..0ef35d5 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/AsyncFuture.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/AsyncFuture.java
@@ -123,7 +123,7 @@ public class AsyncFuture<V> extends AbstractFuture<V>
     }
 
     /**
-     * Support {@link com.google.common.util.concurrent.Futures#transform(ListenableFuture, com.google.common.base.Function, Executor)} natively
+     * Support {@link com.google.common.util.concurrent.Futures#transform} natively
      *
      * See {@link #addListener(GenericFutureListener)} for ordering semantics.
      */
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Awaitable.java b/src/java/org/apache/cassandra/utils/concurrent/Awaitable.java
index 03aab5f..25bdf02 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Awaitable.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Awaitable.java
@@ -105,91 +105,96 @@ public interface Awaitable
      */
     Awaitable awaitUninterruptibly();
 
-    public static boolean await(Awaitable await, long time, TimeUnit unit) throws InterruptedException
+    // we must declare the static implementation methods outside of the interface,
+    // so that they can be loaded by different classloaders during simulation
+    class Defaults
     {
-        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)
+        public static boolean await(Awaitable await, long time, TimeUnit unit) throws InterruptedException
         {
-            throw new UncheckedInterruptedException();
+            return await.awaitUntil(nanoTime() + unit.toNanos(time));
         }
-        return await;
-    }
 
-    public static boolean awaitUntilThrowUncheckedOnInterrupt(Awaitable await, long nanoTimeDeadline) throws UncheckedInterruptedException
-    {
-        try
+        public static boolean awaitThrowUncheckedOnInterrupt(Awaitable await, long time, TimeUnit units) throws UncheckedInterruptedException
         {
-            return await.awaitUntil(nanoTimeDeadline);
+            return awaitUntilThrowUncheckedOnInterrupt(await, nanoTime() + units.toNanos(time));
         }
-        catch (InterruptedException e)
+
+        public static boolean awaitUninterruptibly(Awaitable await, long time, TimeUnit units)
         {
-            throw new UncheckedInterruptedException();
+            return awaitUntilUninterruptibly(await, nanoTime() + units.toNanos(time));
         }
-    }
 
-    /**
-     * {@link Awaitable#awaitUntilUninterruptibly(long)}
-     */
-    public static boolean awaitUntilUninterruptibly(Awaitable await, long nanoTimeDeadline)
-    {
-        boolean interrupted = false;
-        boolean result;
-        while (true)
+        public static <A extends Awaitable> A awaitThrowUncheckedOnInterrupt(A await) throws UncheckedInterruptedException
         {
             try
             {
-                result = await.awaitUntil(nanoTimeDeadline);
-                break;
+                await.await();
             }
             catch (InterruptedException e)
             {
-                interrupted = true;
+                throw new UncheckedInterruptedException();
             }
+            return await;
         }
-        if (interrupted)
-            Thread.currentThread().interrupt();
-        return result;
-    }
 
-    /**
-     * {@link Awaitable#awaitUninterruptibly()}
-     */
-    public static <A extends Awaitable> A awaitUninterruptibly(A await)
-    {
-        boolean interrupted = false;
-        while (true)
+        public static boolean awaitUntilThrowUncheckedOnInterrupt(Awaitable await, long nanoTimeDeadline) throws UncheckedInterruptedException
         {
             try
             {
-                await.await();
-                break;
+                return await.awaitUntil(nanoTimeDeadline);
             }
             catch (InterruptedException e)
             {
-                interrupted = true;
+                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;
         }
-        if (interrupted)
-            Thread.currentThread().interrupt();
-        return await;
     }
 
     abstract class AbstractAwaitable implements Awaitable
@@ -202,7 +207,7 @@ public interface Awaitable
         @Override
         public boolean await(long time, TimeUnit unit) throws InterruptedException
         {
-            return Awaitable.await(this, time, unit);
+            return Defaults.await(this, time, unit);
         }
 
         /**
@@ -211,7 +216,7 @@ public interface Awaitable
         @Override
         public boolean awaitThrowUncheckedOnInterrupt(long time, TimeUnit units) throws UncheckedInterruptedException
         {
-            return Awaitable.awaitThrowUncheckedOnInterrupt(this, time, units);
+            return Defaults.awaitThrowUncheckedOnInterrupt(this, time, units);
         }
 
         /**
@@ -227,7 +232,7 @@ public interface Awaitable
          */
         public Awaitable awaitThrowUncheckedOnInterrupt() throws UncheckedInterruptedException
         {
-            return Awaitable.awaitThrowUncheckedOnInterrupt(this);
+            return Defaults.awaitThrowUncheckedOnInterrupt(this);
         }
 
         /**
@@ -235,7 +240,7 @@ public interface Awaitable
          */
         public boolean awaitUntilThrowUncheckedOnInterrupt(long nanoTimeDeadline) throws UncheckedInterruptedException
         {
-            return Awaitable.awaitUntilThrowUncheckedOnInterrupt(this, nanoTimeDeadline);
+            return Defaults.awaitUntilThrowUncheckedOnInterrupt(this, nanoTimeDeadline);
         }
 
         /**
@@ -243,7 +248,7 @@ public interface Awaitable
          */
         public boolean awaitUntilUninterruptibly(long nanoTimeDeadline)
         {
-            return Awaitable.awaitUntilUninterruptibly(this, nanoTimeDeadline);
+            return Defaults.awaitUntilUninterruptibly(this, nanoTimeDeadline);
         }
 
         /**
@@ -251,7 +256,7 @@ public interface Awaitable
          */
         public Awaitable awaitUninterruptibly()
         {
-            return Awaitable.awaitUninterruptibly(this);
+            return Defaults.awaitUninterruptibly(this);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Future.java b/src/java/org/apache/cassandra/utils/concurrent/Future.java
index 69dc83d..fae5d43 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Future.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Future.java
@@ -96,6 +96,17 @@ public interface Future<V> extends io.netty.util.concurrent.Future<V>, Listenabl
         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
+     */
+    default Future<V> syncThrowUncheckedOnInterrupt()
+    {
+        awaitThrowUncheckedOnInterrupt();
+        rethrowIfFailed();
+        return this;
+    }
+
     @Deprecated
     @Override
     default boolean await(long l) throws InterruptedException
diff --git a/src/java/org/apache/cassandra/utils/concurrent/ListenerList.java b/src/java/org/apache/cassandra/utils/concurrent/ListenerList.java
index 57737ea..40b908b 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/ListenerList.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/ListenerList.java
@@ -38,7 +38,7 @@ import static org.apache.cassandra.utils.concurrent.ListenerList.Notifying.NOTIF
 /**
  * Encapsulate one or more items in a linked-list that is immutable whilst shared, forming a prepend-only list (or stack).
  * Once the list is ready to consume, exclusive ownership is taken by clearing the shared variable containing it, after
- * which the list may be invoked using {@link #notify}, which reverses the list before invoking the work it contains.
+ * which the list may be invoked using {@link #notifyExclusive(ListenerList, Future)}, which reverses the list before invoking the work it contains.
  */
 abstract class ListenerList<V> extends IntrusiveStack<ListenerList<V>>
 {
@@ -93,7 +93,7 @@ abstract class ListenerList<V> extends IntrusiveStack<ListenerList<V>>
             {
                 while (true)
                 {
-                    notify(listeners, in);
+                    notifyExclusive(listeners, in);
                     if (updater.compareAndSet(in, NOTIFYING, null))
                         return;
 
@@ -113,17 +113,13 @@ abstract class ListenerList<V> extends IntrusiveStack<ListenerList<V>>
      *
      * @param head must be either a {@link ListenerList} or {@link GenericFutureListener}
      */
-    static <T> void notify(ListenerList<T> head, Future<T> future)
+    static <T> void notifyExclusive(ListenerList<T> head, Future<T> future)
     {
-        Executor notifyExecutor = future.notifyExecutor();
-        if (inExecutor(notifyExecutor))
-            notifyExecutor = null;
-
-        notify(head, notifyExecutor, future);
-    }
+        Executor notifyExecutor; {
+            Executor exec = future.notifyExecutor();
+            notifyExecutor = inExecutor(exec) ? null : exec;
+        }
 
-    private static <T> void notify(ListenerList<T> head, Executor notifyExecutor, Future<T> future)
-    {
         head = reverse(head);
         forEach(head, i -> i.notifySelf(notifyExecutor, future));
     }
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Ref.java b/src/java/org/apache/cassandra/utils/concurrent/Ref.java
index c077467..f40e08f 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Ref.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Ref.java
@@ -47,12 +47,15 @@ import org.apache.cassandra.io.util.SafeMemory;
 import org.apache.cassandra.utils.ExecutorUtils;
 import org.apache.cassandra.utils.NoSpamLogger;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.Shared;
+
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 
 import static java.util.Collections.emptyList;
 
 import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.SimulatorSafe.UNSAFE;
+import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
 import static org.apache.cassandra.utils.Throwables.maybeFail;
 import static org.apache.cassandra.utils.Throwables.merge;
 
@@ -91,6 +94,13 @@ public final class Ref<T> implements RefCounted<T>
 {
     static final Logger logger = LoggerFactory.getLogger(Ref.class);
     public static final boolean DEBUG_ENABLED = System.getProperty("cassandra.debugrefcount", "false").equalsIgnoreCase("true");
+    static OnLeak ON_LEAK;
+
+    @Shared(scope = SIMULATION)
+    public interface OnLeak
+    {
+        void onLeak(Object state);
+    }
 
     final State state;
     final T referent;
@@ -227,6 +237,9 @@ public final class Ref<T> implements RefCounted<T>
                 logger.error("LEAK DETECTED: a reference ({}) to {} was not released before the reference was garbage collected", id, globalState);
                 if (DEBUG_ENABLED)
                     debug.log(id);
+                OnLeak onLeak = ON_LEAK;
+                if (onLeak != null)
+                    onLeak.onLeak(this);
             }
             else if (DEBUG_ENABLED)
             {
@@ -235,6 +248,12 @@ public final class Ref<T> implements RefCounted<T>
             if (fail != null)
                 logger.error("Error when closing {}", globalState, fail);
         }
+
+        @Override
+        public String toString()
+        {
+            return globalState.toString();
+        }
     }
 
     static final class Debug
@@ -678,7 +697,10 @@ public final class Ref<T> implements RefCounted<T>
         {
             final Set<Tidy> candidates = Collections.newSetFromMap(new IdentityHashMap<>());
             for (GlobalState state : globallyExtant)
-                candidates.add(state.tidy);
+            {
+                if (state.tidy != null)
+                    candidates.add(state.tidy);
+            }
             removeExpected(candidates);
             this.candidates.retainAll(candidates);
             if (!this.candidates.isEmpty())
@@ -706,6 +728,11 @@ public final class Ref<T> implements RefCounted<T>
         }
     }
 
+    public static void setOnLeak(OnLeak onLeak)
+    {
+        ON_LEAK = onLeak;
+    }
+
     @VisibleForTesting
     public static void shutdownReferenceReaper(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
     {
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Semaphore.java b/src/java/org/apache/cassandra/utils/concurrent/Semaphore.java
index c3f03a5..01c52c5 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Semaphore.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Semaphore.java
@@ -19,15 +19,10 @@
 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 org.apache.cassandra.utils.Shared;
 
-import static java.lang.System.nanoTime;
-import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue;
 import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
 
 @Shared(scope = SIMULATION)
@@ -88,7 +83,7 @@ public interface Semaphore
     @Intercept
     public static Semaphore newSemaphore(int permits)
     {
-        return new UnfairAsync(permits);
+        return new Standard(permits, false);
     }
 
     /**
@@ -99,160 +94,19 @@ public interface Semaphore
     @Intercept
     public static Semaphore newFairSemaphore(int permits)
     {
-        return new FairJDK(permits);
+        return new Standard(permits, true);
     }
 
-    /**
-     * 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
+    public static class Standard extends java.util.concurrent.Semaphore 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
+        public Standard(int permits)
         {
-            acquire(acquire, WaitQueue.Signal::await);
+            this(permits, false);
         }
 
-        /**
-         * {@link Semaphore#acquireThrowUncheckedOnInterrupt(int)}
-         */
-        public void acquireThrowUncheckedOnInterrupt(int acquire)
+        public Standard(int permits, boolean fair)
         {
-            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); // checkstyle: permit this instantiation
+            super(permits, fair);
         }
 
         /**
@@ -260,7 +114,7 @@ public interface Semaphore
          */
         public int drain()
         {
-            return wrapped.drainPermits();
+            return drainPermits();
         }
 
         /**
@@ -268,7 +122,7 @@ public interface Semaphore
          */
         public int permits()
         {
-            return wrapped.availablePermits();
+            return availablePermits();
         }
 
         /**
@@ -276,31 +130,7 @@ public interface Semaphore
          */
         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);
+            return getQueueLength();
         }
 
         /**
@@ -309,15 +139,7 @@ public interface Semaphore
         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);
+            return tryAcquire(acquire, Math.max(0, wait), TimeUnit.NANOSECONDS);
         }
 
         @Override
diff --git a/src/java/org/apache/cassandra/utils/concurrent/SyncFuture.java b/src/java/org/apache/cassandra/utils/concurrent/SyncFuture.java
index 43648c0..a7b3473 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/SyncFuture.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/SyncFuture.java
@@ -90,7 +90,7 @@ public class SyncFuture<V> extends AbstractFuture<V>
     }
 
     /**
-     * Support {@link com.google.common.util.concurrent.Futures#transform(ListenableFuture, com.google.common.base.Function, Executor)} natively
+     * Support {@link com.google.common.util.concurrent.Futures#transform} natively
      *
      * See {@link #addListener(GenericFutureListener)} for ordering semantics.
      */
@@ -165,7 +165,7 @@ public class SyncFuture<V> extends AbstractFuture<V>
 
     private void notifyListeners()
     {
-        ListenerList.notify(listeners, this);
+        ListenerList.notifyExclusive(listeners, this);
         listenersUpdater.lazySet(this, null);
     }
 }
diff --git a/src/java/org/apache/cassandra/utils/memory/BufferPool.java b/src/java/org/apache/cassandra/utils/memory/BufferPool.java
index d656616..f302f4f 100644
--- a/src/java/org/apache/cassandra/utils/memory/BufferPool.java
+++ b/src/java/org/apache/cassandra/utils/memory/BufferPool.java
@@ -48,6 +48,7 @@ import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.metrics.BufferPoolMetrics;
 import org.apache.cassandra.utils.NoSpamLogger;
+import org.apache.cassandra.utils.Shared;
 import org.apache.cassandra.utils.concurrent.Ref;
 
 import static com.google.common.collect.ImmutableList.of;
@@ -55,6 +56,7 @@ import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFac
 import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.SimulatorSafe.UNSAFE;
 import static org.apache.cassandra.utils.ExecutorUtils.*;
 import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemory;
+import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
 import static org.apache.cassandra.utils.memory.MemoryUtil.isExactlyDirect;
 
 /**
@@ -119,6 +121,7 @@ public class BufferPool
     private static final ByteBuffer EMPTY_BUFFER = ByteBuffer.allocateDirect(0);
 
     private volatile Debug debug = Debug.NO_OP;
+    private volatile DebugLeaks debugLeaks = DebugLeaks.NO_OP;
 
     protected final String name;
     protected final BufferPoolMetrics metrics;
@@ -305,10 +308,19 @@ public class BufferPool
         void recyclePartial(Chunk chunk);
     }
 
-    public void debug(Debug setDebug)
+    @Shared(scope = SIMULATION)
+    public interface DebugLeaks
     {
-        assert setDebug != null;
-        this.debug = setDebug;
+        public static DebugLeaks NO_OP = () -> {};
+        void leak();
+    }
+
+    public void debug(Debug newDebug, DebugLeaks newDebugLeaks)
+    {
+        if (newDebug != null)
+            this.debug = newDebug;
+        if (newDebugLeaks != null)
+            this.debugLeaks = newDebugLeaks;
     }
 
     interface Recycler
@@ -1025,6 +1037,7 @@ public class BufferPool
         Object obj = localPoolRefQueue.remove(100);
         if (obj instanceof LocalPoolRef)
         {
+            debugLeaks.leak();
             ((LocalPoolRef) obj).release();
             localPoolReferences.remove(obj);
         }
diff --git a/src/java/org/apache/cassandra/utils/memory/HeapPool.java b/src/java/org/apache/cassandra/utils/memory/HeapPool.java
index 0596aeb..ebe92ac 100644
--- a/src/java/org/apache/cassandra/utils/memory/HeapPool.java
+++ b/src/java/org/apache/cassandra/utils/memory/HeapPool.java
@@ -21,8 +21,11 @@ package org.apache.cassandra.utils.memory;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.utils.Shared;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
+import static org.apache.cassandra.utils.Shared.Scope.SIMULATION;
+
 public class HeapPool extends MemtablePool
 {
     private static final EnsureOnHeap ENSURE_NOOP = new EnsureOnHeap.NoOp();
@@ -58,6 +61,7 @@ public class HeapPool extends MemtablePool
 
     public static class Logged extends MemtablePool
     {
+        @Shared(scope = SIMULATION)
         public interface Listener
         {
             public void accept(long size, String table);
diff --git a/test/burn/org/apache/cassandra/utils/memory/LongBufferPoolTest.java b/test/burn/org/apache/cassandra/utils/memory/LongBufferPoolTest.java
index d9e8372..9bb217c 100644
--- a/test/burn/org/apache/cassandra/utils/memory/LongBufferPoolTest.java
+++ b/test/burn/org/apache/cassandra/utils/memory/LongBufferPoolTest.java
@@ -265,7 +265,7 @@ public class LongBufferPoolTest
         logger.info("{} - testing {} threads for {}m", DATE_FORMAT.format(new Date()), threadCount, TimeUnit.NANOSECONDS.toMinutes(duration));
         logger.info("Testing BufferPool with memoryUsageThreshold={} and enabling BufferPool.DEBUG", bufferPool.memoryUsageThreshold());
         Debug debug = new Debug();
-        bufferPool.debug(debug);
+        bufferPool.debug(debug, null);
 
         TestEnvironment testEnv = new TestEnvironment(threadCount, duration, bufferPool.memoryUsageThreshold());
 
@@ -305,7 +305,7 @@ public class LongBufferPoolTest
         assertEquals(0, testEnv.executorService.shutdownNow().size());
 
         logger.info("Reverting BufferPool DEBUG config");
-        bufferPool.debug(BufferPool.Debug.NO_OP);
+        bufferPool.debug(BufferPool.Debug.NO_OP, null);
 
         testEnv.assertCheckedThreadsSucceeded();
 
diff --git a/test/unit/org/apache/cassandra/concurrent/AbstractExecutorPlusTest.java b/test/unit/org/apache/cassandra/concurrent/AbstractExecutorPlusTest.java
index eadacd1..52650ad 100644
--- a/test/unit/org/apache/cassandra/concurrent/AbstractExecutorPlusTest.java
+++ b/test/unit/org/apache/cassandra/concurrent/AbstractExecutorPlusTest.java
@@ -32,6 +32,8 @@ import org.apache.cassandra.utils.WithResources;
 import org.apache.cassandra.utils.concurrent.Future;
 import org.apache.cassandra.utils.concurrent.Semaphore;
 
+import static org.apache.cassandra.utils.concurrent.Semaphore.newSemaphore;
+
 @Ignore
 public abstract class AbstractExecutorPlusTest
 {
@@ -100,9 +102,9 @@ public abstract class AbstractExecutorPlusTest
 
         SequentialExecutorPlus exec = builder.build();
 
-        Semaphore enter = new Semaphore.UnfairAsync(0);
-        Semaphore exit = new Semaphore.UnfairAsync(0);
-        Semaphore runAfter = new Semaphore.UnfairAsync(0);
+        Semaphore enter = newSemaphore(0);
+        Semaphore exit = newSemaphore(0);
+        Semaphore runAfter = newSemaphore(0);
         SequentialExecutorPlus.AtLeastOnceTrigger trigger;
         trigger = exec.atLeastOnceTrigger(() -> { enter.release(1); exit.acquireThrowUncheckedOnInterrupt(1); });
 
diff --git a/test/unit/org/apache/cassandra/utils/concurrent/SemaphoreTest.java b/test/unit/org/apache/cassandra/utils/concurrent/SemaphoreTest.java
index 77ed5ab..c4cb86d 100644
--- a/test/unit/org/apache/cassandra/utils/concurrent/SemaphoreTest.java
+++ b/test/unit/org/apache/cassandra/utils/concurrent/SemaphoreTest.java
@@ -36,7 +36,7 @@ public class SemaphoreTest
     @Test
     public void testUnfair() throws InterruptedException
     {
-        Semaphore s = new Semaphore.UnfairAsync(2);
+        Semaphore s = Semaphore.newSemaphore(2);
         List<Future<Boolean>> fs = start(s);
         s.release(1);
         while (s.permits() == 1) Thread.yield();
@@ -54,7 +54,7 @@ public class SemaphoreTest
     @Test
     public void testFair() throws InterruptedException, ExecutionException, TimeoutException
     {
-        Semaphore s = new Semaphore.FairJDK(2);
+        Semaphore s = Semaphore.newFairSemaphore(2);
         List<Future<Boolean>> fs = start(s);
         s.release(1);
         fs.get(0).get(1L, TimeUnit.MINUTES);
@@ -83,9 +83,9 @@ public class SemaphoreTest
             try { s.tryAcquireUntil(1, System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(1L)); Assert.fail(); } catch (InterruptedException ignore) { }
             List<Future<Boolean>> fs = new ArrayList<>();
             fs.add(exec.submit(() -> s.tryAcquire(1, 1L, TimeUnit.MINUTES)));
-            while (s instanceof Semaphore.FairJDK && ((Semaphore.FairJDK) s).waiting() == 0) Thread.yield();
+            while (s instanceof Semaphore.Standard && ((Semaphore.Standard) s).waiting() == 0) Thread.yield();
             fs.add(exec.submit(() -> s.tryAcquireUntil(1, System.nanoTime() + TimeUnit.MINUTES.toNanos(1L))));
-            while (s instanceof Semaphore.FairJDK && ((Semaphore.FairJDK) s).waiting() == 1) Thread.yield();
+            while (s instanceof Semaphore.Standard && ((Semaphore.Standard) s).waiting() == 1) Thread.yield();
             fs.add(exec.submit(() -> { s.acquire(1); return true; } ));
             return fs;
         }

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


[cassandra] 01/03: [CEP-10] Cluster and Code Simulations: Minor fixes

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 31bea0b0d41e4e81095f0d088094f03db14af490
Author: Benedict Elliott Smith <be...@apple.com>
AuthorDate: Wed Nov 17 14:50:09 2021 +0000

    [CEP-10] Cluster and Code Simulations: Minor fixes
    
    - fix repair timeout
    - fix secondary index flushing
    - fix race condition with Ref
    - fix resource leaks
    - fix incorrect exists()
    - fix repair error reporting when null exception message
    - fix dtest use of System.nanoTime
    
    patch by Benedict; reviewed by Sam Tunnicliffe for CASSANDRA-17008
---
 .../apache/cassandra/cache/AutoSavingCache.java    |  1 -
 .../cassandra/concurrent/NamedThreadFactory.java   |  2 +-
 .../org/apache/cassandra/db/ColumnFamilyStore.java |  2 +-
 src/java/org/apache/cassandra/db/Directories.java  |  1 -
 src/java/org/apache/cassandra/db/ReadResponse.java |  1 -
 .../org/apache/cassandra/db/SystemKeyspace.java    |  1 -
 .../cassandra/db/WindowsFailedSnapshotTracker.java |  2 --
 .../commitlog/CommitLogSegmentManagerStandard.java |  1 -
 .../apache/cassandra/db/compaction/Scrubber.java   |  1 -
 .../apache/cassandra/db/compaction/Verifier.java   | 11 ++-----
 .../compaction/writers/CompactionAwareWriter.java  |  1 -
 .../apache/cassandra/db/lifecycle/LogRecord.java   |  1 -
 .../org/apache/cassandra/gms/EndpointState.java    |  1 -
 .../org/apache/cassandra/gms/GossipDigest.java     |  1 -
 .../org/apache/cassandra/gms/GossipDigestAck2.java |  1 -
 .../org/apache/cassandra/gms/GossipDigestSyn.java  |  1 -
 .../org/apache/cassandra/gms/HeartBeatState.java   |  1 -
 .../org/apache/cassandra/gms/TokenSerializer.java  |  3 --
 .../org/apache/cassandra/gms/VersionedValue.java   |  1 -
 .../cassandra/hints/ChecksummedDataInput.java      |  1 -
 .../apache/cassandra/index/sasi/TermIterator.java  |  4 ---
 .../index/sasi/disk/OnDiskIndexBuilder.java        |  1 -
 .../io/compress/CompressedSequentialWriter.java    |  1 -
 .../cassandra/io/compress/CompressionMetadata.java |  3 --
 .../apache/cassandra/io/sstable/Descriptor.java    |  2 --
 .../io/sstable/SSTableIdentityIterator.java        |  1 -
 .../cassandra/io/sstable/format/SSTableReader.java | 30 ++++++++++-------
 .../io/sstable/format/SSTableReaderBuilder.java    |  2 --
 .../io/sstable/format/big/BigTableWriter.java      |  2 +-
 .../io/sstable/metadata/MetadataSerializer.java    |  1 -
 .../io/util/BufferedDataOutputStreamPlus.java      |  3 ++
 .../org/apache/cassandra/io/util/FileHandle.java   | 12 +++++--
 .../org/apache/cassandra/io/util/PathUtils.java    |  2 --
 src/java/org/apache/cassandra/net/Verb.java        |  2 +-
 .../repair/consistent/SyncStatSummary.java         |  1 -
 .../schema/SystemDistributedKeyspace.java          |  5 ++-
 .../apache/cassandra/service/StartupChecks.java    |  2 --
 .../apache/cassandra/service/StorageService.java   |  1 -
 .../org/apache/cassandra/service/paxos/Commit.java | 34 +++++++++++++++++++
 .../apache/cassandra/service/paxos/PaxosState.java |  6 ++--
 .../service/snapshot/SnapshotManifest.java         | 14 ++++++--
 .../cassandra/streaming/StreamResultFuture.java    |  5 ++-
 .../apache/cassandra/streaming/StreamSession.java  |  9 +++++
 .../cassandra/streaming/StreamTransferTask.java    | 38 +++++++++++++---------
 .../async/StreamingMultiplexedChannel.java         |  1 +
 .../cassandra/tools/BulkLoadConnectionFactory.java |  1 -
 .../cassandra/utils/BloomFilterSerializer.java     |  2 --
 .../org/apache/cassandra/utils/FBUtilities.java    |  2 +-
 .../cassandra/utils/binlog/ExternalArchiver.java   |  1 -
 .../apache/cassandra/utils/concurrent/Threads.java |  5 +++
 .../apache/cassandra/utils/obs/OffHeapBitSet.java  |  1 -
 .../cassandra/distributed/impl/Coordinator.java    |  2 +-
 .../apache/cassandra/distributed/impl/Query.java   |  4 ++-
 .../streaming/StreamTransferTaskTest.java          |  1 +
 54 files changed, 140 insertions(+), 96 deletions(-)

diff --git a/src/java/org/apache/cassandra/cache/AutoSavingCache.java b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
index 03cbde2..0e022d4 100644
--- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java
+++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
@@ -27,7 +27,6 @@ import java.util.*;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.cassandra.io.util.File;
 import org.cliffc.high_scale_lib.NonBlockingHashSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java b/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
index 88e0d10..9816649 100644
--- a/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
+++ b/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
@@ -177,7 +177,7 @@ public class NamedThreadFactory implements ThreadFactory
     @Override
     public String toString()
     {
-        return id;
+        return threadGroup != null ? id + " in " + threadGroup.getName() : id;
     }
 
     public void close()
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index a9bcdaa..be03a34 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -297,7 +297,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                     }
                 }
             };
-            ScheduledExecutors.scheduledTasks.schedule(runnable, period, TimeUnit.MILLISECONDS);
+            ScheduledExecutors.scheduledTasks.scheduleSelfRecurring(runnable, period, TimeUnit.MILLISECONDS);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 8b48c5c..c2ad4bd 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -30,7 +30,6 @@ import java.io.IOException;
 import java.nio.file.FileStore;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.util.*;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.function.BiPredicate;
diff --git a/src/java/org/apache/cassandra/db/ReadResponse.java b/src/java/org/apache/cassandra/db/ReadResponse.java
index 568b1a1..52e6fd5 100644
--- a/src/java/org/apache/cassandra/db/ReadResponse.java
+++ b/src/java/org/apache/cassandra/db/ReadResponse.java
@@ -30,7 +30,6 @@ import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 6ff22df..3ce81e3 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -38,7 +38,6 @@ import com.google.common.collect.SetMultimap;
 import com.google.common.collect.Sets;
 import com.google.common.io.ByteStreams;
 
-import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java b/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java
index 2cb5eb1..b61c4d0 100644
--- a/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java
+++ b/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java
@@ -23,8 +23,6 @@ import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
-import java.nio.file.Files;
-import java.nio.file.Paths;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.cassandra.io.util.File;
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java
index c144d09..0e051cf 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.db.commitlog;
 
 import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.io.util.FileUtils;
 
 public class CommitLogSegmentManagerStandard extends AbstractCommitLogSegmentManager
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 666b037..ebb2303 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -44,7 +44,6 @@ import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.*;
-import org.apache.cassandra.utils.Closeable;
 import org.apache.cassandra.utils.concurrent.Refs;
 import org.apache.cassandra.utils.memory.HeapAllocator;
 
diff --git a/src/java/org/apache/cassandra/db/compaction/Verifier.java b/src/java/org/apache/cassandra/db/compaction/Verifier.java
index 8b7f0d6..5a73416 100644
--- a/src/java/org/apache/cassandra/db/compaction/Verifier.java
+++ b/src/java/org/apache/cassandra/db/compaction/Verifier.java
@@ -36,7 +36,6 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
 import org.apache.cassandra.io.sstable.metadata.MetadataType;
 import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
-import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataIntegrityMetadata;
 import org.apache.cassandra.io.util.DataIntegrityMetadata.FileDigestValidator;
 import org.apache.cassandra.io.util.FileInputStreamPlus;
@@ -52,16 +51,12 @@ import org.apache.cassandra.utils.IFilter;
 import org.apache.cassandra.utils.OutputHandler;
 import org.apache.cassandra.utils.UUIDGen;
 
-import java.io.BufferedInputStream;
 import java.io.Closeable;
-import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.IOError;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.util.*;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
@@ -448,10 +443,10 @@ public class Verifier implements Closeable
 
     private void deserializeBloomFilter(SSTableReader sstable) throws IOException
     {
-        Path bfPath = Paths.get(sstable.descriptor.filenameFor(Component.FILTER));
-        if (Files.exists(bfPath))
+        File bfPath = new File(sstable.descriptor.filenameFor(Component.FILTER));
+        if (bfPath.exists())
         {
-            try (FileInputStreamPlus stream = new File(bfPath).newInputStream();
+            try (FileInputStreamPlus stream = bfPath.newInputStream();
                  IFilter bf = BloomFilterSerializer.deserialize(stream, sstable.descriptor.version.hasOldBfFormat()))
             {
             }
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
index 74ebac7..2251f6a 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
@@ -23,7 +23,6 @@ import java.util.List;
 import java.util.Set;
 import java.util.UUID;
 
-import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java b/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
index 4fb3947..45653c4 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
@@ -22,7 +22,6 @@ package org.apache.cassandra.db.lifecycle;
 
 
 import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.util.*;
 import java.util.function.BiPredicate;
 import java.util.regex.Matcher;
diff --git a/src/java/org/apache/cassandra/gms/EndpointState.java b/src/java/org/apache/cassandra/gms/EndpointState.java
index b7f6bdb..2cc9c0d 100644
--- a/src/java/org/apache/cassandra/gms/EndpointState.java
+++ b/src/java/org/apache/cassandra/gms/EndpointState.java
@@ -26,7 +26,6 @@ import javax.annotation.Nullable;
 
 import com.google.common.annotations.VisibleForTesting;
 
-import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/gms/GossipDigest.java b/src/java/org/apache/cassandra/gms/GossipDigest.java
index 4115c38..53f6c5c 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigest.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigest.java
@@ -23,7 +23,6 @@ import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.InetAddressAndPort;
 
 import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer;
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAck2.java b/src/java/org/apache/cassandra/gms/GossipDigestAck2.java
index 7324763..0e4062b 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestAck2.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestAck2.java
@@ -25,7 +25,6 @@ import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.InetAddressAndPort;
 
 import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer;
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestSyn.java b/src/java/org/apache/cassandra/gms/GossipDigestSyn.java
index c2c736f..7c2ae94 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestSyn.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestSyn.java
@@ -30,7 +30,6 @@ import org.apache.cassandra.io.util.DataOutputPlus;
  * This is the first message that gets sent out as a start of the Gossip protocol in a
  * round.
  */
-import org.apache.cassandra.io.util.File;
 
 public class GossipDigestSyn
 {
diff --git a/src/java/org/apache/cassandra/gms/HeartBeatState.java b/src/java/org/apache/cassandra/gms/HeartBeatState.java
index d0a7142..cad6a48 100644
--- a/src/java/org/apache/cassandra/gms/HeartBeatState.java
+++ b/src/java/org/apache/cassandra/gms/HeartBeatState.java
@@ -29,7 +29,6 @@ import org.apache.cassandra.io.util.DataOutputPlus;
 /**
  * HeartBeat State associated with any given endpoint.
  */
-import org.apache.cassandra.io.util.File;
 
 public class HeartBeatState
 {
diff --git a/src/java/org/apache/cassandra/gms/TokenSerializer.java b/src/java/org/apache/cassandra/gms/TokenSerializer.java
index d73b077..0048e7c 100644
--- a/src/java/org/apache/cassandra/gms/TokenSerializer.java
+++ b/src/java/org/apache/cassandra/gms/TokenSerializer.java
@@ -30,9 +30,6 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 
-
-import org.apache.cassandra.io.util.File;
-
 public class TokenSerializer
 {
     private static final Logger logger = LoggerFactory.getLogger(TokenSerializer.class);
diff --git a/src/java/org/apache/cassandra/gms/VersionedValue.java b/src/java/org/apache/cassandra/gms/VersionedValue.java
index 659f61b..880cb98 100644
--- a/src/java/org/apache/cassandra/gms/VersionedValue.java
+++ b/src/java/org/apache/cassandra/gms/VersionedValue.java
@@ -24,7 +24,6 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.stream.Collectors;
 
-import org.apache.cassandra.io.util.File;
 import static java.nio.charset.StandardCharsets.ISO_8859_1;
 
 import com.google.common.annotations.VisibleForTesting;
diff --git a/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java b/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
index 463f33e..339e45e 100644
--- a/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
+++ b/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
@@ -25,7 +25,6 @@ import com.google.common.base.Preconditions;
 
 import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.io.util.*;
-import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.utils.Throwables;
 import org.apache.cassandra.utils.NativeLibrary;
 
diff --git a/src/java/org/apache/cassandra/index/sasi/TermIterator.java b/src/java/org/apache/cassandra/index/sasi/TermIterator.java
index c84b209..d65b386 100644
--- a/src/java/org/apache/cassandra/index/sasi/TermIterator.java
+++ b/src/java/org/apache/cassandra/index/sasi/TermIterator.java
@@ -20,11 +20,9 @@ package org.apache.cassandra.index.sasi;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.*;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
 import io.netty.util.concurrent.FastThreadLocal;
-import org.apache.cassandra.concurrent.ExecutorFactory;
 import org.apache.cassandra.concurrent.ImmediateExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.index.sasi.disk.Token;
@@ -33,8 +31,6 @@ import org.apache.cassandra.index.sasi.utils.RangeUnionIterator;
 import org.apache.cassandra.index.sasi.utils.RangeIterator;
 import org.apache.cassandra.io.util.FileUtils;
 
-import com.google.common.util.concurrent.MoreExecutors;
-
 import org.apache.cassandra.utils.concurrent.CountDownLatch;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java
index 9ba9f9c..a6faa04 100644
--- a/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java
+++ b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java
@@ -31,7 +31,6 @@ import org.apache.cassandra.index.sasi.sa.SuffixSA;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.*;
-import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
diff --git a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 8321345..024e4ef 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -30,7 +30,6 @@ import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.*;
-import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.CompressionParams;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
diff --git a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
index cc6ce6b..5af9c92 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
@@ -17,11 +17,8 @@
  */
 package org.apache.cassandra.io.compress;
 
-import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
-import java.nio.file.Paths;
 import java.io.DataInput;
-import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.EOFException;
 
diff --git a/src/java/org/apache/cassandra/io/sstable/Descriptor.java b/src/java/org/apache/cassandra/io/sstable/Descriptor.java
index f4e7f00..7c70b5a 100644
--- a/src/java/org/apache/cassandra/io/sstable/Descriptor.java
+++ b/src/java/org/apache/cassandra/io/sstable/Descriptor.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.IOError;
-import java.io.IOException;
 import java.util.*;
 import java.util.regex.Pattern;
 
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
index 2b32278..76e12c8 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.io.sstable;
 
 import java.io.*;
 
-import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index 804a59a..998d374 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -45,7 +45,6 @@ import com.clearspring.analytics.stream.cardinality.ICardinality;
 
 import org.apache.cassandra.cache.InstrumentingCache;
 import org.apache.cassandra.cache.KeyCacheKey;
-import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.concurrent.ScheduledExecutorPlus;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -2212,18 +2211,27 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         public static Ref<GlobalTidy> get(SSTableReader sstable)
         {
             Descriptor descriptor = sstable.descriptor;
-            Ref<GlobalTidy> refc = lookup.get(descriptor);
-            if (refc != null)
-                return refc.ref();
-            final GlobalTidy tidy = new GlobalTidy(sstable);
-            refc = new Ref<>(tidy, tidy);
-            Ref<?> ex = lookup.putIfAbsent(descriptor, refc);
-            if (ex != null)
+
+            while (true)
             {
-                refc.close();
-                throw new AssertionError();
+                Ref<GlobalTidy> ref = lookup.get(descriptor);
+                if (ref == null)
+                {
+                    final GlobalTidy tidy = new GlobalTidy(sstable);
+                    ref = new Ref<>(tidy, tidy);
+                    Ref<GlobalTidy> ex = lookup.putIfAbsent(descriptor, ref);
+                    if (ex == null)
+                        return ref;
+                    ref = ex;
+                }
+
+                Ref<GlobalTidy> newRef = ref.tryRef();
+                if (newRef != null)
+                    return newRef;
+
+                // raced with tidy
+                lookup.remove(descriptor, ref);
             }
-            return refc;
         }
     }
 
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java
index 3386c23..6ca74f0 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java
@@ -39,12 +39,10 @@ import org.apache.cassandra.utils.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.BufferedInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.file.Files;
-import java.nio.file.Paths;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
index 7929059..889547d 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
@@ -25,7 +25,7 @@ import java.util.*;
 
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
-import org.apache.cassandra.io.util.File;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
index 91889a7..e57a50e 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
@@ -28,7 +28,6 @@ import java.util.zip.CRC32;
 import com.google.common.base.Throwables;
 import com.google.common.collect.Lists;
 
-import org.apache.cassandra.io.util.File;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java
index a490ff6..e56b7b0 100644
--- a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java
+++ b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java
@@ -243,6 +243,9 @@ public class BufferedDataOutputStreamPlus extends DataOutputStreamPlus
     @Override
     public void close() throws IOException
     {
+        if (buffer == null)
+            return;
+
         doFlush(0);
         channel.close();
         FileUtils.clean(buffer);
diff --git a/src/java/org/apache/cassandra/io/util/FileHandle.java b/src/java/org/apache/cassandra/io/util/FileHandle.java
index 6d3ae7c..6bab460 100644
--- a/src/java/org/apache/cassandra/io/util/FileHandle.java
+++ b/src/java/org/apache/cassandra/io/util/FileHandle.java
@@ -148,8 +148,16 @@ public class FileHandle extends SharedCloseableImpl
     public FileDataInput createReader(long position)
     {
         RandomAccessReader reader = createReader();
-        reader.seek(position);
-        return reader;
+        try
+        {
+            reader.seek(position);
+            return reader;
+        }
+        catch (Throwable t)
+        {
+            try { reader.close(); } catch (Throwable t2) { t.addSuppressed(t2); }
+            throw t;
+        }
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/io/util/PathUtils.java b/src/java/org/apache/cassandra/io/util/PathUtils.java
index 26f9dcc..690222f 100644
--- a/src/java/org/apache/cassandra/io/util/PathUtils.java
+++ b/src/java/org/apache/cassandra/io/util/PathUtils.java
@@ -41,8 +41,6 @@ import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.NoSpamLogger;
 
 import static java.nio.file.StandardOpenOption.*;
-import static java.nio.file.StandardOpenOption.CREATE;
-import static java.nio.file.StandardOpenOption.WRITE;
 import static java.util.Collections.unmodifiableSet;
 import static org.apache.cassandra.utils.Throwables.merge;
 
diff --git a/src/java/org/apache/cassandra/net/Verb.java b/src/java/org/apache/cassandra/net/Verb.java
index 9e3f5f6..bcd3070 100644
--- a/src/java/org/apache/cassandra/net/Verb.java
+++ b/src/java/org/apache/cassandra/net/Verb.java
@@ -225,7 +225,7 @@ public enum Verb
     private final Supplier<? extends IVersionedAsymmetricSerializer<?, ?>> serializer;
     private final Supplier<? extends IVerbHandler<?>> handler;
 
-    final Verb responseVerb;
+    public final Verb responseVerb;
 
     private final ToLongFunction<TimeUnit> expiration;
 
diff --git a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java
index 249d1a4..3d21702 100644
--- a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java
+++ b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java
@@ -28,7 +28,6 @@ import java.util.Optional;
 
 import com.google.common.collect.Lists;
 
-import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.RepairResult;
 import org.apache.cassandra.repair.RepairSessionResult;
 import org.apache.cassandra.repair.SyncStat;
diff --git a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java
index a539686..9f17578 100644
--- a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java
@@ -310,7 +310,10 @@ public final class SystemDistributedKeyspace
                                       keyspaceName,
                                       cfname,
                                       id.toString());
-        processSilent(fmtQry, t.getMessage(), sw.toString());
+        String message = t.getMessage();
+        if (message == null)
+            message = t.getClass().getName();
+        processSilent(fmtQry, message, sw.toString());
     }
 
     public static void startViewBuild(String keyspace, String view, UUID hostId)
diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java b/src/java/org/apache/cassandra/service/StartupChecks.java
index 5cb938b..b8fc082 100644
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -54,10 +54,8 @@ import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.NativeLibrary;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JavaUtils;
-import org.apache.cassandra.utils.NoSpamLogger;
 import org.apache.cassandra.utils.SigarLibrary;
 
-import static java.lang.String.format;
 import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_PORT;
 import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VERSION;
 import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VM_NAME;
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index dc3b878..44d757b 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -140,7 +140,6 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static java.util.stream.Collectors.toList;
 import static java.util.stream.Collectors.toMap;
-import static org.apache.cassandra.concurrent.FutureTask.callable;
 import static org.apache.cassandra.config.CassandraRelevantProperties.BOOTSTRAP_SCHEMA_DELAY_MS;
 import static org.apache.cassandra.config.CassandraRelevantProperties.BOOTSTRAP_SKIP_SCHEMA_CHECK;
 import static org.apache.cassandra.config.CassandraRelevantProperties.REPLACEMENT_ALLOW_EMPTY;
diff --git a/src/java/org/apache/cassandra/service/paxos/Commit.java b/src/java/org/apache/cassandra/service/paxos/Commit.java
index 05fa595..134e312 100644
--- a/src/java/org/apache/cassandra/service/paxos/Commit.java
+++ b/src/java/org/apache/cassandra/service/paxos/Commit.java
@@ -24,6 +24,8 @@ package org.apache.cassandra.service.paxos;
 import java.io.IOException;
 import java.util.UUID;
 
+import javax.annotation.Nullable;
+
 import com.google.common.base.Objects;
 
 import org.apache.cassandra.schema.TableMetadata;
@@ -112,6 +114,38 @@ public class Commit
         return String.format("Commit(%s, %s)", ballot, update);
     }
 
+    /**
+     * @return testIfAfter.isAfter(testIfBefore), with non-null > null
+     */
+    public static boolean isAfter(@Nullable Commit testIsAfter, @Nullable Commit testIsBefore)
+    {
+        return testIsAfter != null && testIsAfter.isAfter(testIsBefore);
+    }
+
+    /**
+     * @return testIfAfter.isAfter(testIfBefore), with non-null > null
+     */
+    public static boolean isAfter(@Nullable UUID testIsAfter, @Nullable Commit testIsBefore)
+    {
+        return testIsAfter != null && (testIsBefore == null || testIsAfter.timestamp() > testIsBefore.ballot.timestamp());
+    }
+
+    /**
+     * @return testIfAfter.isAfter(testIfBefore), with non-null > null
+     */
+    public static boolean isAfter(@Nullable Commit testIsAfter, @Nullable UUID testIsBefore)
+    {
+        return testIsAfter != null && (testIsBefore == null || testIsAfter.ballot.timestamp() > testIsBefore.timestamp());
+    }
+
+    /**
+     * @return testIfAfter.isAfter(testIfBefore), with non-null > null
+     */
+    public static boolean isAfter(@Nullable UUID testIsAfter, @Nullable UUID testIsBefore)
+    {
+        return testIsAfter != null && (testIsBefore == null || testIsAfter.timestamp() > testIsBefore.timestamp());
+    }
+
     public static class CommitSerializer implements IVersionedSerializer<Commit>
     {
         public void serialize(Commit commit, DataOutputPlus out, int version) throws IOException
diff --git a/src/java/org/apache/cassandra/service/paxos/PaxosState.java b/src/java/org/apache/cassandra/service/paxos/PaxosState.java
index f15dfdb..4d57ef4 100644
--- a/src/java/org/apache/cassandra/service/paxos/PaxosState.java
+++ b/src/java/org/apache/cassandra/service/paxos/PaxosState.java
@@ -53,9 +53,9 @@ public class PaxosState
         }
     }
 
-    private final Commit promised;
-    private final Commit accepted;
-    private final Commit mostRecentCommit;
+    public final Commit promised;
+    public final Commit accepted;
+    public final Commit mostRecentCommit;
 
     public PaxosState(DecoratedKey key, TableMetadata metadata)
     {
diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotManifest.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotManifest.java
index d8d900a..e1bd4df 100644
--- a/src/java/org/apache/cassandra/service/snapshot/SnapshotManifest.java
+++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotManifest.java
@@ -30,6 +30,10 @@ import org.apache.cassandra.config.Duration;
 import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule;
 import com.fasterxml.jackson.databind.DeserializationFeature;
 import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
+import org.apache.cassandra.io.util.FileOutputStreamPlus;
+
+import static org.apache.cassandra.io.util.File.WriteMode.OVERWRITE;
 
 // Only serialize fields
 @JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.ANY,
@@ -84,12 +88,18 @@ public class SnapshotManifest
 
     public void serializeToJsonFile(File outputFile) throws IOException
     {
-        mapper.writeValue(outputFile.toJavaIOFile(), this);
+        try (FileOutputStreamPlus out = outputFile.newOutputStream(OVERWRITE))
+        {
+            mapper.writeValue((OutputStream) out, this);
+        }
     }
 
     public static SnapshotManifest deserializeFromJsonFile(File file) throws IOException
     {
-        return mapper.readValue(file.toJavaIOFile(), SnapshotManifest.class);
+        try (FileInputStreamPlus in = file.newInputStream())
+        {
+            return mapper.readValue((InputStream) in, SnapshotManifest.class);
+        }
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
index 6bcd074..f7a0b63 100644
--- a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
+++ b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
@@ -21,6 +21,8 @@ import java.util.Collection;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentLinkedQueue;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.cassandra.utils.concurrent.AsyncFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -71,7 +73,8 @@ public final class StreamResultFuture extends AsyncFuture<StreamState>
             trySuccess(getCurrentState());
     }
 
-    private StreamResultFuture(UUID planId, StreamOperation streamOperation, UUID pendingRepair, PreviewKind previewKind)
+    @VisibleForTesting
+    public StreamResultFuture(UUID planId, StreamOperation streamOperation, UUID pendingRepair, PreviewKind previewKind)
     {
         this(planId, streamOperation, new StreamCoordinator(streamOperation, 0, streamingFactory(), true, false, pendingRepair, previewKind));
     }
diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java
index 4e1f3e1..bcf19d5 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -890,6 +890,15 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     }
 
     /**
+     * Call back on receiving {@code StreamMessage.Type.SESSION_FAILED} message.
+     */
+    public synchronized void sessionTimeout()
+    {
+        logger.error("[Stream #{}] timeout with {}.", planId(), peer.toString());
+        closeSession(State.FAILED);
+    }
+
+    /**
      * @return Current snapshot of this session info.
      */
     public SessionInfo getSessionInfo()
diff --git a/src/java/org/apache/cassandra/streaming/StreamTransferTask.java b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
index 70ad7d8..980193b 100644
--- a/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -33,6 +32,7 @@ import com.google.common.base.Throwables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.concurrent.ScheduledExecutorPlus;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.streaming.messages.OutgoingStreamMessage;
 
@@ -46,7 +46,7 @@ import static org.apache.cassandra.utils.ExecutorUtils.shutdown;
 public class StreamTransferTask extends StreamTask
 {
     private static final Logger logger = LoggerFactory.getLogger(StreamTransferTask.class);
-    private static final ScheduledExecutorService timeoutExecutor = executorFactory().scheduled("StreamingTransferTaskTimeouts");
+    private static final ScheduledExecutorPlus timeoutExecutor = executorFactory().scheduled("StreamingTransferTaskTimeouts");
 
     private final AtomicInteger sequenceNumber = new AtomicInteger(0);
     private boolean aborted = false;
@@ -100,6 +100,26 @@ public class StreamTransferTask extends StreamTask
             session.taskCompleted(this);
     }
 
+    /**
+     * Received ACK for stream at {@code sequenceNumber}.
+     *
+     * @param sequenceNumber sequence number of stream
+     */
+    public void timeout(int sequenceNumber)
+    {
+        synchronized (this)
+        {
+            timeoutTasks.remove(sequenceNumber);
+            OutgoingStreamMessage stream = streams.remove(sequenceNumber);
+            if (stream == null) return;
+            stream.complete();
+
+            logger.debug("timeout sequenceNumber {}, remaining files {}", sequenceNumber, streams.keySet());
+        }
+
+        session.sessionTimeout();
+    }
+
     public synchronized void abort()
     {
         if (aborted)
@@ -169,19 +189,7 @@ public class StreamTransferTask extends StreamTask
         if (!streams.containsKey(sequenceNumber))
             return null;
 
-        ScheduledFuture future = timeoutExecutor.schedule(new Runnable()
-        {
-            public void run()
-            {
-                synchronized (StreamTransferTask.this)
-                {
-                    // remove so we don't cancel ourselves
-                    timeoutTasks.remove(sequenceNumber);
-                    StreamTransferTask.this.complete(sequenceNumber);
-                }
-            }
-        }, time, unit);
-
+        ScheduledFuture future = timeoutExecutor.scheduleTimeoutWithDelay(() -> StreamTransferTask.this.timeout(sequenceNumber), time, unit);
         ScheduledFuture prev = timeoutTasks.put(sequenceNumber, future);
         assert prev == null;
         return future;
diff --git a/src/java/org/apache/cassandra/streaming/async/StreamingMultiplexedChannel.java b/src/java/org/apache/cassandra/streaming/async/StreamingMultiplexedChannel.java
index 0a7a470..4f3a443 100644
--- a/src/java/org/apache/cassandra/streaming/async/StreamingMultiplexedChannel.java
+++ b/src/java/org/apache/cassandra/streaming/async/StreamingMultiplexedChannel.java
@@ -59,6 +59,7 @@ import static org.apache.cassandra.config.Config.PROPERTY_PREFIX;
 import static org.apache.cassandra.streaming.StreamSession.createLogTag;
 import static org.apache.cassandra.streaming.messages.StreamMessage.serialize;
 import static org.apache.cassandra.streaming.messages.StreamMessage.serializedSize;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
 import static org.apache.cassandra.utils.FBUtilities.getAvailableProcessors;
 import static org.apache.cassandra.utils.JVMStabilityInspector.inspectThrowable;
 import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue;
diff --git a/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java b/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
index 079c08b..cd77d4f 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 
 import org.apache.cassandra.config.EncryptionOptions;
-import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.OutboundConnectionSettings;
 import org.apache.cassandra.streaming.StreamingChannel;
 import org.apache.cassandra.streaming.async.NettyStreamingConnectionFactory;
diff --git a/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java b/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
index 8506ce5..3df4314 100644
--- a/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
+++ b/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
@@ -18,12 +18,10 @@
 package org.apache.cassandra.utils;
 
 import java.io.DataInput;
-import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 
 import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.obs.IBitSet;
 import org.apache.cassandra.utils.obs.OffHeapBitSet;
diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java
index 912d907..58e66ec 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -44,7 +44,7 @@ import javax.annotation.Nullable;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
-import com.google.common.base.Strings;
+
 import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.utils.concurrent.*;
 import org.apache.commons.lang3.StringUtils;
diff --git a/src/java/org/apache/cassandra/utils/binlog/ExternalArchiver.java b/src/java/org/apache/cassandra/utils/binlog/ExternalArchiver.java
index 86b6510..b3ce484 100644
--- a/src/java/org/apache/cassandra/utils/binlog/ExternalArchiver.java
+++ b/src/java/org/apache/cassandra/utils/binlog/ExternalArchiver.java
@@ -36,7 +36,6 @@ import org.slf4j.Logger;
 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;
 
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Threads.java b/src/java/org/apache/cassandra/utils/concurrent/Threads.java
index 439a77f..f02519a 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Threads.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Threads.java
@@ -132,4 +132,9 @@ public class Threads
         return Stream.of(st).collect(new StackTraceCombiner(printBriefPackages, prefix, delimiter, suffix));
     }
 
+    public static String prettyPrint(Stream<StackTraceElement> st, boolean printBriefPackages, String prefix, String delimiter, String suffix)
+    {
+        return st.collect(new StackTraceCombiner(printBriefPackages, prefix, delimiter, suffix));
+    }
+
 }
diff --git a/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java b/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
index 8b0550f..ae89594 100644
--- a/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
+++ b/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.utils.obs;
 
 import java.io.*;
-import java.io.IOException;
 
 import com.google.common.annotations.VisibleForTesting;
 
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java b/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java
index b409c88..2a02d26 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java
@@ -111,7 +111,7 @@ public class Coordinator implements ICoordinator
                                                        null,
                                                        ProtocolVersion.CURRENT,
                                                        null),
-                                   System.nanoTime());
+                                   nanoTime());
             // Collect warnings reported during the query.
             CoordinatorWarnings.done();
             if (res != null)
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Query.java b/test/distributed/org/apache/cassandra/distributed/impl/Query.java
index 7950d75..823113f 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/Query.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/Query.java
@@ -35,6 +35,8 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
 public class Query implements IIsolatedExecutor.SerializableCallable<Object[][]>
 {
     private static final long serialVersionUID = 1L;
@@ -81,7 +83,7 @@ public class Query implements IIsolatedExecutor.SerializableCallable<Object[][]>
                                                                  null,
                                                                  timestamp,
                                                                  FBUtilities.nowInSeconds()),
-                                             System.nanoTime());
+                                             nanoTime());
 
         // Collect warnings reported during the query.
         if (res != null)
diff --git a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
index f061e51..8e026fd 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
@@ -95,6 +95,7 @@ public class StreamTransferTaskTest
     {
         InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
         StreamSession session = new StreamSession(StreamOperation.BOOTSTRAP, peer, FACTORY, null, current_version, false, 0, UUID.randomUUID(), PreviewKind.ALL);
+        session.init(new StreamResultFuture(UUID.randomUUID(), StreamOperation.OTHER, UUID.randomUUID(), PreviewKind.NONE));
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD);
 
         // create two sstables

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