You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2019/06/12 15:27:25 UTC

[cassandra] branch trunk updated (532c033 -> 0d48503)

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

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


 discard 532c033  Introduce a proxy test handler, extra unit tests for connection closure and message expirations
    omit 32d7e01  Rewrite the internode messaging subsystem
     new 310a48e  Rewrite the internode messaging subsystem
     new 0d48503  Introduce a proxy test handler, extra unit tests for connection closure and message expirations

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

 * -- * -- B -- O -- O -- O   (532c033)
            \
             N -- N -- N   refs/heads/trunk (0d48503)

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

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

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


Summary of changes:


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


[cassandra] 01/02: Rewrite the internode messaging subsystem

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

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

commit 310a48efcfdc0fb77fbd5fd6e734bf17667f617d
Author: Benedict Elliott Smith <be...@apache.org>
AuthorDate: Wed Jun 12 16:26:54 2019 +0100

    Rewrite the internode messaging subsystem
    
    patch by Aleksey Yeschenko and Benedict Elliott Smith; reviewed by Alex
    Petrov, Aleksey Yeschenko, and Benedict Elliott Smith for CASSANDRA-15066
    
    Co-authored-by: Aleksey Yeschenko <al...@apache.org>
    Co-authored-by: Benedict Elliott Smith <be...@apache.org>
---
 build.xml                                          |   15 +
 conf/cassandra.yaml                                |   26 +
 .../org/apache/cassandra/auth/IAuthenticator.java  |    2 +-
 .../cassandra/batchlog/BatchRemoveVerbHandler.java |    6 +-
 .../cassandra/batchlog/BatchStoreVerbHandler.java  |    9 +-
 .../apache/cassandra/batchlog/BatchlogManager.java |   26 +-
 .../cassandra/concurrent/ImmediateExecutor.java    |   56 +
 .../cassandra/concurrent/InfiniteLoopExecutor.java |    2 +-
 .../cassandra/concurrent/NamedThreadFactory.java   |    2 +
 .../cassandra/concurrent/ScheduledExecutors.java   |   15 +-
 .../cassandra/concurrent/SharedExecutorPool.java   |    9 +-
 .../org/apache/cassandra/concurrent/Stage.java     |   20 +-
 .../apache/cassandra/concurrent/StageManager.java  |   25 +-
 src/java/org/apache/cassandra/config/Config.java   |   26 +-
 .../cassandra/config/DatabaseDescriptor.java       |  170 +-
 .../apache/cassandra/config/EncryptionOptions.java |  312 +++-
 .../org/apache/cassandra/db/ColumnFamilyStore.java |   34 +-
 src/java/org/apache/cassandra/db/Columns.java      |    3 +-
 .../org/apache/cassandra/db/CounterMutation.java   |   18 +-
 .../cassandra/db/CounterMutationVerbHandler.java   |   18 +-
 src/java/org/apache/cassandra/db/IMutation.java    |    3 +-
 src/java/org/apache/cassandra/db/Keyspace.java     |    7 +-
 src/java/org/apache/cassandra/db/Mutation.java     |   37 +-
 .../apache/cassandra/db/MutationVerbHandler.java   |   52 +-
 .../cassandra/db/PartitionRangeReadCommand.java    |   20 +-
 src/java/org/apache/cassandra/db/ReadCommand.java  |   33 +-
 .../cassandra/db/ReadCommandVerbHandler.java       |   38 +-
 .../cassandra/db/ReadExecutionController.java      |   10 +-
 .../apache/cassandra/db/ReadRepairVerbHandler.java |    8 +-
 .../apache/cassandra/db/SerializationHeader.java   |    5 +-
 .../cassandra/db/SinglePartitionReadCommand.java   |   24 +-
 .../org/apache/cassandra/db/SnapshotCommand.java   |    9 +-
 .../org/apache/cassandra/db/SystemKeyspace.java    |    7 +-
 .../org/apache/cassandra/db/TruncateRequest.java   |   68 +
 .../org/apache/cassandra/db/TruncateResponse.java  |    7 -
 .../apache/cassandra/db/TruncateVerbHandler.java   |   28 +-
 src/java/org/apache/cassandra/db/Truncation.java   |   74 -
 .../db/commitlog/AbstractCommitLogService.java     |   12 +-
 .../cassandra/db/context/CounterContext.java       |    4 +-
 .../cassandra/db/filter/ColumnSubselection.java    |    3 +-
 .../apache/cassandra/db/marshal/CompositeType.java |    2 +-
 .../cassandra/db/monitoring/ApproximateTime.java   |   61 -
 .../cassandra/db/monitoring/Monitorable.java       |    6 +-
 .../cassandra/db/monitoring/MonitorableImpl.java   |   46 +-
 .../cassandra/db/monitoring/MonitoringTask.java    |   76 +-
 .../streaming/CassandraCompressedStreamReader.java |    3 +-
 .../streaming/CassandraCompressedStreamWriter.java |   40 +-
 .../CassandraEntireSSTableStreamWriter.java        |   10 +-
 .../db/streaming/CassandraOutgoingFile.java        |    7 +-
 .../db/streaming/CassandraStreamReader.java        |    9 +-
 .../db/streaming/CassandraStreamWriter.java        |   75 +-
 .../db/virtual/InternodeInboundTable.java          |  135 ++
 .../db/virtual/InternodeOutboundTable.java         |  140 ++
 .../apache/cassandra/db/virtual/SettingsTable.java |    2 +-
 .../cassandra/db/virtual/SystemViewsKeyspace.java  |    4 +-
 .../cassandra/db/virtual/VirtualMutation.java      |    5 +-
 .../org/apache/cassandra/dht/BootStrapper.java     |   26 -
 .../org/apache/cassandra/dht/IPartitioner.java     |   21 +
 ...ption.java => IncompatibleSchemaException.java} |    9 +-
 .../exceptions/RequestFailureException.java        |   13 +-
 .../cassandra/exceptions/RequestFailureReason.java |   98 +-
 .../UnknownColumnException.java}                   |    9 +-
 .../exceptions/UnknownTableException.java          |    4 +-
 src/java/org/apache/cassandra/gms/EchoMessage.java |   56 -
 .../org/apache/cassandra/gms/FailureDetector.java  |   11 +-
 .../org/apache/cassandra/gms/GossipDigest.java     |    9 +-
 .../org/apache/cassandra/gms/GossipDigestAck.java  |    9 +-
 .../org/apache/cassandra/gms/GossipDigestAck2.java |    9 +-
 .../cassandra/gms/GossipDigestAck2VerbHandler.java |   13 +-
 .../cassandra/gms/GossipDigestAckVerbHandler.java  |   22 +-
 .../cassandra/gms/GossipDigestSynVerbHandler.java  |   28 +-
 .../cassandra/gms/GossipShutdownVerbHandler.java   |   10 +-
 .../GossipVerbHandler.java}                        |   11 +-
 src/java/org/apache/cassandra/gms/Gossiper.java    |   67 +-
 .../apache/cassandra/hints/EncodedHintMessage.java |   94 -
 .../org/apache/cassandra/hints/HintMessage.java    |  114 +-
 .../org/apache/cassandra/hints/HintResponse.java   |   58 -
 .../apache/cassandra/hints/HintVerbHandler.java    |   20 +-
 .../apache/cassandra/hints/HintsDispatcher.java    |   52 +-
 .../SerializableHintMessage.java}                  |    6 +-
 .../cassandra/index/SecondaryIndexManager.java     |   13 +-
 .../org/apache/cassandra/index/sasi/SASIIndex.java |    5 +-
 .../cassandra/io/DummyByteVersionedSerializer.java |   55 -
 ...er.java => IVersionedAsymmetricSerializer.java} |   10 +-
 .../apache/cassandra/io/IVersionedSerializer.java  |   33 +-
 .../cassandra/io/sstable/format/SSTableReader.java |   53 +-
 .../sstable/format/big/BigTableZeroCopyWriter.java |   21 +-
 .../io/util/BufferedDataOutputStreamPlus.java      |  100 +-
 .../apache/cassandra/io/util/DataOutputPlus.java   |   13 +-
 .../cassandra/io/util/DataOutputStreamPlus.java    |    2 +-
 .../io/util/FastByteArrayInputStream.java          |  249 ---
 src/java/org/apache/cassandra/io/util/Memory.java  |    2 +-
 .../apache/cassandra/io/util/SequentialWriter.java |   19 +
 .../io/util/UnbufferedDataOutputStreamPlus.java    |   11 -
 .../cassandra/locator/DynamicEndpointSnitch.java   |    9 +-
 .../org/apache/cassandra/locator/Endpoints.java    |   20 +
 .../cassandra/locator/InetAddressAndPort.java      |  135 +-
 .../locator/ReconnectableSnitchHelper.java         |   12 +-
 .../cassandra/metrics/ConnectionMetrics.java       |  154 --
 .../cassandra/metrics/DroppedMessageMetrics.java   |    3 +-
 .../apache/cassandra/metrics/FrequencySampler.java |   18 +-
 .../cassandra/metrics/InternodeInboundMetrics.java |   98 +
 .../metrics/InternodeOutboundMetrics.java          |  205 +++
 .../org/apache/cassandra/metrics/MaxSampler.java   |   16 +-
 .../apache/cassandra/metrics/MessagingMetrics.java |  204 ++-
 src/java/org/apache/cassandra/metrics/Sampler.java |    8 +-
 .../org/apache/cassandra/metrics/TableMetrics.java |    2 +-
 .../org/apache/cassandra/net/AcceptVersions.java   |   28 +-
 .../cassandra/net/AsyncChannelOutputPlus.java      |  268 +++
 .../apache/cassandra/net/AsyncChannelPromise.java  |  164 ++
 .../cassandra/net/AsyncMessageOutputPlus.java      |  131 ++
 .../org/apache/cassandra/net/AsyncOneResponse.java |   39 +-
 .../org/apache/cassandra/net/AsyncPromise.java     |  488 +++++
 .../cassandra/net/AsyncStreamingInputPlus.java     |  251 +++
 .../cassandra/net/AsyncStreamingOutputPlus.java    |  220 +++
 .../apache/cassandra/net/BackPressureState.java    |    2 +-
 .../apache/cassandra/net/BufferPoolAllocator.java  |  116 ++
 .../org/apache/cassandra/net/CallbackInfo.java     |   70 -
 .../org/apache/cassandra/net/ChunkedInputPlus.java |   97 +
 .../net/CompactEndpointSerializationHelper.java    |  128 --
 ...piredException.java => ConnectionCategory.java} |   20 +-
 .../org/apache/cassandra/net/ConnectionType.java   |   69 +
 src/java/org/apache/cassandra/net/Crc.java         |  136 ++
 .../cassandra/net/EndpointMessagingVersions.java   |   94 +
 .../apache/cassandra/net/ForwardToSerializer.java  |   86 -
 .../org/apache/cassandra/net/ForwardingInfo.java   |  139 ++
 .../org/apache/cassandra/net/FrameDecoder.java     |  400 +++++
 .../org/apache/cassandra/net/FrameDecoderCrc.java  |  158 ++
 .../org/apache/cassandra/net/FrameDecoderLZ4.java  |  165 ++
 .../apache/cassandra/net/FrameDecoderLegacy.java   |  184 ++
 .../cassandra/net/FrameDecoderLegacyLZ4.java       |  377 ++++
 .../cassandra/net/FrameDecoderUnprotected.java     |  100 ++
 .../cassandra/net/FrameDecoderWith8bHeader.java    |  144 ++
 .../org/apache/cassandra/net/FrameEncoder.java     |  140 ++
 .../org/apache/cassandra/net/FrameEncoderCrc.java  |   98 +
 .../org/apache/cassandra/net/FrameEncoderLZ4.java  |  118 ++
 .../{IMessageSink.java => FrameEncoderLegacy.java} |   29 +-
 .../cassandra/net/FrameEncoderLegacyLZ4.java       |  137 ++
 .../cassandra/net/FrameEncoderUnprotected.java     |   66 +
 .../org/apache/cassandra/net/FutureCombiner.java   |   85 +
 .../org/apache/cassandra/net/FutureDelegate.java   |  145 ++
 .../net/{IAsyncCallback.java => FutureResult.java} |   32 +-
 ...Handler.java => GlobalBufferPoolAllocator.java} |   26 +-
 .../apache/cassandra/net/HandshakeProtocol.java    |  414 +++++
 .../org/apache/cassandra/net/IVerbHandler.java     |    4 +-
 .../cassandra/net/InboundConnectionInitiator.java  |  495 +++++
 .../cassandra/net/InboundConnectionSettings.java   |  213 +++
 .../org/apache/cassandra/net/InboundCounters.java  |  130 ++
 .../cassandra/net/InboundMessageCallbacks.java     |   99 +
 .../cassandra/net/InboundMessageHandler.java       | 1194 ++++++++++++
 .../cassandra/net/InboundMessageHandlers.java      |  447 +++++
 src/java/org/apache/cassandra/net/InboundSink.java |  161 ++
 .../org/apache/cassandra/net/InboundSockets.java   |  235 +++
 ...ge.java => InvalidSerializedSizeException.java} |   39 +-
 .../LatencyConsumer.java}                          |    8 +-
 .../apache/cassandra/net/LatencySubscribers.java   |   75 +
 .../{db/WriteResponse.java => net/LegacyFlag.java} |   45 +-
 .../apache/cassandra/net/LegacyLZ4Constants.java   |   54 +
 .../cassandra/net/LocalBufferPoolAllocator.java    |   64 +
 .../net/ManyToOneConcurrentLinkedQueue.java        |  350 ++++
 src/java/org/apache/cassandra/net/Message.java     | 1338 ++++++++++++++
 .../apache/cassandra/net/MessageDeliveryTask.java  |  134 --
 src/java/org/apache/cassandra/net/MessageFlag.java |   83 +
 src/java/org/apache/cassandra/net/MessageIn.java   |  234 ---
 src/java/org/apache/cassandra/net/MessageOut.java  |  406 -----
 .../org/apache/cassandra/net/MessagingService.java | 1897 ++++----------------
 .../cassandra/net/MessagingServiceMBeanImpl.java   |  304 ++++
 .../net/{PongMessage.java => NoPayload.java}       |   33 +-
 ...llbackWithFailure.java => NoSizeEstimator.java} |   17 +-
 .../apache/cassandra/net/OutboundConnection.java   | 1729 ++++++++++++++++++
 .../cassandra/net/OutboundConnectionInitiator.java |  469 +++++
 .../cassandra/net/OutboundConnectionSettings.java  |  517 ++++++
 .../apache/cassandra/net/OutboundConnections.java  |  323 ++++
 .../cassandra/net/OutboundDebugCallbacks.java      |   40 +
 ...ontainer.java => OutboundMessageCallbacks.java} |   31 +-
 .../apache/cassandra/net/OutboundMessageQueue.java |  484 +++++
 .../org/apache/cassandra/net/OutboundSink.java     |  108 ++
 src/java/org/apache/cassandra/net/ParamType.java   |  116 ++
 .../org/apache/cassandra/net/ParameterType.java    |   70 -
 src/java/org/apache/cassandra/net/PingMessage.java |   82 -
 src/java/org/apache/cassandra/net/PingRequest.java |   72 +
 .../org/apache/cassandra/net/PingVerbHandler.java  |   12 +-
 .../apache/cassandra/net/PrunableArrayQueue.java   |  172 ++
 .../cassandra/net/RateBasedBackPressure.java       |    4 +-
 .../cassandra/net/RateBasedBackPressureState.java  |    2 +-
 .../{IAsyncCallback.java => RequestCallback.java}  |   31 +-
 .../org/apache/cassandra/net/RequestCallbacks.java |  382 ++++
 .../org/apache/cassandra/net/ResourceLimits.java   |  201 +++
 .../apache/cassandra/net/ResponseVerbHandler.java  |   37 +-
 .../org/apache/cassandra/net/ShareableBytes.java   |  174 ++
 .../cassandra/net/SharedDefaultFileRegion.java     |   93 +
 .../org/apache/cassandra/net/SocketFactory.java    |  327 ++++
 .../net/StartupClusterConnectivityChecker.java     |   44 +-
 src/java/org/apache/cassandra/net/Verb.java        |  331 ++++
 .../apache/cassandra/net/WriteCallbackInfo.java    |   80 -
 .../cassandra/net/async/BaseMessageInHandler.java  |  197 --
 .../cassandra/net/async/ByteBufDataInputPlus.java  |   51 -
 .../cassandra/net/async/ByteBufDataOutputPlus.java |  139 --
 .../net/async/ByteBufDataOutputStreamPlus.java     |  243 ---
 .../apache/cassandra/net/async/ChannelWriter.java  |  418 -----
 .../cassandra/net/async/HandshakeProtocol.java     |  314 ----
 .../net/async/InboundHandshakeHandler.java         |  321 ----
 .../cassandra/net/async/MessageInHandler.java      |  211 ---
 .../cassandra/net/async/MessageInHandlerPre40.java |  269 ---
 .../cassandra/net/async/MessageOutHandler.java     |  324 ----
 .../apache/cassandra/net/async/MessageResult.java  |   51 -
 .../apache/cassandra/net/async/NettyFactory.java   |  418 -----
 .../net/async/NonClosingDefaultFileRegion.java     |   51 -
 .../cassandra/net/async/OptionalSslHandler.java    |   67 -
 .../net/async/OutboundConnectionIdentifier.java    |  194 --
 .../net/async/OutboundConnectionParams.java        |  230 ---
 .../net/async/OutboundHandshakeHandler.java        |  262 ---
 .../net/async/OutboundMessagingConnection.java     |  747 --------
 .../cassandra/net/async/OutboundMessagingPool.java |  177 --
 .../apache/cassandra/net/async/QueuedMessage.java  |   75 -
 .../net/async/RebufferingByteBufDataInputPlus.java |  291 ---
 .../cassandra/repair/AsymmetricRemoteSyncTask.java |    7 +-
 .../org/apache/cassandra/repair/RepairJobDesc.java |    5 +-
 .../cassandra/repair/RepairMessageVerbHandler.java |   51 +-
 .../org/apache/cassandra/repair/SnapshotTask.java  |   27 +-
 .../cassandra/repair/StreamingRepairTask.java      |   11 +-
 .../cassandra/repair/SymmetricRemoteSyncTask.java  |    7 +-
 .../org/apache/cassandra/repair/SyncNodePair.java  |   15 +-
 .../apache/cassandra/repair/ValidationTask.java    |    5 +-
 .../org/apache/cassandra/repair/Validator.java     |    8 +-
 .../repair/consistent/CoordinatorSession.java      |    7 +-
 .../cassandra/repair/consistent/LocalSessions.java |    9 +-
 .../repair/messages/AsymmetricSyncRequest.java     |   28 +-
 .../cassandra/repair/messages/FinalizePromise.java |    9 +-
 .../repair/messages/PrepareConsistentRequest.java  |   15 +-
 .../repair/messages/PrepareConsistentResponse.java |    9 +-
 .../cassandra/repair/messages/PrepareMessage.java  |    5 +-
 .../cassandra/repair/messages/RepairMessage.java   |    9 +-
 .../cassandra/repair/messages/SyncRequest.java     |   23 +-
 .../apache/cassandra/schema/CompressionParams.java |    7 +-
 .../apache/cassandra/schema/MigrationManager.java  |   26 +-
 .../org/apache/cassandra/schema/MigrationTask.java |   39 +-
 .../cassandra/schema/SchemaMigrationEvent.java     |    4 +-
 .../cassandra/schema/SchemaPullVerbHandler.java    |   23 +-
 .../cassandra/schema/SchemaPushVerbHandler.java    |   10 +-
 .../cassandra/schema/SchemaVersionVerbHandler.java |   22 +-
 .../org/apache/cassandra/security/SSLFactory.java  |    4 +-
 .../serializers/CollectionSerializer.java          |    2 +-
 .../service/AbstractWriteResponseHandler.java      |   39 +-
 .../cassandra/service/ActiveRepairService.java     |   31 +-
 .../cassandra/service/BatchlogResponseHandler.java |   14 +-
 .../apache/cassandra/service/CassandraDaemon.java  |    8 +-
 .../DatacenterSyncWriteResponseHandler.java        |   12 +-
 .../service/DatacenterWriteResponseHandler.java    |    8 +-
 .../apache/cassandra/service/EchoVerbHandler.java  |   21 +-
 .../service/PendingRangeCalculatorService.java     |    6 +-
 .../cassandra/service/SnapshotVerbHandler.java     |   14 +-
 .../org/apache/cassandra/service/StorageProxy.java |  307 ++--
 .../cassandra/service/StorageProxyMBean.java       |    2 +
 .../apache/cassandra/service/StorageService.java   |  132 +-
 .../cassandra/service/TruncateResponseHandler.java |   20 +-
 .../cassandra/service/WriteResponseHandler.java    |    9 +-
 .../service/paxos/AbstractPaxosCallback.java       |   19 +-
 .../cassandra/service/paxos/CommitVerbHandler.java |   11 +-
 .../cassandra/service/paxos/PrepareCallback.java   |   10 +-
 .../service/paxos/PrepareVerbHandler.java          |   13 +-
 .../cassandra/service/paxos/ProposeCallback.java   |    8 +-
 .../service/paxos/ProposeVerbHandler.java          |   14 +-
 .../service/reads/AbstractReadExecutor.java        |   23 +-
 .../cassandra/service/reads/DataResolver.java      |   13 +-
 .../cassandra/service/reads/DigestResolver.java    |   32 +-
 .../cassandra/service/reads/ReadCallback.java      |   37 +-
 .../cassandra/service/reads/ResponseResolver.java  |   10 +-
 .../reads/ShortReadPartitionsProtection.java       |    2 +-
 .../service/reads/repair/AbstractReadRepair.java   |   18 +-
 .../reads/repair/BlockingPartitionRepair.java      |   29 +-
 .../service/reads/repair/BlockingReadRepair.java   |    5 +-
 .../service/reads/repair/BlockingReadRepairs.java  |    2 +-
 .../streaming/DefaultConnectionFactory.java        |   93 +-
 ...andler.java => ReplicationDoneVerbHandler.java} |   19 +-
 .../apache/cassandra/streaming/SessionSummary.java |   16 +-
 .../streaming/StreamConnectionFactory.java         |    4 +-
 .../apache/cassandra/streaming/StreamRequest.java  |   17 +-
 .../apache/cassandra/streaming/StreamSession.java  |   47 +-
 .../streaming/StreamingMessageSender.java          |    6 +-
 .../async/NettyStreamingMessageSender.java         |   60 +-
 .../async/StreamCompressionSerializer.java         |   38 +-
 .../streaming/async/StreamingInboundHandler.java   |   35 +-
 .../ByteBufCompressionDataOutputStreamPlus.java    |   76 -
 .../compress/StreamCompressionInputStream.java     |    6 +-
 .../streaming/messages/StreamInitMessage.java      |    9 +-
 .../streaming/messages/StreamMessage.java          |   78 +-
 .../streaming/messages/StreamMessageHeader.java    |    9 +-
 .../cassandra/tools/BulkLoadConnectionFactory.java |   20 +-
 .../org/apache/cassandra/tools/BulkLoader.java     |    2 +-
 .../org/apache/cassandra/tools/LoaderOptions.java  |   21 +-
 src/java/org/apache/cassandra/tracing/Tracing.java |   65 +-
 .../cassandra/transport/ProtocolVersion.java       |    4 +-
 .../transport/ProtocolVersionTracker.java          |    5 +-
 .../cassandra/transport/messages/ErrorMessage.java |    2 +-
 .../apache/cassandra/utils/ApproximateTime.java    |  192 ++
 .../org/apache/cassandra/utils/ByteBufferUtil.java |    8 +-
 src/java/org/apache/cassandra/utils/Clock.java     |   80 -
 .../cassandra/utils/CoalescingStrategies.java      |  444 -----
 .../org/apache/cassandra/utils/ExecutorUtils.java  |  109 ++
 .../org/apache/cassandra/utils/ExpiringMap.java    |  186 --
 .../org/apache/cassandra/utils/FBUtilities.java    |    6 -
 .../cassandra/utils/JVMStabilityInspector.java     |   10 +-
 .../org/apache/cassandra/utils/MonotonicClock.java |  346 ++++
 .../MonotonicClockTranslation.java}                |   20 +-
 .../utils/NanoTimeToCurrentTimeMillis.java         |   72 -
 .../org/apache/cassandra/utils/Throwables.java     |   14 +-
 .../org/apache/cassandra/utils/concurrent/Ref.java |   14 +-
 .../utils/concurrent/SimpleCondition.java          |   11 +-
 .../cassandra/utils/concurrent/WaitQueue.java      |   20 +
 .../apache/cassandra/utils/memory/BufferPool.java  |  878 ++++++---
 .../apache/cassandra/utils/memory/MemoryUtil.java  |   29 +-
 .../cassandra/utils/memory/MemtablePool.java       |    8 +-
 .../apache/cassandra/utils/vint/VIntCoding.java    |   87 +-
 .../cassandra/net/BytesInFlightController.java     |  174 ++
 test/burn/org/apache/cassandra/net/Connection.java |  397 ++++
 .../apache/cassandra/net/ConnectionBurnTest.java   |  656 +++++++
 .../cassandra/net/GlobalInboundSettings.java       |   57 +
 .../org/apache/cassandra/net/LogbackFilter.java    |   80 +
 .../org/apache/cassandra/net/MessageGenerator.java |  190 ++
 .../apache/cassandra/net/MessageGenerators.java    |   28 +-
 test/burn/org/apache/cassandra/net/Reporters.java  |  322 ++++
 test/burn/org/apache/cassandra/net/Verifier.java   | 1637 +++++++++++++++++
 .../cassandra/utils/memory/LongBufferPoolTest.java |   73 +-
 test/conf/cassandra-murmur.yaml                    |    4 +-
 test/conf/cassandra-seeds.yaml                     |    2 +-
 test/conf/cassandra.yaml                           |    4 +-
 .../{logback-dtest.xml => logback-burntest.xml}    |   58 +-
 test/conf/logback-dtest.xml                        |    9 -
 .../serialization/4.0/service.SyncComplete.bin     |  Bin 554 -> 258 bytes
 .../data/serialization/4.0/service.SyncRequest.bin |  Bin 241 -> 111 bytes
 .../4.0/service.ValidationComplete.bin             |  Bin 1257 -> 600 bytes
 .../4.0/service.ValidationRequest.bin              |  Bin 169 -> 75 bytes
 .../org/apache/cassandra/distributed/Cluster.java  |   19 +-
 .../cassandra/distributed/UpgradeableCluster.java  |   23 +-
 .../cassandra/distributed/api/IInstanceConfig.java |    1 +
 .../apache/cassandra/distributed/api/IMessage.java |    1 +
 .../cassandra/distributed/api/IMessageFilters.java |    6 +-
 .../distributed/impl/AbstractCluster.java          |  118 +-
 .../cassandra/distributed/impl/ExecUtil.java       |   38 +-
 .../cassandra/distributed/impl/Instance.java       |  134 +-
 .../distributed/impl/InstanceClassLoader.java      |   10 +-
 .../cassandra/distributed/impl/InstanceConfig.java |   31 +-
 .../cassandra/distributed/impl/MessageFilters.java |   42 +-
 .../impl/{Message.java => MessageImpl.java}        |   21 +-
 .../test/DistributedReadWritePathTest.java         |   41 +-
 .../distributed/test/DistributedTestBase.java      |    7 +-
 .../distributed/test/GossipSettlesTest.java}       |   33 +-
 .../distributed/test/LargeColumnTest.java          |   96 +
 .../cassandra/distributed/test/RepairTest.java     |  134 ++
 .../cassandra/distributed/test/StreamingTest.java  |   75 +
 .../apache/cassandra/distributed/util/PyDtest.java |  186 ++
 .../locator/DynamicEndpointSnitchLongTest.java     |    4 +-
 .../cassandra/test/microbench/MessageOutBench.java |   67 +-
 .../cassandra/test/microbench/MutationBench.java   |   14 +-
 .../microbench/PreaggregatedByteBufsBench.java     |  107 ++
 .../microbench/ZeroCopyStreamingBenchmark.java     |   16 +-
 .../cassandra/batchlog/BatchlogManagerTest.java    |    7 +-
 .../cassandra/concurrent/SEPExecutorTest.java      |    4 +-
 .../org/apache/cassandra/db/ReadCommandTest.java   |    7 +-
 .../cassandra/db/ReadCommandVerbHandlerTest.java   |   84 +-
 .../db/commitlog/AbstractCommitLogServiceTest.java |    1 -
 .../db/compaction/AbstractPendingRepairTest.java   |   19 +-
 .../db/monitoring/MonitoringTaskTest.java          |   55 +-
 .../CassandraEntireSSTableStreamWriterTest.java    |   14 +-
 .../db/streaming/ComponentManifestTest.java        |   21 +-
 .../cassandra/db/virtual/SettingsTableTest.java    |   18 +-
 .../org/apache/cassandra/gms/ShadowRoundTest.java  |   20 +-
 test/unit/org/apache/cassandra/hints/HintTest.java |   13 +-
 .../apache/cassandra/hints/HintsServiceTest.java   |   20 +-
 .../apache/cassandra/index/sasi/SASIIndexTest.java |    4 +-
 .../cassandra/io/compress/CompressorTest.java      |    2 +-
 .../format/big/BigTableZeroCopyWriterTest.java     |   18 +-
 .../io/util/BufferedDataOutputStreamTest.java      |   44 -
 .../locator/DynamicEndpointSnitchTest.java         |    5 +-
 .../InetAddressAndPortSerializerTest.java}         |   18 +-
 .../cassandra/net/AsyncChannelPromiseTest.java     |   72 +
 .../cassandra/net/AsyncMessageOutputPlusTest.java  |  100 ++
 .../apache/cassandra/net/AsyncOneResponseTest.java |   30 +-
 .../org/apache/cassandra/net/AsyncPromiseTest.java |   73 +
 ...sTest.java => AsyncStreamingInputPlusTest.java} |  148 +-
 .../net/AsyncStreamingOutputPlusTest.java          |  114 ++
 .../apache/cassandra/net/ChunkedInputPlusTest.java |  159 ++
 .../org/apache/cassandra/net/ConnectionTest.java   |  811 +++++++++
 .../org/apache/cassandra/net/ConnectionUtils.java  |  253 +++
 ...oContainerTest.java => ForwardingInfoTest.java} |   12 +-
 .../unit/org/apache/cassandra/net/FramingTest.java |  432 +++++
 .../org/apache/cassandra/net/HandshakeTest.java    |  219 +++
 .../net/ManyToOneConcurrentLinkedQueueTest.java    |  301 ++++
 test/unit/org/apache/cassandra/net/Matcher.java    |    2 +-
 .../org/apache/cassandra/net/MatcherResponse.java  |  112 +-
 .../cassandra/net/MessageDeliveryTaskTest.java     |  121 --
 .../org/apache/cassandra/net/MessageInTest.java    |   66 -
 .../unit/org/apache/cassandra/net/MessageTest.java |  254 +++
 .../apache/cassandra/net/MessagingServiceTest.java |  470 ++---
 .../apache/cassandra/net/MockMessagingService.java |   17 +-
 .../cassandra/net/MockMessagingServiceTest.java    |   36 +-
 .../org/apache/cassandra/net/MockMessagingSpy.java |   36 +-
 .../net/OutboundConnectionSettingsTest.java        |  145 ++
 .../cassandra/net/OutboundConnectionsTest.java     |  162 ++
 .../cassandra/net/OutboundMessageQueueTest.java    |   94 +
 .../cassandra/net/PrunableArrayQueueTest.java      |  130 ++
 .../apache/cassandra/net/ResourceLimitsTest.java   |  152 ++
 .../unit/org/apache/cassandra/net/SocketUtils.java |   47 +-
 .../net/StartupClusterConnectivityCheckerTest.java |   91 +-
 .../cassandra/net/TestAbstractAsyncPromise.java    |  234 +++
 .../apache/cassandra/net/TestAbstractPromise.java  |  112 ++
 .../unit/org/apache/cassandra/net/TestChannel.java |  127 ++
 .../net/{async => }/TestScheduledFuture.java       |    2 +-
 .../cassandra/net/WriteCallbackInfoTest.java       |   10 +-
 .../net/async/ByteBufDataOutputPlusTest.java       |  178 --
 .../cassandra/net/async/ChannelWriterTest.java     |  314 ----
 .../cassandra/net/async/HandshakeHandlersTest.java |  227 ---
 .../cassandra/net/async/HandshakeProtocolTest.java |   95 -
 .../net/async/InboundHandshakeHandlerTest.java     |  291 ---
 .../cassandra/net/async/MessageInHandlerTest.java  |  339 ----
 .../cassandra/net/async/MessageOutHandlerTest.java |  288 ---
 .../cassandra/net/async/NettyFactoryTest.java      |  336 ----
 .../NonSendingOutboundMessagingConnection.java     |   42 -
 .../net/async/OutboundConnectionParamsTest.java    |   67 -
 .../net/async/OutboundHandshakeHandlerTest.java    |  258 ---
 .../net/async/OutboundMessagingConnectionTest.java |  521 ------
 .../net/async/OutboundMessagingPoolTest.java       |  150 --
 .../apache/cassandra/repair/LocalSyncTaskTest.java |    2 -
 .../org/apache/cassandra/repair/RepairJobTest.java |   75 +-
 .../org/apache/cassandra/repair/ValidatorTest.java |   51 +-
 .../consistent/CoordinatorMessagingTest.java       |   26 +-
 .../apache/cassandra/security/SSLFactoryTest.java  |   87 +-
 .../service/NativeTransportServiceTest.java        |   12 +-
 .../org/apache/cassandra/service/RemoveTest.java   |   15 +-
 .../cassandra/service/SerializationsTest.java      |   17 +-
 .../service/WriteResponseHandlerTest.java          |   48 +-
 .../service/reads/AbstractReadResponseTest.java    |   16 +-
 .../cassandra/service/reads/ReadExecutorTest.java  |   26 +-
 .../reads/repair/AbstractReadRepairTest.java       |   20 +-
 .../reads/repair/BlockingReadRepairTest.java       |    4 +-
 .../repair/DiagEventsBlockingReadRepairTest.java   |    4 +-
 .../service/reads/repair/ReadRepairTest.java       |    4 +-
 .../cassandra/streaming/StreamSessionTest.java     |   67 -
 .../streaming/StreamTransferTaskTest.java          |    2 +-
 .../async/NettyStreamingMessageSenderTest.java     |   10 +-
 .../async/StreamCompressionSerializerTest.java     |   25 +-
 .../async/StreamingInboundHandlerTest.java         |   43 +-
 .../apache/cassandra/utils/ByteBufferUtilTest.java |    4 +-
 .../cassandra/utils/CoalescingStrategiesTest.java  |  124 --
 .../apache/cassandra/utils/FreeRunningClock.java   |   28 +-
 ...TimeMillisTest.java => MonotonicClockTest.java} |    7 +-
 .../cassandra/utils/memory/BufferPoolTest.java     |   84 +-
 .../cassandra/utils/vint/VIntCodingTest.java       |   12 +-
 .../cassandra/stress/generate/SeedManager.java     |    1 -
 .../stress/settings/SettingsTransport.java         |   23 +-
 .../cassandra/stress/util/JavaDriverClient.java    |    2 +-
 452 files changed, 30394 insertions(+), 16913 deletions(-)

diff --git a/build.xml b/build.xml
index acd3b7a..bdf5ae2 100644
--- a/build.xml
+++ b/build.xml
@@ -1522,6 +1522,7 @@
   <target name="burn-testsome" depends="build-test" description="Execute specific burn unit tests" >
     <testmacro inputdir="${test.burn.src}" timeout="${test.burn.timeout}">
       <test name="${test.name}" methods="${test.methods}"/>
+      <jvmarg value="-Dlogback.configurationFile=test/conf/logback-burntest.xml"/>
     </testmacro>
   </target>
 
@@ -1945,6 +1946,20 @@
     <delete file="${test.distributed.listfile}"/>
   </target>
 
+  <!-- Build a self-contained jar for e.g. remote execution; not currently used for running burn tests with this build script -->
+  <target name="burn-test-jar" depends="build-test, build" description="Create dtest-compatible jar, including all dependencies">
+      <jar jarfile="${build.dir}/burntest.jar">
+          <zipgroupfileset dir="${build.lib}" includes="*.jar" excludes="META-INF/*.SF"/>
+          <fileset dir="${build.classes.main}"/>
+          <fileset dir="${test.classes}"/>
+          <fileset dir="${test.conf}" excludes="logback*.xml"/>
+          <fileset dir="${basedir}/conf" includes="logback*.xml"/>
+          <zipgroupfileset dir="${build.dir.lib}/jars">
+              <include name="junit*.jar"/>
+          </zipgroupfileset>
+      </jar>
+  </target>
+
   <target name="dtest-jar" depends="build-test, build" description="Create dtest-compatible jar, including all dependencies">
       <jar jarfile="${build.dir}/dtest-${base.version}.jar">
           <zipgroupfileset dir="${build.lib}" includes="*.jar" excludes="META-INF/*.SF"/>
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index ca854ca..3002857 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -870,6 +870,32 @@ request_timeout_in_ms: 10000
 # which picks up the OS default and configure the net.ipv4.tcp_retries2 sysctl to be ~8.
 # internode_tcp_user_timeout_in_ms = 30000
 
+# The maximum continuous period a connection may be unwritable in application space
+# internode_application_timeout_in_ms = 30000
+
+# Global, per-endpoint and per-connection limits imposed on messages queued for delivery to other nodes
+# and waiting to be processed on arrival from other nodes in the cluster.  These limits are applied to the on-wire
+# size of the message being sent or received.
+#
+# The basic per-link limit is consumed in isolation before any endpoint or global limit is imposed.
+# Each node-pair has three links: urgent, small and large.  So any given node may have a maximum of
+# N*3*(internode_application_send_queue_capacity_in_bytes+internode_application_receive_queue_capacity_in_bytes)
+# messages queued without any coordination between them although in practice, with token-aware routing, only RF*tokens
+# nodes should need to communicate with significant bandwidth.
+#
+# The per-endpoint limit is imposed on all messages exceeding the per-link limit, simultaneously with the global limit,
+# on all links to or from a single node in the cluster.
+# The global limit is imposed on all messages exceeding the per-link limit, simultaneously with the per-endpoint limit,
+# on all links to or from any node in the cluster.
+#
+# internode_application_send_queue_capacity_in_bytes: 4194304                       #4MiB
+# internode_application_send_queue_reserve_endpoint_capacity_in_bytes: 134217728    #128MiB
+# internode_application_send_queue_reserve_global_capacity_in_bytes: 536870912      #512MiB
+# internode_application_receive_queue_capacity_in_bytes: 4194304                    #4MiB
+# internode_application_receive_queue_reserve_endpoint_capacity_in_bytes: 134217728 #128MiB
+# internode_application_receive_queue_reserve_global_capacity_in_bytes: 536870912   #512MiB
+
+
 # How long before a node logs slow queries. Select queries that take longer than
 # this timeout to execute, will generate an aggregated log message, so that slow queries
 # can be identified. Set this value to zero to disable slow query logging.
diff --git a/src/java/org/apache/cassandra/auth/IAuthenticator.java b/src/java/org/apache/cassandra/auth/IAuthenticator.java
index 212e774..80ea719 100644
--- a/src/java/org/apache/cassandra/auth/IAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/IAuthenticator.java
@@ -105,7 +105,7 @@ public interface IAuthenticator
     public interface SaslNegotiator
     {
         /**
-         * Evaluates the client response data and generates a byte[] reply which may be a further challenge or purely
+         * Evaluates the client response data and generates a byte[] response which may be a further challenge or purely
          * informational in the case that the negotiation is completed on this round.
          *
          * This method is called each time a {@link org.apache.cassandra.transport.messages.AuthResponse} is received
diff --git a/src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java b/src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java
index 3c3fcec..3443cab 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java
@@ -20,11 +20,13 @@ package org.apache.cassandra.batchlog;
 import java.util.UUID;
 
 import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
 
 public final class BatchRemoveVerbHandler implements IVerbHandler<UUID>
 {
-    public void doVerb(MessageIn<UUID> message, int id)
+    public static final BatchRemoveVerbHandler instance = new BatchRemoveVerbHandler();
+
+    public void doVerb(Message<UUID> message)
     {
         BatchlogManager.remove(message.payload);
     }
diff --git a/src/java/org/apache/cassandra/batchlog/BatchStoreVerbHandler.java b/src/java/org/apache/cassandra/batchlog/BatchStoreVerbHandler.java
index 4bc878c..77335cb 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchStoreVerbHandler.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchStoreVerbHandler.java
@@ -17,16 +17,17 @@
  */
 package org.apache.cassandra.batchlog;
 
-import org.apache.cassandra.db.WriteResponse;
 import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
 
 public final class BatchStoreVerbHandler implements IVerbHandler<Batch>
 {
-    public void doVerb(MessageIn<Batch> message, int id)
+    public static final BatchStoreVerbHandler instance = new BatchStoreVerbHandler();
+
+    public void doVerb(Message<Batch> message)
     {
         BatchlogManager.store(message.payload);
-        MessagingService.instance().sendReply(WriteResponse.createMessage(), id, message.from);
+        MessagingService.instance().send(message.emptyResponse(), message.from());
     }
 }
diff --git a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
index b2b851d..d0d3411 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
@@ -65,8 +65,8 @@ import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.locator.ReplicaPlan;
 import org.apache.cassandra.locator.Replicas;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.Message;
+import org.apache.cassandra.net.MessageFlag;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.schema.TableId;
@@ -77,8 +77,10 @@ import org.apache.cassandra.utils.MBeanWrapper;
 import org.apache.cassandra.utils.UUIDGen;
 
 import static com.google.common.collect.Iterables.transform;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternalWithPaging;
+import static org.apache.cassandra.net.Verb.MUTATION_REQ;
 
 public class BatchlogManager implements BatchlogManagerMBean
 {
@@ -88,7 +90,7 @@ public class BatchlogManager implements BatchlogManagerMBean
 
     private static final Logger logger = LoggerFactory.getLogger(BatchlogManager.class);
     public static final BatchlogManager instance = new BatchlogManager();
-    public static final long BATCHLOG_REPLAY_TIMEOUT = Long.getLong("cassandra.batchlog.replay_timeout_in_ms", DatabaseDescriptor.getWriteRpcTimeout() * 2);
+    public static final long BATCHLOG_REPLAY_TIMEOUT = Long.getLong("cassandra.batchlog.replay_timeout_in_ms", DatabaseDescriptor.getWriteRpcTimeout(MILLISECONDS) * 2);
 
     private volatile long totalBatchesReplayed = 0; // no concurrency protection necessary as only written by replay thread.
     private volatile UUID lastReplayedUuid = UUIDGen.minTimeUUID(0);
@@ -112,7 +114,7 @@ public class BatchlogManager implements BatchlogManagerMBean
         batchlogTasks.scheduleWithFixedDelay(this::replayFailedBatches,
                                              StorageService.RING_DELAY,
                                              REPLAY_INTERVAL,
-                                             TimeUnit.MILLISECONDS);
+                                             MILLISECONDS);
     }
 
     public void shutdown() throws InterruptedException
@@ -356,7 +358,7 @@ public class BatchlogManager implements BatchlogManagerMBean
                 return 0;
 
             int gcgs = gcgs(mutations);
-            if (TimeUnit.MILLISECONDS.toSeconds(writtenAt) + gcgs <= FBUtilities.nowInSeconds())
+            if (MILLISECONDS.toSeconds(writtenAt) + gcgs <= FBUtilities.nowInSeconds())
                 return 0;
 
             replayHandlers = sendReplays(mutations, writtenAt, hintedNodes);
@@ -419,7 +421,7 @@ public class BatchlogManager implements BatchlogManagerMBean
             int gcgs = gcgs(mutations);
 
             // expired
-            if (TimeUnit.MILLISECONDS.toSeconds(writtenAt) + gcgs <= FBUtilities.nowInSeconds())
+            if (MILLISECONDS.toSeconds(writtenAt) + gcgs <= FBUtilities.nowInSeconds())
                 return;
 
             for (int i = startFrom; i < replayHandlers.size(); i++)
@@ -490,9 +492,9 @@ public class BatchlogManager implements BatchlogManagerMBean
             ReplicaPlan.ForTokenWrite replicaPlan = new ReplicaPlan.ForTokenWrite(keyspace, ConsistencyLevel.ONE,
                     liveRemoteOnly.pending(), liveRemoteOnly.all(), liveRemoteOnly.all(), liveRemoteOnly.all());
             ReplayWriteResponseHandler<Mutation> handler = new ReplayWriteResponseHandler<>(replicaPlan, System.nanoTime());
-            MessageOut<Mutation> message = mutation.createMessage();
+            Message<Mutation> message = Message.outWithFlag(MUTATION_REQ, mutation, MessageFlag.CALL_BACK_ON_FAILURE);
             for (Replica replica : liveRemoteOnly.all())
-                MessagingService.instance().sendWriteRR(message, replica, handler, false);
+                MessagingService.instance().sendWriteWithCallback(message, replica, handler, false);
             return handler;
         }
 
@@ -506,7 +508,7 @@ public class BatchlogManager implements BatchlogManagerMBean
 
         /**
          * A wrapper of WriteResponseHandler that stores the addresses of the endpoints from
-         * which we did not receive a successful reply.
+         * which we did not receive a successful response.
          */
         private static class ReplayWriteResponseHandler<T> extends WriteResponseHandler<T>
         {
@@ -525,11 +527,11 @@ public class BatchlogManager implements BatchlogManagerMBean
             }
 
             @Override
-            public void response(MessageIn<T> m)
+            public void onResponse(Message<T> m)
             {
-                boolean removed = undelivered.remove(m == null ? FBUtilities.getBroadcastAddressAndPort() : m.from);
+                boolean removed = undelivered.remove(m == null ? FBUtilities.getBroadcastAddressAndPort() : m.from());
                 assert removed;
-                super.response(m);
+                super.onResponse(m);
             }
         }
     }
diff --git a/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java b/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java
new file mode 100644
index 0000000..1a00e4f
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.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.concurrent;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.AbstractExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class ImmediateExecutor extends AbstractExecutorService implements LocalAwareExecutorService
+{
+    public static final ImmediateExecutor INSTANCE = new ImmediateExecutor();
+
+    private ImmediateExecutor() {}
+
+    public void execute(Runnable command, ExecutorLocals locals)
+    {
+        command.run();
+    }
+
+    public void maybeExecuteImmediately(Runnable command)
+    {
+        command.run();
+    }
+
+    public void execute(Runnable command)
+    {
+        command.run();
+    }
+
+    public int getActiveTaskCount() { return 0; }
+    public long getCompletedTaskCount() { return 0; }
+    public int getPendingTaskCount() { return 0; }
+    public int getMaximumPoolSize() { return 0; }
+    public void shutdown() { }
+    public List<Runnable> shutdownNow() { return Collections.emptyList(); }
+    public boolean isShutdown() { return false; }
+    public boolean isTerminated() { return false; }
+    public boolean awaitTermination(long timeout, TimeUnit unit) { return true; }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java b/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java
index 199803f..b54fa3f 100644
--- a/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java
@@ -70,7 +70,7 @@ public class InfiniteLoopExecutor
         return this;
     }
 
-    public void shutdown()
+    public void shutdownNow()
     {
         isShutdown = true;
         thread.interrupt();
diff --git a/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java b/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
index 33f1312..7cc73bd 100644
--- a/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
+++ b/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java
@@ -24,6 +24,7 @@ import com.google.common.annotations.VisibleForTesting;
 
 import io.netty.util.concurrent.FastThreadLocal;
 import io.netty.util.concurrent.FastThreadLocalThread;
+import org.apache.cassandra.utils.memory.BufferPool;
 
 /**
  * This class is an implementation of the <i>ThreadFactory</i> interface. This
@@ -35,6 +36,7 @@ public class NamedThreadFactory implements ThreadFactory
 {
     private static volatile String globalPrefix;
     public static void setGlobalPrefix(String prefix) { globalPrefix = prefix; }
+    public static String globalPrefix() { return globalPrefix == null ? "" : globalPrefix; }
 
     public final String id;
     private final int priority;
diff --git a/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java b/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java
index 5e3e5cf..90ceca5 100644
--- a/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java
+++ b/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java
@@ -17,10 +17,15 @@
  */
 package org.apache.cassandra.concurrent;
 
+import java.util.List;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.cassandra.utils.ExecutorUtils;
 
 /**
  * Centralized location for shared executors
@@ -48,12 +53,10 @@ public class ScheduledExecutors
     public static final DebuggableScheduledThreadPoolExecutor optionalTasks = new DebuggableScheduledThreadPoolExecutor("OptionalTasks");
 
     @VisibleForTesting
-    public static void shutdownAndWait() throws InterruptedException
+    public static void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
     {
-        ExecutorService[] executors = new ExecutorService[] { scheduledFastTasks, scheduledTasks, nonPeriodicTasks, optionalTasks };
-        for (ExecutorService executor : executors)
-            executor.shutdownNow();
-        for (ExecutorService executor : executors)
-            executor.awaitTermination(60, TimeUnit.SECONDS);
+        List<ExecutorService> executors = ImmutableList.of(scheduledFastTasks, scheduledTasks, nonPeriodicTasks, optionalTasks);
+        ExecutorUtils.shutdownNow(executors);
+        ExecutorUtils.awaitTermination(timeout, unit, executors);
     }
 }
diff --git a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
index 62bede9..53792ec 100644
--- a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
+++ b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
@@ -22,6 +22,7 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.LockSupport;
@@ -114,7 +115,7 @@ public class SharedExecutorPool
         return executor;
     }
 
-    public void shutdown() throws InterruptedException
+    public void shutdown(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
     {
         shuttingDown = true;
         for (SEPExecutor executor : executors)
@@ -122,9 +123,13 @@ public class SharedExecutorPool
 
         terminateWorkers();
 
-        long until = System.nanoTime() + TimeUnit.MINUTES.toNanos(1L);
+        long until = System.nanoTime() + unit.toNanos(timeout);
         for (SEPExecutor executor : executors)
+        {
             executor.shutdown.await(until - System.nanoTime(), TimeUnit.NANOSECONDS);
+            if (!executor.isTerminated())
+                throw new TimeoutException(executor.name + " not terminated");
+        }
     }
 
     void terminateWorkers()
diff --git a/src/java/org/apache/cassandra/concurrent/Stage.java b/src/java/org/apache/cassandra/concurrent/Stage.java
index ccb1565..ed13eeb 100644
--- a/src/java/org/apache/cassandra/concurrent/Stage.java
+++ b/src/java/org/apache/cassandra/concurrent/Stage.java
@@ -17,11 +17,6 @@
  */
 package org.apache.cassandra.concurrent;
 
-import java.util.Arrays;
-
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
-
 public enum Stage
 {
     READ,
@@ -35,18 +30,7 @@ public enum Stage
     MISC,
     TRACING,
     INTERNAL_RESPONSE,
-    READ_REPAIR;
-
-    public static Iterable<Stage> jmxEnabledStages()
-    {
-        return Iterables.filter(Arrays.asList(values()), new Predicate<Stage>()
-        {
-            public boolean apply(Stage stage)
-            {
-                return stage != TRACING;
-            }
-        });
-    }
+    IMMEDIATE;
 
     public String getJmxType()
     {
@@ -58,13 +42,13 @@ public enum Stage
             case MISC:
             case TRACING:
             case INTERNAL_RESPONSE:
+            case IMMEDIATE:
                 return "internal";
             case MUTATION:
             case COUNTER_MUTATION:
             case VIEW_MUTATION:
             case READ:
             case REQUEST_RESPONSE:
-            case READ_REPAIR:
                 return "request";
             default:
                 throw new AssertionError("Unknown stage " + this);
diff --git a/src/java/org/apache/cassandra/concurrent/StageManager.java b/src/java/org/apache/cassandra/concurrent/StageManager.java
index 608a005..46e8cea 100644
--- a/src/java/org/apache/cassandra/concurrent/StageManager.java
+++ b/src/java/org/apache/cassandra/concurrent/StageManager.java
@@ -17,7 +17,9 @@
  */
 package org.apache.cassandra.concurrent;
 
+import java.util.Collections;
 import java.util.EnumMap;
+import java.util.List;
 import java.util.concurrent.*;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -25,9 +27,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.Verb;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.apache.cassandra.config.DatabaseDescriptor.*;
+import static org.apache.cassandra.utils.ExecutorUtils.*;
 
 
 /**
@@ -56,24 +60,18 @@ public class StageManager
         stages.put(Stage.ANTI_ENTROPY, new JMXEnabledThreadPoolExecutor(Stage.ANTI_ENTROPY));
         stages.put(Stage.MIGRATION, new JMXEnabledThreadPoolExecutor(Stage.MIGRATION));
         stages.put(Stage.MISC, new JMXEnabledThreadPoolExecutor(Stage.MISC));
-        stages.put(Stage.READ_REPAIR, multiThreadedStage(Stage.READ_REPAIR, FBUtilities.getAvailableProcessors()));
         stages.put(Stage.TRACING, tracingExecutor());
+        stages.put(Stage.IMMEDIATE, ImmediateExecutor.INSTANCE);
     }
 
     private static LocalAwareExecutorService tracingExecutor()
     {
-        RejectedExecutionHandler reh = new RejectedExecutionHandler()
-        {
-            public void rejectedExecution(Runnable r, ThreadPoolExecutor executor)
-            {
-                MessagingService.instance().incrementDroppedMessages(MessagingService.Verb._TRACE);
-            }
-        };
+        RejectedExecutionHandler reh = (r, executor) -> MessagingService.instance().metrics.recordSelfDroppedMessage(Verb._TRACE);
         return new TracingExecutor(1,
                                    1,
                                    KEEPALIVE,
                                    TimeUnit.SECONDS,
-                                   new ArrayBlockingQueue<Runnable>(1000),
+                                   new ArrayBlockingQueue<>(1000),
                                    new NamedThreadFactory(Stage.TRACING.getJmxName()),
                                    reh);
     }
@@ -114,12 +112,10 @@ public class StageManager
     }
 
     @VisibleForTesting
-    public static void shutdownAndWait() throws InterruptedException
+    public static void shutdownAndWait(long timeout, TimeUnit units) throws InterruptedException, TimeoutException
     {
-        for (Stage stage : Stage.values())
-            StageManager.stages.get(stage).shutdown();
-        for (Stage stage : Stage.values())
-            StageManager.stages.get(stage).awaitTermination(60, TimeUnit.SECONDS);
+        shutdown(StageManager.stages.values());
+        awaitTermination(timeout, units, StageManager.stages.values());
     }
 
     /**
@@ -155,4 +151,5 @@ public class StageManager
             return getQueue().size();
         }
     }
+
 }
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index a6050be..9713ea2 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -149,8 +149,21 @@ public class Config
     public boolean rpc_interface_prefer_ipv6 = false;
     public String broadcast_rpc_address;
     public boolean rpc_keepalive = true;
-    public int internode_send_buff_size_in_bytes = 0;
-    public int internode_recv_buff_size_in_bytes = 0;
+
+    public Integer internode_max_message_size_in_bytes;
+
+    public int internode_socket_send_buffer_size_in_bytes = 0;
+    public int internode_socket_receive_buffer_size_in_bytes = 0;
+
+    // TODO: derive defaults from system memory settings?
+    public int internode_application_send_queue_capacity_in_bytes = 1 << 22; // 4MiB
+    public int internode_application_send_queue_reserve_endpoint_capacity_in_bytes = 1 << 27; // 128MiB
+    public int internode_application_send_queue_reserve_global_capacity_in_bytes = 1 << 29; // 512MiB
+
+    public int internode_application_receive_queue_capacity_in_bytes = 1 << 22; // 4MiB
+    public int internode_application_receive_queue_reserve_endpoint_capacity_in_bytes = 1 << 27; // 128MiB
+    public int internode_application_receive_queue_reserve_global_capacity_in_bytes = 1 << 29; // 512MiB
+
     // Defensive settings for protecting Cassandra from true network partitions. See (CASSANDRA-14358) for details.
     // The amount of time to wait for internode tcp connections to establish.
     public int internode_tcp_connect_timeout_in_ms = 2000;
@@ -171,6 +184,7 @@ public class Config
     public volatile boolean native_transport_allow_older_protocols = true;
     public int native_transport_frame_block_size_in_kb = 32;
 
+
     /**
      * Max size of values in SSTables, in MegaBytes.
      * Default is the same as the native protocol frame limit: 256Mb.
@@ -322,7 +336,7 @@ public class Config
     public volatile ConsistencyLevel ideal_consistency_level = null;
 
     /*
-     * Strategy to use for coalescing messages in {@link OutboundMessagingPool}.
+     * Strategy to use for coalescing messages in {@link OutboundConnections}.
      * Can be fixed, movingaverage, timehorizon, disabled. Setting is case and leading/trailing
      * whitespace insensitive. You can also specify a subclass of
      * {@link org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy} by name.
@@ -339,12 +353,6 @@ public class Config
     public int otc_coalescing_window_us = otc_coalescing_window_us_default;
     public int otc_coalescing_enough_coalesced_messages = 8;
 
-    /**
-     * Backlog expiration interval in milliseconds for the OutboundTcpConnection.
-     */
-    public static final int otc_backlog_expiration_interval_ms_default = 200;
-    public volatile int otc_backlog_expiration_interval_ms = otc_backlog_expiration_interval_ms_default;
-
     public int windows_timer_interval = 0;
 
     /**
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index b3ab054..3b7009b 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -27,6 +27,7 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
 import java.util.function.Supplier;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -72,6 +73,7 @@ import org.apache.cassandra.utils.FBUtilities;
 
 import org.apache.commons.lang3.StringUtils;
 
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.cassandra.io.util.FileUtils.ONE_GB;
 
 public class DatabaseDescriptor
@@ -80,6 +82,7 @@ public class DatabaseDescriptor
     {
         // This static block covers most usages
         FBUtilities.preventIllegalAccessWarnings();
+        System.setProperty("io.netty.transport.estimateSizeOnSubmit", "false");
     }
 
     private static final Logger logger = LoggerFactory.getLogger(DatabaseDescriptor.class);
@@ -801,6 +804,28 @@ public class DatabaseDescriptor
         if (conf.otc_coalescing_enough_coalesced_messages <= 0)
             throw new ConfigurationException("otc_coalescing_enough_coalesced_messages must be positive", false);
 
+        Integer maxMessageSize = conf.internode_max_message_size_in_bytes;
+        if (maxMessageSize != null)
+        {
+            if (maxMessageSize > conf.internode_application_receive_queue_reserve_endpoint_capacity_in_bytes)
+                throw new ConfigurationException("internode_max_message_size_in_mb must no exceed internode_application_receive_queue_reserve_endpoint_capacity_in_bytes", false);
+
+            if (maxMessageSize > conf.internode_application_receive_queue_reserve_global_capacity_in_bytes)
+                throw new ConfigurationException("internode_max_message_size_in_mb must no exceed internode_application_receive_queue_reserve_global_capacity_in_bytes", false);
+
+            if (maxMessageSize > conf.internode_application_send_queue_reserve_endpoint_capacity_in_bytes)
+                throw new ConfigurationException("internode_max_message_size_in_mb must no exceed internode_application_send_queue_reserve_endpoint_capacity_in_bytes", false);
+
+            if (maxMessageSize > conf.internode_application_send_queue_reserve_global_capacity_in_bytes)
+                throw new ConfigurationException("internode_max_message_size_in_mb must no exceed internode_application_send_queue_reserve_global_capacity_in_bytes", false);
+        }
+        else
+        {
+            conf.internode_max_message_size_in_bytes =
+                Math.min(conf.internode_application_receive_queue_reserve_endpoint_capacity_in_bytes,
+                         conf.internode_application_send_queue_reserve_endpoint_capacity_in_bytes);
+        }
+
         validateMaxConcurrentAutoUpgradeTasksConf(conf.max_concurrent_automatic_sstable_upgrades);
     }
 
@@ -1448,9 +1473,9 @@ public class DatabaseDescriptor
         return Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "ssl_storage_port", Integer.toString(conf.ssl_storage_port)));
     }
 
-    public static long getRpcTimeout()
+    public static long getRpcTimeout(TimeUnit unit)
     {
-        return conf.request_timeout_in_ms;
+        return unit.convert(conf.request_timeout_in_ms, MILLISECONDS);
     }
 
     public static void setRpcTimeout(long timeOutInMillis)
@@ -1458,9 +1483,9 @@ public class DatabaseDescriptor
         conf.request_timeout_in_ms = timeOutInMillis;
     }
 
-    public static long getReadRpcTimeout()
+    public static long getReadRpcTimeout(TimeUnit unit)
     {
-        return conf.read_request_timeout_in_ms;
+        return unit.convert(conf.read_request_timeout_in_ms, MILLISECONDS);
     }
 
     public static void setReadRpcTimeout(long timeOutInMillis)
@@ -1468,9 +1493,9 @@ public class DatabaseDescriptor
         conf.read_request_timeout_in_ms = timeOutInMillis;
     }
 
-    public static long getRangeRpcTimeout()
+    public static long getRangeRpcTimeout(TimeUnit unit)
     {
-        return conf.range_request_timeout_in_ms;
+        return unit.convert(conf.range_request_timeout_in_ms, MILLISECONDS);
     }
 
     public static void setRangeRpcTimeout(long timeOutInMillis)
@@ -1478,9 +1503,9 @@ public class DatabaseDescriptor
         conf.range_request_timeout_in_ms = timeOutInMillis;
     }
 
-    public static long getWriteRpcTimeout()
+    public static long getWriteRpcTimeout(TimeUnit unit)
     {
-        return conf.write_request_timeout_in_ms;
+        return unit.convert(conf.write_request_timeout_in_ms, MILLISECONDS);
     }
 
     public static void setWriteRpcTimeout(long timeOutInMillis)
@@ -1488,9 +1513,9 @@ public class DatabaseDescriptor
         conf.write_request_timeout_in_ms = timeOutInMillis;
     }
 
-    public static long getCounterWriteRpcTimeout()
+    public static long getCounterWriteRpcTimeout(TimeUnit unit)
     {
-        return conf.counter_write_request_timeout_in_ms;
+        return unit.convert(conf.counter_write_request_timeout_in_ms, MILLISECONDS);
     }
 
     public static void setCounterWriteRpcTimeout(long timeOutInMillis)
@@ -1498,9 +1523,9 @@ public class DatabaseDescriptor
         conf.counter_write_request_timeout_in_ms = timeOutInMillis;
     }
 
-    public static long getCasContentionTimeout()
+    public static long getCasContentionTimeout(TimeUnit unit)
     {
-        return conf.cas_contention_timeout_in_ms;
+        return unit.convert(conf.cas_contention_timeout_in_ms, MILLISECONDS);
     }
 
     public static void setCasContentionTimeout(long timeOutInMillis)
@@ -1508,9 +1533,9 @@ public class DatabaseDescriptor
         conf.cas_contention_timeout_in_ms = timeOutInMillis;
     }
 
-    public static long getTruncateRpcTimeout()
+    public static long getTruncateRpcTimeout(TimeUnit unit)
     {
-        return conf.truncate_request_timeout_in_ms;
+        return unit.convert(conf.truncate_request_timeout_in_ms, MILLISECONDS);
     }
 
     public static void setTruncateRpcTimeout(long timeOutInMillis)
@@ -1523,27 +1548,32 @@ public class DatabaseDescriptor
         return conf.cross_node_timeout;
     }
 
-    public static long getSlowQueryTimeout()
+    public static void setCrossNodeTimeout(boolean crossNodeTimeout)
+    {
+        conf.cross_node_timeout = crossNodeTimeout;
+    }
+
+    public static long getSlowQueryTimeout(TimeUnit units)
     {
-        return conf.slow_query_log_timeout_in_ms;
+        return units.convert(conf.slow_query_log_timeout_in_ms, MILLISECONDS);
     }
 
     /**
      * @return the minimum configured {read, write, range, truncate, misc} timeout
      */
-    public static long getMinRpcTimeout()
+    public static long getMinRpcTimeout(TimeUnit unit)
     {
-        return Longs.min(getRpcTimeout(),
-                         getReadRpcTimeout(),
-                         getRangeRpcTimeout(),
-                         getWriteRpcTimeout(),
-                         getCounterWriteRpcTimeout(),
-                         getTruncateRpcTimeout());
+        return Longs.min(getRpcTimeout(unit),
+                         getReadRpcTimeout(unit),
+                         getRangeRpcTimeout(unit),
+                         getWriteRpcTimeout(unit),
+                         getCounterWriteRpcTimeout(unit),
+                         getTruncateRpcTimeout(unit));
     }
 
-    public static long getPingTimeout()
+    public static long getPingTimeout(TimeUnit unit)
     {
-        return TimeUnit.SECONDS.toMillis(getBlockForPeersTimeoutInSeconds());
+        return unit.convert(getBlockForPeersTimeoutInSeconds(), TimeUnit.SECONDS);
     }
 
     public static double getPhiConvictThreshold()
@@ -1833,14 +1863,44 @@ public class DatabaseDescriptor
         return conf.rpc_keepalive;
     }
 
-    public static int getInternodeSendBufferSize()
+    public static int getInternodeSocketSendBufferSizeInBytes()
     {
-        return conf.internode_send_buff_size_in_bytes;
+        return conf.internode_socket_send_buffer_size_in_bytes;
     }
 
-    public static int getInternodeRecvBufferSize()
+    public static int getInternodeSocketReceiveBufferSizeInBytes()
     {
-        return conf.internode_recv_buff_size_in_bytes;
+        return conf.internode_socket_receive_buffer_size_in_bytes;
+    }
+
+    public static int getInternodeApplicationSendQueueCapacityInBytes()
+    {
+        return conf.internode_application_send_queue_capacity_in_bytes;
+    }
+
+    public static int getInternodeApplicationSendQueueReserveEndpointCapacityInBytes()
+    {
+        return conf.internode_application_send_queue_reserve_endpoint_capacity_in_bytes;
+    }
+
+    public static int getInternodeApplicationSendQueueReserveGlobalCapacityInBytes()
+    {
+        return conf.internode_application_send_queue_reserve_global_capacity_in_bytes;
+    }
+
+    public static int getInternodeApplicationReceiveQueueCapacityInBytes()
+    {
+        return conf.internode_application_receive_queue_capacity_in_bytes;
+    }
+
+    public static int getInternodeApplicationReceiveQueueReserveEndpointCapacityInBytes()
+    {
+        return conf.internode_application_receive_queue_reserve_endpoint_capacity_in_bytes;
+    }
+
+    public static int getInternodeApplicationReceiveQueueReserveGlobalCapacityInBytes()
+    {
+        return conf.internode_application_receive_queue_reserve_global_capacity_in_bytes;
     }
 
     public static int getInternodeTcpConnectTimeoutInMS()
@@ -1863,6 +1923,17 @@ public class DatabaseDescriptor
         conf.internode_tcp_user_timeout_in_ms = value;
     }
 
+    public static int getInternodeMaxMessageSizeInBytes()
+    {
+        return conf.internode_max_message_size_in_bytes;
+    }
+
+    @VisibleForTesting
+    public static void setInternodeMaxMessageSizeInBytes(int value)
+    {
+        conf.internode_max_message_size_in_bytes = value;
+    }
+
     public static boolean startNativeTransport()
     {
         return conf.start_native_transport;
@@ -2142,6 +2213,12 @@ public class DatabaseDescriptor
         return conf.client_encryption_options;
     }
 
+    @VisibleForTesting
+    public static void updateNativeProtocolEncryptionOptions(Function<EncryptionOptions, EncryptionOptions> update)
+    {
+        conf.client_encryption_options = update.apply(conf.client_encryption_options);
+    }
+
     public static int getHintedHandoffThrottleInKB()
     {
         return conf.hinted_handoff_throttle_in_kb;
@@ -2485,41 +2562,6 @@ public class DatabaseDescriptor
         return conf.tracetype_query_ttl;
     }
 
-    public static String getOtcCoalescingStrategy()
-    {
-        return conf.otc_coalescing_strategy;
-    }
-
-    public static void setOtcCoalescingStrategy(String strategy)
-    {
-        conf.otc_coalescing_strategy = strategy;
-    }
-
-    public static int getOtcCoalescingWindow()
-    {
-        return conf.otc_coalescing_window_us;
-    }
-
-    public static int getOtcCoalescingEnoughCoalescedMessages()
-    {
-        return conf.otc_coalescing_enough_coalesced_messages;
-    }
-
-    public static void setOtcCoalescingEnoughCoalescedMessages(int otc_coalescing_enough_coalesced_messages)
-    {
-        conf.otc_coalescing_enough_coalesced_messages = otc_coalescing_enough_coalesced_messages;
-    }
-
-    public static int getOtcBacklogExpirationInterval()
-    {
-        return conf.otc_backlog_expiration_interval_ms;
-    }
-
-    public static void setOtcBacklogExpirationInterval(int intervalInMillis)
-    {
-        conf.otc_backlog_expiration_interval_ms = intervalInMillis;
-    }
- 
     public static int getWindowsTimerInterval()
     {
         return conf.windows_timer_interval;
diff --git a/src/java/org/apache/cassandra/config/EncryptionOptions.java b/src/java/org/apache/cassandra/config/EncryptionOptions.java
index 9524cec..0a33dcc 100644
--- a/src/java/org/apache/cassandra/config/EncryptionOptions.java
+++ b/src/java/org/apache/cassandra/config/EncryptionOptions.java
@@ -17,30 +17,61 @@
  */
 package org.apache.cassandra.config;
 
-import java.util.Arrays;
+import java.util.List;
 import java.util.Objects;
 
+import com.google.common.collect.ImmutableList;
+
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
+
 public class EncryptionOptions
 {
-    public String keystore = "conf/.keystore";
-    public String keystore_password = "cassandra";
-    public String truststore = "conf/.truststore";
-    public String truststore_password = "cassandra";
-    public String[] cipher_suites = {};
-    public String protocol = "TLS";
-    public String algorithm = null;
-    public String store_type = "JKS";
-    public boolean require_client_auth = false;
-    public boolean require_endpoint_verification = false;
-    public boolean enabled = false;
-    public boolean optional = false;
+    public final String keystore;
+    public final String keystore_password;
+    public final String truststore;
+    public final String truststore_password;
+    public final List<String> cipher_suites;
+    public final String protocol;
+    public final String algorithm;
+    public final String store_type;
+    public final boolean require_client_auth;
+    public final boolean require_endpoint_verification;
+    public final boolean enabled;
+    public final boolean optional;
 
     public EncryptionOptions()
-    {   }
+    {
+        keystore = "conf/.keystore";
+        keystore_password = "cassandra";
+        truststore = "conf/.truststore";
+        truststore_password = "cassandra";
+        cipher_suites = ImmutableList.of();
+        protocol = "TLS";
+        algorithm = null;
+        store_type = "JKS";
+        require_client_auth = false;
+        require_endpoint_verification = false;
+        enabled = false;
+        optional = false;
+    }
+
+    public EncryptionOptions(String keystore, String keystore_password, String truststore, String truststore_password, List<String> cipher_suites, String protocol, String algorithm, String store_type, boolean require_client_auth, boolean require_endpoint_verification, boolean enabled, boolean optional)
+    {
+        this.keystore = keystore;
+        this.keystore_password = keystore_password;
+        this.truststore = truststore;
+        this.truststore_password = truststore_password;
+        this.cipher_suites = cipher_suites;
+        this.protocol = protocol;
+        this.algorithm = algorithm;
+        this.store_type = store_type;
+        this.require_client_auth = require_client_auth;
+        this.require_endpoint_verification = require_endpoint_verification;
+        this.enabled = enabled;
+        this.optional = optional;
+    }
 
-    /**
-     * Copy constructor
-     */
     public EncryptionOptions(EncryptionOptions options)
     {
         keystore = options.keystore;
@@ -57,6 +88,97 @@ public class EncryptionOptions
         optional = options.optional;
     }
 
+    public EncryptionOptions withKeyStore(String keystore)
+    {
+        return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                           protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                           enabled, optional);
+    }
+
+    public EncryptionOptions withKeyStorePassword(String keystore_password)
+    {
+        return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                           protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                           enabled, optional);
+    }
+
+    public EncryptionOptions withTrustStore(String truststore)
+    {
+        return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                           protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                           enabled, optional);
+    }
+
+    public EncryptionOptions withTrustStorePassword(String truststore_password)
+    {
+        return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                           protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                           enabled, optional);
+    }
+
+    public EncryptionOptions withCipherSuites(List<String> cipher_suites)
+    {
+        return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                           protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                           enabled, optional);
+    }
+
+    public EncryptionOptions withCipherSuites(String ... cipher_suites)
+    {
+        return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, ImmutableList.copyOf(cipher_suites),
+                                           protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                           enabled, optional);
+    }
+
+    public EncryptionOptions withProtocol(String protocol)
+    {
+        return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                           protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                           enabled, optional);
+    }
+
+    public EncryptionOptions withAlgorithm(String algorithm)
+    {
+        return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                           protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                           enabled, optional);
+    }
+
+    public EncryptionOptions withStoreType(String store_type)
+    {
+        return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                           protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                           enabled, optional);
+    }
+
+    public EncryptionOptions withRequireClientAuth(boolean require_client_auth)
+    {
+        return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                           protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                           enabled, optional);
+    }
+
+    public EncryptionOptions withRequireEndpointVerification(boolean require_endpoint_verification)
+    {
+        return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                           protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                           enabled, optional);
+    }
+
+    public EncryptionOptions withEnabled(boolean enabled)
+    {
+        return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                           protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                           enabled, optional);
+    }
+
+    public EncryptionOptions withOptional(boolean optional)
+    {
+        return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                           protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                           enabled, optional);
+    }
+
     /**
      * The method is being mainly used to cache SslContexts therefore, we only consider
      * fields that would make a difference when the TrustStore or KeyStore files are updated
@@ -81,7 +203,7 @@ public class EncryptionOptions
                Objects.equals(protocol, opt.protocol) &&
                Objects.equals(algorithm, opt.algorithm) &&
                Objects.equals(store_type, opt.store_type) &&
-               Arrays.equals(cipher_suites, opt.cipher_suites);
+               Objects.equals(cipher_suites, opt.cipher_suites);
     }
 
     /**
@@ -101,7 +223,7 @@ public class EncryptionOptions
         result += 31 * (store_type == null ? 0 : store_type.hashCode());
         result += 31 * Boolean.hashCode(enabled);
         result += 31 * Boolean.hashCode(optional);
-        result += 31 * Arrays.hashCode(cipher_suites);
+        result += 31 * (cipher_suites == null ? 0 : cipher_suites.hashCode());
         result += 31 * Boolean.hashCode(require_client_auth);
         result += 31 * Boolean.hashCode(require_endpoint_verification);
         return result;
@@ -114,20 +236,156 @@ public class EncryptionOptions
             all, none, dc, rack
         }
 
-        public InternodeEncryption internode_encryption = InternodeEncryption.none;
-        public boolean enable_legacy_ssl_storage_port = false;
+        public final InternodeEncryption internode_encryption;
+        public final boolean enable_legacy_ssl_storage_port;
 
         public ServerEncryptionOptions()
-        {   }
+        {
+            this.internode_encryption = InternodeEncryption.none;
+            this.enable_legacy_ssl_storage_port = false;
+        }
+        public ServerEncryptionOptions(String keystore, String keystore_password, String truststore, String truststore_password, List<String> cipher_suites, String protocol, String algorithm, String store_type, boolean require_client_auth, boolean require_endpoint_verification, boolean enabled, boolean optional, InternodeEncryption internode_encryption, boolean enable_legacy_ssl_storage_port)
+        {
+            super(keystore, keystore_password, truststore, truststore_password, cipher_suites, protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional);
+            this.internode_encryption = internode_encryption;
+            this.enable_legacy_ssl_storage_port = enable_legacy_ssl_storage_port;
+        }
 
-        /**
-         * Copy constructor
-         */
         public ServerEncryptionOptions(ServerEncryptionOptions options)
         {
             super(options);
-            internode_encryption = options.internode_encryption;
-            enable_legacy_ssl_storage_port = options.enable_legacy_ssl_storage_port;
+            this.internode_encryption = options.internode_encryption;
+            this.enable_legacy_ssl_storage_port = options.enable_legacy_ssl_storage_port;
         }
+
+        public boolean shouldEncrypt(InetAddressAndPort endpoint)
+        {
+            IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
+            switch (internode_encryption)
+            {
+                case none:
+                    return false; // if nothing needs to be encrypted then return immediately.
+                case all:
+                    break;
+                case dc:
+                    if (snitch.getDatacenter(endpoint).equals(snitch.getLocalDatacenter()))
+                        return false;
+                    break;
+                case rack:
+                    // for rack then check if the DC's are the same.
+                    if (snitch.getRack(endpoint).equals(snitch.getLocalRack())
+                        && snitch.getDatacenter(endpoint).equals(snitch.getLocalDatacenter()))
+                        return false;
+                    break;
+            }
+            return true;
+        }
+
+
+        public ServerEncryptionOptions withKeyStore(String keystore)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
+        public ServerEncryptionOptions withKeyStorePassword(String keystore_password)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
+        public ServerEncryptionOptions withTrustStore(String truststore)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
+        public ServerEncryptionOptions withTrustStorePassword(String truststore_password)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
+        public ServerEncryptionOptions withCipherSuites(List<String> cipher_suites)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
+        public ServerEncryptionOptions withCipherSuites(String ... cipher_suites)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, ImmutableList.copyOf(cipher_suites),
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
+        public ServerEncryptionOptions withProtocol(String protocol)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
+        public ServerEncryptionOptions withAlgorithm(String algorithm)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
+        public ServerEncryptionOptions withStoreType(String store_type)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
+        public ServerEncryptionOptions withRequireClientAuth(boolean require_client_auth)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
+        public ServerEncryptionOptions withRequireEndpointVerification(boolean require_endpoint_verification)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
+        public ServerEncryptionOptions withEnabled(boolean enabled)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
+        public ServerEncryptionOptions withOptional(boolean optional)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
+        public ServerEncryptionOptions withInternodeEncryption(InternodeEncryption internode_encryption)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
+        public ServerEncryptionOptions withLegacySslStoragePort(boolean enable_legacy_ssl_storage_port)
+        {
+            return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites,
+                                               protocol, algorithm, store_type, require_client_auth, require_endpoint_verification,
+                                               enabled, optional, internode_encryption, enable_legacy_ssl_storage_port);
+        }
+
     }
 }
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index c09b884..5414f23 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -87,6 +87,9 @@ import org.apache.cassandra.utils.memory.MemtableAllocator;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.cassandra.utils.ExecutorUtils.*;
+import static org.apache.cassandra.utils.ExecutorUtils.awaitTermination;
 import static org.apache.cassandra.utils.Throwables.maybeFail;
 
 public class ColumnFamilyStore implements ColumnFamilyStoreMBean
@@ -217,31 +220,22 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     private volatile boolean neverPurgeTombstones = false;
 
-    public static void shutdownFlushExecutor() throws InterruptedException
-    {
-        flushExecutor.shutdown();
-        flushExecutor.awaitTermination(60, TimeUnit.SECONDS);
-    }
-
-
     public static void shutdownPostFlushExecutor() throws InterruptedException
     {
         postFlushExecutor.shutdown();
         postFlushExecutor.awaitTermination(60, TimeUnit.SECONDS);
     }
 
-    public static void shutdownReclaimExecutor() throws InterruptedException
-    {
-        reclaimExecutor.shutdown();
-        reclaimExecutor.awaitTermination(60, TimeUnit.SECONDS);
-    }
-
-    public static void shutdownPerDiskFlushExecutors() throws InterruptedException
+    public static void shutdownExecutorsAndWait(long timeout, TimeUnit units) throws InterruptedException, TimeoutException
     {
-        for (ExecutorService executorService : perDiskflushExecutors)
-            executorService.shutdown();
-        for (ExecutorService executorService : perDiskflushExecutors)
-            executorService.awaitTermination(60, TimeUnit.SECONDS);
+        List<ExecutorService> executors = ImmutableList.<ExecutorService>builder()
+                                          .add(perDiskflushExecutors)
+                                          .add(reclaimExecutor)
+                                          .add(postFlushExecutor)
+                                          .add(flushExecutor)
+                                          .build();
+        shutdown(executors);
+        awaitTermination(timeout, units, executors);
     }
 
     public void reload()
@@ -401,8 +395,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         viewManager = keyspace.viewManager.forTable(metadata.id);
         metric = new TableMetrics(this);
         fileIndexGenerator.set(generation);
-        sampleReadLatencyNanos = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getReadRpcTimeout() / 2);
-        additionalWriteLatencyNanos = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getWriteRpcTimeout() / 2);
+        sampleReadLatencyNanos = DatabaseDescriptor.getReadRpcTimeout(NANOSECONDS) / 2;
+        additionalWriteLatencyNanos = DatabaseDescriptor.getWriteRpcTimeout(NANOSECONDS) / 2;
 
         logger.info("Initializing {}.{}", keyspace.getName(), name);
 
diff --git a/src/java/org/apache/cassandra/db/Columns.java b/src/java/org/apache/cassandra/db/Columns.java
index 5081739..bf9e174 100644
--- a/src/java/org/apache/cassandra/db/Columns.java
+++ b/src/java/org/apache/cassandra/db/Columns.java
@@ -28,6 +28,7 @@ import com.google.common.collect.Iterators;
 import com.google.common.hash.Hasher;
 
 import net.nicoulaj.compilecommand.annotations.DontInline;
+import org.apache.cassandra.exceptions.UnknownColumnException;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.ColumnIdentifier;
@@ -454,7 +455,7 @@ public class Columns extends AbstractCollection<ColumnMetadata> implements Colle
                     // deserialization. The column will be ignore later on anyway.
                     column = metadata.getDroppedColumn(name);
                     if (column == null)
-                        throw new RuntimeException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
+                        throw new UnknownColumnException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
                 }
                 builder.add(column);
             }
diff --git a/src/java/org/apache/cassandra/db/CounterMutation.java b/src/java/org/apache/cassandra/db/CounterMutation.java
index d04ddd8..bb10a6a 100644
--- a/src/java/org/apache/cassandra/db/CounterMutation.java
+++ b/src/java/org/apache/cassandra/db/CounterMutation.java
@@ -24,7 +24,6 @@ import java.util.concurrent.locks.Lock;
 
 import com.google.common.base.Function;
 import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.PeekingIterator;
@@ -39,14 +38,14 @@ import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.*;
 import org.apache.cassandra.utils.btree.BTreeSet;
 
+import static java.util.concurrent.TimeUnit.*;
+
 public class CounterMutation implements IMutation
 {
     public static final CounterMutationSerializer serializer = new CounterMutationSerializer();
@@ -92,11 +91,6 @@ public class CounterMutation implements IMutation
         return consistency;
     }
 
-    public MessageOut<CounterMutation> makeMutationMessage()
-    {
-        return new MessageOut<>(MessagingService.Verb.COUNTER_MUTATION, this, serializer);
-    }
-
     /**
      * Applies the counter mutation, returns the result Mutation (for replication to other nodes).
      *
@@ -146,10 +140,10 @@ public class CounterMutation implements IMutation
 
         for (Lock lock : LOCKS.bulkGet(getCounterLockKeys()))
         {
-            long timeout = TimeUnit.MILLISECONDS.toNanos(getTimeout()) - (System.nanoTime() - startTime);
+            long timeout = getTimeout(NANOSECONDS) - (System.nanoTime() - startTime);
             try
             {
-                if (!lock.tryLock(timeout, TimeUnit.NANOSECONDS))
+                if (!lock.tryLock(timeout, NANOSECONDS))
                     throw new WriteTimeoutException(WriteType.COUNTER, consistency(), 0, consistency().blockFor(keyspace));
                 locks.add(lock);
             }
@@ -309,9 +303,9 @@ public class CounterMutation implements IMutation
         }
     }
 
-    public long getTimeout()
+    public long getTimeout(TimeUnit unit)
     {
-        return DatabaseDescriptor.getCounterWriteRpcTimeout();
+        return DatabaseDescriptor.getCounterWriteRpcTimeout(unit);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
index c946ea5..a30ce66 100644
--- a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
@@ -22,16 +22,17 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageProxy;
-import org.apache.cassandra.utils.FBUtilities;
 
 public class CounterMutationVerbHandler implements IVerbHandler<CounterMutation>
 {
+    public static final CounterMutationVerbHandler instance = new CounterMutationVerbHandler();
+
     private static final Logger logger = LoggerFactory.getLogger(CounterMutationVerbHandler.class);
 
-    public void doVerb(final MessageIn<CounterMutation> message, final int id)
+    public void doVerb(final Message<CounterMutation> message)
     {
         long queryStartNanoTime = System.nanoTime();
         final CounterMutation cm = message.payload;
@@ -45,12 +46,9 @@ public class CounterMutationVerbHandler implements IVerbHandler<CounterMutation>
         // will not be called if the request timeout, but this is ok
         // because the coordinator of the counter mutation will timeout on
         // it's own in that case.
-        StorageProxy.applyCounterMutationOnLeader(cm, localDataCenter, new Runnable()
-        {
-            public void run()
-            {
-                MessagingService.instance().sendReply(WriteResponse.createMessage(), id, message.from);
-            }
-        }, queryStartNanoTime);
+        StorageProxy.applyCounterMutationOnLeader(cm,
+                                                  localDataCenter,
+                                                  () -> MessagingService.instance().send(message.emptyResponse(), message.from()),
+                                                  queryStartNanoTime);
     }
 }
diff --git a/src/java/org/apache/cassandra/db/IMutation.java b/src/java/org/apache/cassandra/db/IMutation.java
index 9eaf19b..1710cfd 100644
--- a/src/java/org/apache/cassandra/db/IMutation.java
+++ b/src/java/org/apache/cassandra/db/IMutation.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.db;
 
 import java.util.Collection;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.schema.TableId;
@@ -28,7 +29,7 @@ public interface IMutation
     public String getKeyspaceName();
     public Collection<TableId> getTableIds();
     public DecoratedKey key();
-    public long getTimeout();
+    public long getTimeout(TimeUnit unit);
     public String toString(boolean shallow);
     public Collection<PartitionUpdate> getPartitionUpdates();
 
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index bc382ee..4c1c2cf 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -56,6 +56,9 @@ import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.*;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+
 /**
  * It represents a Keyspace.
  */
@@ -544,7 +547,7 @@ public class Keyspace
                     if (lock == null)
                     {
                         //throw WTE only if request is droppable
-                        if (isDroppable && (System.currentTimeMillis() - mutation.createdAt) > DatabaseDescriptor.getWriteRpcTimeout())
+                        if (isDroppable && (approxTime.isAfter(mutation.approxCreatedAtNanos + DatabaseDescriptor.getWriteRpcTimeout(NANOSECONDS))))
                         {
                             for (int j = 0; j < i; j++)
                                 locks[j].unlock();
@@ -605,7 +608,7 @@ public class Keyspace
             if (isDroppable)
             {
                 for(TableId tableId : tableIds)
-                    columnFamilyStores.get(tableId).metric.viewLockAcquireTime.update(acquireTime, TimeUnit.MILLISECONDS);
+                    columnFamilyStores.get(tableId).metric.viewLockAcquireTime.update(acquireTime, MILLISECONDS);
             }
         }
         int nowInSec = FBUtilities.nowInSeconds();
diff --git a/src/java/org/apache/cassandra/db/Mutation.java b/src/java/org/apache/cassandra/db/Mutation.java
index 6195fe4..22c4ed8 100644
--- a/src/java/org/apache/cassandra/db/Mutation.java
+++ b/src/java/org/apache/cassandra/db/Mutation.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db;
 import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.collect.ImmutableCollection;
@@ -32,13 +33,13 @@ import org.apache.cassandra.db.rows.SerializationHelper;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+
 public class Mutation implements IMutation
 {
     public static final MutationSerializer serializer = new MutationSerializer();
@@ -52,7 +53,7 @@ public class Mutation implements IMutation
     private final ImmutableMap<TableId, PartitionUpdate> modifications;
 
     // Time at which this mutation or the builder that built it was instantiated
-    final long createdAt;
+    final long approxCreatedAtNanos;
     // keep track of when mutation has started waiting for a MV partition lock
     final AtomicLong viewLockAcquireStart = new AtomicLong(0);
 
@@ -60,10 +61,10 @@ public class Mutation implements IMutation
 
     public Mutation(PartitionUpdate update)
     {
-        this(update.metadata().keyspace, update.partitionKey(), ImmutableMap.of(update.metadata().id, update), System.currentTimeMillis());
+        this(update.metadata().keyspace, update.partitionKey(), ImmutableMap.of(update.metadata().id, update), approxTime.now());
     }
 
-    public Mutation(String keyspaceName, DecoratedKey key, ImmutableMap<TableId, PartitionUpdate> modifications, long createdAt)
+    public Mutation(String keyspaceName, DecoratedKey key, ImmutableMap<TableId, PartitionUpdate> modifications, long approxCreatedAtNanos)
     {
         this.keyspaceName = keyspaceName;
         this.key = key;
@@ -73,7 +74,7 @@ public class Mutation implements IMutation
         for (PartitionUpdate pu : modifications.values())
             cdc |= pu.metadata().params.cdc;
         this.cdcEnabled = cdc;
-        this.createdAt = createdAt;
+        this.approxCreatedAtNanos = approxCreatedAtNanos;
     }
 
     public Mutation without(Set<TableId> tableIds)
@@ -90,7 +91,7 @@ public class Mutation implements IMutation
             }
         }
 
-        return new Mutation(keyspaceName, key, builder.build(), createdAt);
+        return new Mutation(keyspaceName, key, builder.build(), approxCreatedAtNanos);
     }
 
     public Mutation without(TableId tableId)
@@ -177,7 +178,7 @@ public class Mutation implements IMutation
             modifications.put(table, updates.size() == 1 ? updates.get(0) : PartitionUpdate.merge(updates));
             updates.clear();
         }
-        return new Mutation(ks, key, modifications.build(), System.currentTimeMillis());
+        return new Mutation(ks, key, modifications.build(), approxTime.now());
     }
 
     public CompletableFuture<?> applyFuture()
@@ -210,19 +211,9 @@ public class Mutation implements IMutation
         apply(false);
     }
 
-    public MessageOut<Mutation> createMessage()
-    {
-        return createMessage(MessagingService.Verb.MUTATION);
-    }
-
-    public MessageOut<Mutation> createMessage(MessagingService.Verb verb)
-    {
-        return new MessageOut<>(verb, this, serializer);
-    }
-
-    public long getTimeout()
+    public long getTimeout(TimeUnit unit)
     {
-        return DatabaseDescriptor.getWriteRpcTimeout();
+        return DatabaseDescriptor.getWriteRpcTimeout(unit);
     }
 
     public int smallestGCGS()
@@ -363,7 +354,7 @@ public class Mutation implements IMutation
                 update = PartitionUpdate.serializer.deserialize(in, version, flag);
                 modifications.put(update.metadata().id, update);
             }
-            return new Mutation(update.metadata().keyspace, dk, modifications.build(), System.currentTimeMillis());
+            return new Mutation(update.metadata().keyspace, dk, modifications.build(), approxTime.now());
         }
 
         public Mutation deserialize(DataInputPlus in, int version) throws IOException
@@ -389,7 +380,7 @@ public class Mutation implements IMutation
         private final ImmutableMap.Builder<TableId, PartitionUpdate> modifications = new ImmutableMap.Builder<>();
         private final String keyspaceName;
         private final DecoratedKey key;
-        private final long createdAt = System.currentTimeMillis();
+        private final long approxCreatedAtNanos = approxTime.now();
         private boolean empty = true;
 
         public PartitionUpdateCollector(String keyspaceName, DecoratedKey key)
@@ -425,7 +416,7 @@ public class Mutation implements IMutation
 
         public Mutation build()
         {
-            return new Mutation(keyspaceName, key, modifications.build(), createdAt);
+            return new Mutation(keyspaceName, key, modifications.build(), approxCreatedAtNanos);
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/db/MutationVerbHandler.java b/src/java/org/apache/cassandra/db/MutationVerbHandler.java
index 9660f65..bcb9cc7 100644
--- a/src/java/org/apache/cassandra/db/MutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/MutationVerbHandler.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.util.Iterator;
-
 import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.*;
@@ -26,10 +24,12 @@ import org.apache.cassandra.tracing.Tracing;
 
 public class MutationVerbHandler implements IVerbHandler<Mutation>
 {
-    private void reply(int id, InetAddressAndPort replyTo)
+    public static final MutationVerbHandler instance = new MutationVerbHandler();
+
+    private void respond(Message<?> respondTo, InetAddressAndPort respondToAddress)
     {
-        Tracing.trace("Enqueuing response to {}", replyTo);
-        MessagingService.instance().sendReply(WriteResponse.createMessage(), id, replyTo);
+        Tracing.trace("Enqueuing response to {}", respondToAddress);
+        MessagingService.instance().send(respondTo.emptyResponse(), respondToAddress);
     }
 
     private void failed()
@@ -37,27 +37,25 @@ public class MutationVerbHandler implements IVerbHandler<Mutation>
         Tracing.trace("Payload application resulted in WriteTimeout, not replying");
     }
 
-    public void doVerb(MessageIn<Mutation> message, int id)
+    public void doVerb(Message<Mutation> message)
     {
         // Check if there were any forwarding headers in this message
-        InetAddressAndPort from = (InetAddressAndPort)message.parameters.get(ParameterType.FORWARD_FROM);
-        InetAddressAndPort replyTo;
+        InetAddressAndPort from = message.respondTo();
+        InetAddressAndPort respondToAddress;
         if (from == null)
         {
-            replyTo = message.from;
-            ForwardToContainer forwardTo = (ForwardToContainer)message.parameters.get(ParameterType.FORWARD_TO);
-            if (forwardTo != null)
-                forwardToLocalNodes(message.payload, message.verb, forwardTo, message.from);
+            respondToAddress = message.from();
+            ForwardingInfo forwardTo = message.forwardTo();
+            if (forwardTo != null) forwardToLocalNodes(message, forwardTo);
         }
         else
         {
-
-            replyTo = from;
+            respondToAddress = from;
         }
 
         try
         {
-            message.payload.applyFuture().thenAccept(o -> reply(id, replyTo)).exceptionally(wto -> {
+            message.payload.applyFuture().thenAccept(o -> respond(message, respondToAddress)).exceptionally(wto -> {
                 failed();
                 return null;
             });
@@ -68,17 +66,21 @@ public class MutationVerbHandler implements IVerbHandler<Mutation>
         }
     }
 
-    private static void forwardToLocalNodes(Mutation mutation, MessagingService.Verb verb, ForwardToContainer forwardTo, InetAddressAndPort from)
+    private static void forwardToLocalNodes(Message<Mutation> originalMessage, ForwardingInfo forwardTo)
     {
-        // tell the recipients who to send their ack to
-        MessageOut<Mutation> message = new MessageOut<>(verb, mutation, Mutation.serializer).withParameter(ParameterType.FORWARD_FROM, from);
-        Iterator<InetAddressAndPort> iterator = forwardTo.targets.iterator();
-        // Send a message to each of the addresses on our Forward List
-        for (int i = 0; i < forwardTo.targets.size(); i++)
+        Message.Builder<Mutation> builder =
+            Message.builder(originalMessage)
+                   .withParam(ParamType.RESPOND_TO, originalMessage.from())
+                   .withoutParam(ParamType.FORWARD_TO);
+
+        boolean useSameMessageID = forwardTo.useSameMessageID();
+        // reuse the same Message if all ids are identical (as they will be for 4.0+ node originated messages)
+        Message<Mutation> message = useSameMessageID ? builder.build() : null;
+
+        forwardTo.forEach((id, target) ->
         {
-            InetAddressAndPort address = iterator.next();
-            Tracing.trace("Enqueuing forwarded write to {}", address);
-            MessagingService.instance().sendOneWay(message, forwardTo.messageIds[i], address);
-        }
+            Tracing.trace("Enqueuing forwarded write to {}", target);
+            MessagingService.instance().send(useSameMessageID ? message : builder.withId(id).build(), target);
+        });
     }
 }
diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index b5f6fb5..2145389 100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@ -18,9 +18,12 @@
 package org.apache.cassandra.db;
 
 import java.io.IOException;
+import java.util.concurrent.TimeUnit;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import org.apache.cassandra.net.MessageFlag;
+import org.apache.cassandra.net.Verb;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.filter.*;
@@ -38,8 +41,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReadsListener;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.metrics.TableMetrics;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.Message;
 import org.apache.cassandra.schema.IndexMetadata;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.StorageProxy;
@@ -233,9 +235,9 @@ public class PartitionRangeReadCommand extends ReadCommand implements PartitionR
                                              indexMetadata());
     }
 
-    public long getTimeout()
+    public long getTimeout(TimeUnit unit)
     {
-        return DatabaseDescriptor.getRangeRpcTimeout();
+        return DatabaseDescriptor.getRangeRpcTimeout(unit);
     }
 
     public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState, long queryStartNanoTime) throws RequestExecutionException
@@ -345,9 +347,10 @@ public class PartitionRangeReadCommand extends ReadCommand implements PartitionR
         return Transformation.apply(iter, new CacheFilter());
     }
 
-    public MessageOut<ReadCommand> createMessage()
+    @Override
+    public Verb verb()
     {
-        return new MessageOut<>(MessagingService.Verb.RANGE_SLICE, this, serializer);
+        return Verb.RANGE_REQ;
     }
 
     protected void appendCQLWhereClause(StringBuilder sb)
@@ -414,6 +417,11 @@ public class PartitionRangeReadCommand extends ReadCommand implements PartitionR
             && dataRange.startKey().equals(dataRange.stopKey());
     }
 
+    public boolean isRangeRequest()
+    {
+        return true;
+    }
+
     private static class Deserializer extends SelectionDeserializer
     {
         public ReadCommand deserialize(DataInputPlus in,
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index 32b91ad..68ce2ea 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
+import java.util.concurrent.TimeUnit;
 import java.util.function.BiFunction;
 import java.util.function.LongPredicate;
 
@@ -35,7 +36,9 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.db.filter.*;
-import org.apache.cassandra.db.monitoring.ApproximateTime;
+import org.apache.cassandra.net.MessageFlag;
+import org.apache.cassandra.net.Verb;
+import org.apache.cassandra.utils.ApproximateTime;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.transform.RTBoundCloser;
@@ -52,9 +55,8 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.locator.Replica;
-import org.apache.cassandra.locator.ReplicaCollection;
 import org.apache.cassandra.metrics.TableMetrics;
-import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.Message;
 import org.apache.cassandra.schema.IndexMetadata;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.SchemaConstants;
@@ -69,6 +71,7 @@ import org.apache.cassandra.utils.HashingUtils;
 
 import static com.google.common.collect.Iterables.any;
 import static com.google.common.collect.Iterables.filter;
+import static org.apache.cassandra.utils.MonotonicClock.approxTime;
 
 /**
  * General interface for storage-engine read commands (common to both range and
@@ -164,6 +167,8 @@ public abstract class ReadCommand extends AbstractReadQuery
 
     public abstract boolean isLimitedToOnePartition();
 
+    public abstract boolean isRangeRequest();
+
     /**
      * Creates a new <code>ReadCommand</code> instance with new limits.
      *
@@ -177,7 +182,7 @@ public abstract class ReadCommand extends AbstractReadQuery
      *
      * @return the configured timeout for this command.
      */
-    public abstract long getTimeout();
+    public abstract long getTimeout(TimeUnit unit);
 
     /**
      * Whether this query is a digest one or not.
@@ -628,14 +633,15 @@ public abstract class ReadCommand extends AbstractReadQuery
         private boolean maybeAbort()
         {
             /**
-             * The value returned by ApproximateTime.currentTimeMillis() is updated only every
-             * {@link ApproximateTime.CHECK_INTERVAL_MS}, by default 10 millis. Since MonitorableImpl
-             * relies on ApproximateTime, we don't need to check unless the approximate time has elapsed.
+             * TODO: this is not a great way to abort early; why not expressly limit checks to 10ms intervals?
+             * The value returned by approxTime.now() is updated only every
+             * {@link org.apache.cassandra.utils.MonotonicClock.SampledClock.CHECK_INTERVAL_MS}, by default 2 millis. Since MonitorableImpl
+             * relies on approxTime, we don't need to check unless the approximate time has elapsed.
              */
-            if (lastChecked == ApproximateTime.currentTimeMillis())
+            if (lastChecked == approxTime.now())
                 return false;
 
-            lastChecked = ApproximateTime.currentTimeMillis();
+            lastChecked = approxTime.now();
 
             if (isAborted())
             {
@@ -661,7 +667,14 @@ public abstract class ReadCommand extends AbstractReadQuery
     /**
      * Creates a message for this command.
      */
-    public abstract MessageOut<ReadCommand> createMessage();
+    public Message<ReadCommand> createMessage(boolean trackRepairedData)
+    {
+        return trackRepairedData
+             ? Message.outWithFlags(verb(), this, MessageFlag.CALL_BACK_ON_FAILURE, MessageFlag.TRACK_REPAIRED_DATA)
+             : Message.outWithFlag (verb(), this, MessageFlag.CALL_BACK_ON_FAILURE);
+    }
+
+    public abstract Verb verb();
 
     protected abstract void appendCQLWhereClause(StringBuilder sb);
 
diff --git a/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java b/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java
index e39e8a8..2c28ed9 100644
--- a/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java
@@ -20,29 +20,26 @@ package org.apache.cassandra.db;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.net.ParameterType;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.tracing.Tracing;
 
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
 public class ReadCommandVerbHandler implements IVerbHandler<ReadCommand>
 {
-    private static final Logger logger = LoggerFactory.getLogger(ReadCommandVerbHandler.class);
+    public static final ReadCommandVerbHandler instance = new ReadCommandVerbHandler();
 
-    protected IVersionedSerializer<ReadResponse> serializer()
-    {
-        return ReadResponse.serializer;
-    }
+    private static final Logger logger = LoggerFactory.getLogger(ReadCommandVerbHandler.class);
 
-    public void doVerb(MessageIn<ReadCommand> message, int id)
+    public void doVerb(Message<ReadCommand> message)
     {
         if (StorageService.instance.isBootstrapMode())
         {
@@ -52,9 +49,10 @@ public class ReadCommandVerbHandler implements IVerbHandler<ReadCommand>
         ReadCommand command = message.payload;
         validateTransientStatus(message);
 
-        command.setMonitoringTime(message.constructionTime, message.isCrossNode(), message.getTimeout(), message.getSlowQueryTimeout());
+        long timeout = message.expiresAtNanos() - message.createdAtNanos();
+        command.setMonitoringTime(message.createdAtNanos(), message.isCrossNode(), timeout, DatabaseDescriptor.getSlowQueryTimeout(NANOSECONDS));
 
-        if (message.parameters.containsKey(ParameterType.TRACK_REPAIRED_DATA))
+        if (message.trackRepairedData())
             command.trackRepairedStatus();
 
         ReadResponse response;
@@ -66,17 +64,17 @@ public class ReadCommandVerbHandler implements IVerbHandler<ReadCommand>
 
         if (!command.complete())
         {
-            Tracing.trace("Discarding partial response to {} (timed out)", message.from);
-            MessagingService.instance().incrementDroppedMessages(message, message.getLifetimeInMS());
+            Tracing.trace("Discarding partial response to {} (timed out)", message.from());
+            MessagingService.instance().metrics.recordDroppedMessage(message, message.elapsedSinceCreated(NANOSECONDS), NANOSECONDS);
             return;
         }
 
-        Tracing.trace("Enqueuing response to {}", message.from);
-        MessageOut<ReadResponse> reply = new MessageOut<>(MessagingService.Verb.REQUEST_RESPONSE, response, serializer());
-        MessagingService.instance().sendReply(reply, id, message.from);
+        Tracing.trace("Enqueuing response to {}", message.from());
+        Message<ReadResponse> reply = message.responseWith(response);
+        MessagingService.instance().send(reply, message.from());
     }
 
-    private void validateTransientStatus(MessageIn<ReadCommand> message)
+    private void validateTransientStatus(Message<ReadCommand> message)
     {
         ReadCommand command = message.payload;
         Token token;
@@ -93,14 +91,14 @@ public class ReadCommandVerbHandler implements IVerbHandler<ReadCommand>
         if (replica == null)
         {
             logger.warn("Received a read request from {} for a range that is not owned by the current replica {}.",
-                        message.from,
+                        message.from(),
                         command);
             return;
         }
 
         if (!command.acceptsTransient() && replica.isTransient())
         {
-            MessagingService.instance().incrementDroppedMessages(message, message.getLifetimeInMS());
+            MessagingService.instance().metrics.recordDroppedMessage(message, message.elapsedSinceCreated(NANOSECONDS), NANOSECONDS);
             throw new InvalidRequestException(String.format("Attempted to serve %s data request from %s node in %s",
                                                             command.acceptsTransient() ? "transient" : "full",
                                                             replica.isTransient() ? "transient" : "full",
diff --git a/src/java/org/apache/cassandra/db/ReadExecutionController.java b/src/java/org/apache/cassandra/db/ReadExecutionController.java
index 29b6fa7..73ddad8 100644
--- a/src/java/org/apache/cassandra/db/ReadExecutionController.java
+++ b/src/java/org/apache/cassandra/db/ReadExecutionController.java
@@ -21,9 +21,11 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.cassandra.index.Index;
 import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.MonotonicClock;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
+import static org.apache.cassandra.utils.MonotonicClock.preciseTime;
+
 public class ReadExecutionController implements AutoCloseable
 {
     private static final long NO_SAMPLING = Long.MIN_VALUE;
@@ -36,7 +38,7 @@ public class ReadExecutionController implements AutoCloseable
     private final ReadExecutionController indexController;
     private final WriteContext writeContext;
     private final ReadCommand command;
-    static Clock clock = Clock.instance; 
+    static MonotonicClock clock = preciseTime;
 
     private final long createdAtNanos; // Only used while sampling
 
@@ -93,7 +95,7 @@ public class ReadExecutionController implements AutoCloseable
         ColumnFamilyStore baseCfs = Keyspace.openAndGetStore(command.metadata());
         ColumnFamilyStore indexCfs = maybeGetIndexCfs(baseCfs, command);
 
-        long createdAtNanos = baseCfs.metric.topLocalReadQueryTime.isEnabled() ? clock.nanoTime() : NO_SAMPLING;
+        long createdAtNanos = baseCfs.metric.topLocalReadQueryTime.isEnabled() ? clock.now() : NO_SAMPLING;
 
         if (indexCfs == null)
             return new ReadExecutionController(command, baseCfs.readOrdering.start(), baseCfs.metadata(), null, null, createdAtNanos);
@@ -172,7 +174,7 @@ public class ReadExecutionController implements AutoCloseable
     private void addSample()
     {
         String cql = command.toCQLString();
-        int timeMicros = (int) Math.min(TimeUnit.NANOSECONDS.toMicros(clock.nanoTime() - createdAtNanos), Integer.MAX_VALUE);
+        int timeMicros = (int) Math.min(TimeUnit.NANOSECONDS.toMicros(clock.now() - createdAtNanos), Integer.MAX_VALUE);
         ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(baseMetadata.id);
         if (cfs != null)
             cfs.metric.topLocalReadQueryTime.addSample(cql, timeMicros);
diff --git a/src/java/org/apache/cassandra/db/ReadRepairVerbHandler.java b/src/java/org/apache/cassandra/db/ReadRepairVerbHandler.java
index 2e499e7..903b3d4 100644
--- a/src/java/org/apache/cassandra/db/ReadRepairVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/ReadRepairVerbHandler.java
@@ -18,14 +18,16 @@
 package org.apache.cassandra.db;
 
 import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
 
 public class ReadRepairVerbHandler implements IVerbHandler<Mutation>
 {
-    public void doVerb(MessageIn<Mutation> message, int id)
+    public static final ReadRepairVerbHandler instance = new ReadRepairVerbHandler();
+
+    public void doVerb(Message<Mutation> message)
     {
         message.payload.apply();
-        MessagingService.instance().sendReply(WriteResponse.createMessage(), id, message.from);
+        MessagingService.instance().send(message.emptyResponse(), message.from());
     }
 }
diff --git a/src/java/org/apache/cassandra/db/SerializationHeader.java b/src/java/org/apache/cassandra/db/SerializationHeader.java
index deadf68..2e5211c 100644
--- a/src/java/org/apache/cassandra/db/SerializationHeader.java
+++ b/src/java/org/apache/cassandra/db/SerializationHeader.java
@@ -27,6 +27,7 @@ import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.TypeParser;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.exceptions.UnknownColumnException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.sstable.metadata.IMetadataComponentSerializer;
@@ -292,7 +293,7 @@ public class SerializationHeader
             return MetadataType.HEADER;
         }
 
-        public SerializationHeader toHeader(TableMetadata metadata)
+        public SerializationHeader toHeader(TableMetadata metadata) throws UnknownColumnException
         {
             Map<ByteBuffer, AbstractType<?>> typeMap = new HashMap<>(staticColumns.size() + regularColumns.size());
 
@@ -320,7 +321,7 @@ public class SerializationHeader
                         // deserialization. The column will be ignore later on anyway.
                         column = metadata.getDroppedColumn(name, isStatic);
                         if (column == null)
-                            throw new RuntimeException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
+                            throw new UnknownColumnException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
                     }
                     builder.add(column);
                 }
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index aec1a54..8c983aa 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -20,10 +20,9 @@ package org.apache.cassandra.db;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
+import java.util.concurrent.TimeUnit;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 
 import org.apache.cassandra.cache.IRowCacheEntry;
@@ -43,12 +42,11 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableReadsListener;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.locator.Replica;
-import org.apache.cassandra.locator.ReplicaCollection;
 import org.apache.cassandra.metrics.TableMetrics;
-import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.Message;
+import org.apache.cassandra.net.MessageFlag;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.net.ParameterType;
+import org.apache.cassandra.net.Verb;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.IndexMetadata;
 import org.apache.cassandra.schema.TableMetadata;
@@ -364,9 +362,9 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
         return clusteringIndexFilter;
     }
 
-    public long getTimeout()
+    public long getTimeout(TimeUnit unit)
     {
-        return DatabaseDescriptor.getReadRpcTimeout();
+        return DatabaseDescriptor.getReadRpcTimeout(unit);
     }
 
     @Override
@@ -1040,9 +1038,10 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
                              nowInSec());
     }
 
-    public MessageOut<ReadCommand> createMessage()
+    @Override
+    public Verb verb()
     {
-        return new MessageOut<>(MessagingService.Verb.READ, this, serializer);
+        return Verb.READ_REQ;
     }
 
     protected void appendCQLWhereClause(StringBuilder sb)
@@ -1078,6 +1077,11 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
         return true;
     }
 
+    public boolean isRangeRequest()
+    {
+        return false;
+    }
+
     /**
      * Groups multiple single partition read commands.
      */
diff --git a/src/java/org/apache/cassandra/db/SnapshotCommand.java b/src/java/org/apache/cassandra/db/SnapshotCommand.java
index eb6f67a..484db2f 100644
--- a/src/java/org/apache/cassandra/db/SnapshotCommand.java
+++ b/src/java/org/apache/cassandra/db/SnapshotCommand.java
@@ -22,8 +22,8 @@ import java.io.IOException;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.Message;
+import org.apache.cassandra.net.Verb;
 
 public class SnapshotCommand
 {
@@ -42,11 +42,6 @@ public class SnapshotCommand
         this.clear_snapshot = clearSnapshot;
     }
 
-    public MessageOut createMessage()
-    {
-        return new MessageOut<SnapshotCommand>(MessagingService.Verb.SNAPSHOT, this, serializer);
-    }
-
     @Override
     public String toString()
     {
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index d48f84f..0d79ae9 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -688,16 +688,17 @@ public final class SystemKeyspace
         executeInternal(String.format(req, PEERS_V2), ep.address, ep.port, tokensAsSet(tokens));
     }
 
-    public static synchronized void updatePreferredIP(InetAddressAndPort ep, InetAddressAndPort preferred_ip)
+    public static synchronized boolean updatePreferredIP(InetAddressAndPort ep, InetAddressAndPort preferred_ip)
     {
-        if (getPreferredIP(ep) == preferred_ip)
-            return;
+        if (preferred_ip.equals(getPreferredIP(ep)))
+            return false;
 
         String req = "INSERT INTO system.%s (peer, preferred_ip) VALUES (?, ?)";
         executeInternal(String.format(req, LEGACY_PEERS), ep.address, preferred_ip.address);
         req = "INSERT INTO system.%s (peer, peer_port, preferred_ip, preferred_port) VALUES (?, ?, ?, ?)";
         executeInternal(String.format(req, PEERS_V2), ep.address, ep.port, preferred_ip.address, preferred_ip.port);
         forceBlockingFlush(LEGACY_PEERS, PEERS_V2);
+        return true;
     }
 
     public static synchronized void updatePeerInfo(InetAddressAndPort ep, String columnName, Object value)
diff --git a/src/java/org/apache/cassandra/db/TruncateRequest.java b/src/java/org/apache/cassandra/db/TruncateRequest.java
new file mode 100644
index 0000000..64950b1
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/TruncateRequest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.db;
+
+import java.io.IOException;
+
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * A truncate operation descriptor
+ */
+public class TruncateRequest
+{
+    public static final IVersionedSerializer<TruncateRequest> serializer = new Serializer();
+
+    public final String keyspace;
+    public final String table;
+
+    public TruncateRequest(String keyspace, String table)
+    {
+        this.keyspace = keyspace;
+        this.table = table;
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("TruncateRequest(keyspace='%s', table='%s')'", keyspace, table);
+    }
+
+    private static class Serializer implements IVersionedSerializer<TruncateRequest>
+    {
+        public void serialize(TruncateRequest request, DataOutputPlus out, int version) throws IOException
+        {
+            out.writeUTF(request.keyspace);
+            out.writeUTF(request.table);
+        }
+
+        public TruncateRequest deserialize(DataInputPlus in, int version) throws IOException
+        {
+            String keyspace = in.readUTF();
+            String table = in.readUTF();
+            return new TruncateRequest(keyspace, table);
+        }
+
+        public long serializedSize(TruncateRequest request, int version)
+        {
+            return TypeSizes.sizeof(request.keyspace) + TypeSizes.sizeof(request.table);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/TruncateResponse.java b/src/java/org/apache/cassandra/db/TruncateResponse.java
index af4ed8f..822c9cc 100644
--- a/src/java/org/apache/cassandra/db/TruncateResponse.java
+++ b/src/java/org/apache/cassandra/db/TruncateResponse.java
@@ -22,8 +22,6 @@ import java.io.IOException;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
 
 /**
  * This message is sent back the truncate operation and basically specifies if
@@ -44,11 +42,6 @@ public class TruncateResponse
         this.success = success;
     }
 
-    public MessageOut<TruncateResponse> createMessage()
-    {
-        return new MessageOut<TruncateResponse>(MessagingService.Verb.REQUEST_RESPONSE, this, serializer);
-    }
-
     public static class TruncateResponseSerializer implements IVersionedSerializer<TruncateResponse>
     {
         public void serialize(TruncateResponse tr, DataOutputPlus out, int version) throws IOException
diff --git a/src/java/org/apache/cassandra/db/TruncateVerbHandler.java b/src/java/org/apache/cassandra/db/TruncateVerbHandler.java
index c2fac65..c605d1f 100644
--- a/src/java/org/apache/cassandra/db/TruncateVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/TruncateVerbHandler.java
@@ -22,21 +22,23 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.tracing.Tracing;
 
-public class TruncateVerbHandler implements IVerbHandler<Truncation>
+public class TruncateVerbHandler implements IVerbHandler<TruncateRequest>
 {
+    public static final TruncateVerbHandler instance = new TruncateVerbHandler();
+
     private static final Logger logger = LoggerFactory.getLogger(TruncateVerbHandler.class);
 
-    public void doVerb(MessageIn<Truncation> message, int id)
+    public void doVerb(Message<TruncateRequest> message)
     {
-        Truncation t = message.payload;
-        Tracing.trace("Applying truncation of {}.{}", t.keyspace, t.columnFamily);
+        TruncateRequest t = message.payload;
+        Tracing.trace("Applying truncation of {}.{}", t.keyspace, t.table);
         try
         {
-            ColumnFamilyStore cfs = Keyspace.open(t.keyspace).getColumnFamilyStore(t.columnFamily);
+            ColumnFamilyStore cfs = Keyspace.open(t.keyspace).getColumnFamilyStore(t.table);
             cfs.truncateBlocking();
         }
         catch (Exception e)
@@ -47,16 +49,16 @@ public class TruncateVerbHandler implements IVerbHandler<Truncation>
             if (FSError.findNested(e) != null)
                 throw FSError.findNested(e);
         }
-        Tracing.trace("Enqueuing response to truncate operation to {}", message.from);
+        Tracing.trace("Enqueuing response to truncate operation to {}", message.from());
 
-        TruncateResponse response = new TruncateResponse(t.keyspace, t.columnFamily, true);
-        logger.trace("{} applied.  Enqueuing response to {}@{} ", t, id, message.from );
-        MessagingService.instance().sendReply(response.createMessage(), id, message.from);
+        TruncateResponse response = new TruncateResponse(t.keyspace, t.table, true);
+        logger.trace("{} applied.  Enqueuing response to {}@{} ", t, message.id(), message.from());
+        MessagingService.instance().send(message.responseWith(response), message.from());
     }
 
-    private static void respondError(Truncation t, MessageIn truncateRequestMessage)
+    private static void respondError(TruncateRequest t, Message truncateRequestMessage)
     {
-        TruncateResponse response = new TruncateResponse(t.keyspace, t.columnFamily, false);
-        MessagingService.instance().sendOneWay(response.createMessage(), truncateRequestMessage.from);
+        TruncateResponse response = new TruncateResponse(t.keyspace, t.table, false);
+        MessagingService.instance().send(truncateRequestMessage.responseWith(response), truncateRequestMessage.from());
     }
 }
diff --git a/src/java/org/apache/cassandra/db/Truncation.java b/src/java/org/apache/cassandra/db/Truncation.java
deleted file mode 100644
index 39a2ec6..0000000
--- a/src/java/org/apache/cassandra/db/Truncation.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db;
-
-import java.io.IOException;
-
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-
-/**
- * A truncate operation descriptor
- */
-public class Truncation
-{
-    public static final IVersionedSerializer<Truncation> serializer = new TruncationSerializer();
-
-    public final String keyspace;
-    public final String columnFamily;
-
-    public Truncation(String keyspace, String columnFamily)
-    {
-        this.keyspace = keyspace;
-        this.columnFamily = columnFamily;
-    }
-
-    public MessageOut<Truncation> createMessage()
-    {
-        return new MessageOut<Truncation>(MessagingService.Verb.TRUNCATE, this, serializer);
-    }
-
-    public String toString()
-    {
-        return "Truncation(" + "keyspace='" + keyspace + '\'' + ", cf='" + columnFamily + "\')";
-    }
-}
-
-class TruncationSerializer implements IVersionedSerializer<Truncation>
-{
-    public void serialize(Truncation t, DataOutputPlus out, int version) throws IOException
-    {
-        out.writeUTF(t.keyspace);
-        out.writeUTF(t.columnFamily);
-    }
-
-    public Truncation deserialize(DataInputPlus in, int version) throws IOException
-    {
-        String keyspace = in.readUTF();
-        String columnFamily = in.readUTF();
-        return new Truncation(keyspace, columnFamily);
-    }
-
-    public long serializedSize(Truncation truncation, int version)
-    {
-        return TypeSizes.sizeof(truncation.keyspace) + TypeSizes.sizeof(truncation.columnFamily);
-    }
-}
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
index b7ab705..45df4eb 100644
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
@@ -30,7 +30,7 @@ import com.codahale.metrics.Timer.Context;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation;
-import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.MonotonicClock;
 import org.apache.cassandra.utils.NoSpamLogger;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
@@ -133,21 +133,21 @@ public abstract class AbstractCommitLogService
             throw new IllegalArgumentException(String.format("Commit log flush interval must be positive: %fms",
                                                              syncIntervalNanos * 1e-6));
         shutdown = false;
-        Runnable runnable = new SyncRunnable(new Clock());
+        Runnable runnable = new SyncRunnable(MonotonicClock.preciseTime);
         thread = NamedThreadFactory.createThread(runnable, name);
         thread.start();
     }
 
     class SyncRunnable implements Runnable
     {
-        private final Clock clock;
+        private final MonotonicClock clock;
         private long firstLagAt = 0;
         private long totalSyncDuration = 0; // total time spent syncing since firstLagAt
         private long syncExceededIntervalBy = 0; // time that syncs exceeded pollInterval since firstLagAt
         private int lagCount = 0;
         private int syncCount = 0;
 
-        SyncRunnable(Clock clock)
+        SyncRunnable(MonotonicClock clock)
         {
             this.clock = clock;
         }
@@ -169,7 +169,7 @@ public abstract class AbstractCommitLogService
             try
             {
                 // sync and signal
-                long pollStarted = clock.nanoTime();
+                long pollStarted = clock.now();
                 boolean flushToDisk = lastSyncedAt + syncIntervalNanos <= pollStarted || shutdownRequested || syncRequested;
                 if (flushToDisk)
                 {
@@ -186,7 +186,7 @@ public abstract class AbstractCommitLogService
                     commitLog.sync(false);
                 }
 
-                long now = clock.nanoTime();
+                long now = clock.now();
                 if (flushToDisk)
                     maybeLogFlushLag(pollStarted, now);
 
diff --git a/src/java/org/apache/cassandra/db/context/CounterContext.java b/src/java/org/apache/cassandra/db/context/CounterContext.java
index 29dc3f0..01c2f1d 100644
--- a/src/java/org/apache/cassandra/db/context/CounterContext.java
+++ b/src/java/org/apache/cassandra/db/context/CounterContext.java
@@ -629,7 +629,7 @@ public class CounterContext
 
         ByteBuffer marked = ByteBuffer.allocate(context.remaining());
         marked.putShort(marked.position(), (short) (count * -1));
-        ByteBufferUtil.arrayCopy(context,
+        ByteBufferUtil.copyBytes(context,
                                  context.position() + HEADER_SIZE_LENGTH,
                                  marked,
                                  marked.position() + HEADER_SIZE_LENGTH,
@@ -668,7 +668,7 @@ public class CounterContext
             cleared.putShort(cleared.position() + HEADER_SIZE_LENGTH + i * HEADER_ELT_LENGTH, globalShardIndexes.get(i));
 
         int origHeaderLength = headerLength(context);
-        ByteBufferUtil.arrayCopy(context,
+        ByteBufferUtil.copyBytes(context,
                                  context.position() + origHeaderLength,
                                  cleared,
                                  cleared.position() + headerLength(cleared),
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java b/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java
index ddc7b1c..d0cc514 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java
@@ -26,6 +26,7 @@ import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.exceptions.UnknownColumnException;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.schema.ColumnMetadata;
@@ -196,7 +197,7 @@ public abstract class ColumnSubselection implements Comparable<ColumnSubselectio
                 // deserialization. The column will be ignore later on anyway.
                 column = metadata.getDroppedColumn(name);
                 if (column == null)
-                    throw new RuntimeException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
+                    throw new UnknownColumnException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
             }
 
             Kind kind = Kind.values()[in.readUnsignedByte()];
diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
index ac4c69f..e3423ff 100644
--- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
@@ -360,7 +360,7 @@ public class CompositeType extends AbstractCompositeType
         {
             ByteBufferUtil.writeShortLength(out, bb.remaining());
             int toCopy = bb.remaining();
-            ByteBufferUtil.arrayCopy(bb, bb.position(), out, out.position(), toCopy);
+            ByteBufferUtil.copyBytes(bb, bb.position(), out, out.position(), toCopy);
             out.position(out.position() + toCopy);
             out.put((byte) 0);
         }
diff --git a/src/java/org/apache/cassandra/db/monitoring/ApproximateTime.java b/src/java/org/apache/cassandra/db/monitoring/ApproximateTime.java
deleted file mode 100644
index cc4b410..0000000
--- a/src/java/org/apache/cassandra/db/monitoring/ApproximateTime.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.cassandra.db.monitoring;
-
-import java.util.concurrent.TimeUnit;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.config.Config;
-
-/**
- * This is an approximation of System.currentTimeInMillis(). It updates its
- * time value at periodic intervals of CHECK_INTERVAL_MS milliseconds
- * (currently 10 milliseconds by default). It can be used as a faster alternative
- * to System.currentTimeInMillis() every time an imprecision of a few milliseconds
- * can be accepted.
- */
-public class ApproximateTime
-{
-    private static final Logger logger = LoggerFactory.getLogger(ApproximateTime.class);
-    private static final int CHECK_INTERVAL_MS = Math.max(5, Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "approximate_time_precision_ms", "10")));
-
-    private static volatile long time = System.currentTimeMillis();
-    static
-    {
-        logger.info("Scheduling approximate time-check task with a precision of {} milliseconds", CHECK_INTERVAL_MS);
-        ScheduledExecutors.scheduledFastTasks.scheduleWithFixedDelay(() -> time = System.currentTimeMillis(),
-                                                                     CHECK_INTERVAL_MS,
-                                                                     CHECK_INTERVAL_MS,
-                                                                     TimeUnit.MILLISECONDS);
-    }
-
-    public static long currentTimeMillis()
-    {
-        return time;
-    }
-
-    public static long precision()
-    {
-        return 2 * CHECK_INTERVAL_MS;
-    }
-
-}
diff --git a/src/java/org/apache/cassandra/db/monitoring/Monitorable.java b/src/java/org/apache/cassandra/db/monitoring/Monitorable.java
index c9bf94e..10bd104 100644
--- a/src/java/org/apache/cassandra/db/monitoring/Monitorable.java
+++ b/src/java/org/apache/cassandra/db/monitoring/Monitorable.java
@@ -21,9 +21,9 @@ package org.apache.cassandra.db.monitoring;
 public interface Monitorable
 {
     String name();
-    long constructionTime();
-    long timeout();
-    long slowTimeout();
+    long creationTimeNanos();
+    long timeoutNanos();
+    long slowTimeoutNanos();
 
     boolean isInProgress();
     boolean isAborted();
diff --git a/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java b/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java
index 48c8152..a6e7947 100644
--- a/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java
+++ b/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java
@@ -18,13 +18,15 @@
 
 package org.apache.cassandra.db.monitoring;
 
+import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+
 public abstract class MonitorableImpl implements Monitorable
 {
     private MonitoringState state;
     private boolean isSlow;
-    private long constructionTime = -1;
-    private long timeout;
-    private long slowTimeout;
+    private long approxCreationTimeNanos = -1;
+    private long timeoutNanos;
+    private long slowTimeoutNanos;
     private boolean isCrossNode;
 
     protected MonitorableImpl()
@@ -38,23 +40,23 @@ public abstract class MonitorableImpl implements Monitorable
      * is too complex, it would require passing new parameters to all serializers
      * or specializing the serializers to accept these message properties.
      */
-    public void setMonitoringTime(long constructionTime, boolean isCrossNode, long timeout, long slowTimeout)
+    public void setMonitoringTime(long approxCreationTimeNanos, boolean isCrossNode, long timeoutNanos, long slowTimeoutNanos)
     {
-        assert constructionTime >= 0;
-        this.constructionTime = constructionTime;
+        assert approxCreationTimeNanos >= 0;
+        this.approxCreationTimeNanos = approxCreationTimeNanos;
         this.isCrossNode = isCrossNode;
-        this.timeout = timeout;
-        this.slowTimeout = slowTimeout;
+        this.timeoutNanos = timeoutNanos;
+        this.slowTimeoutNanos = slowTimeoutNanos;
     }
 
-    public long constructionTime()
+    public long creationTimeNanos()
     {
-        return constructionTime;
+        return approxCreationTimeNanos;
     }
 
-    public long timeout()
+    public long timeoutNanos()
     {
-        return timeout;
+        return timeoutNanos;
     }
 
     public boolean isCrossNode()
@@ -62,9 +64,9 @@ public abstract class MonitorableImpl implements Monitorable
         return isCrossNode;
     }
 
-    public long slowTimeout()
+    public long slowTimeoutNanos()
     {
-        return slowTimeout;
+        return slowTimeoutNanos;
     }
 
     public boolean isInProgress()
@@ -95,8 +97,8 @@ public abstract class MonitorableImpl implements Monitorable
     {
         if (state == MonitoringState.IN_PROGRESS)
         {
-            if (constructionTime >= 0)
-                MonitoringTask.addFailedOperation(this, ApproximateTime.currentTimeMillis());
+            if (approxCreationTimeNanos >= 0)
+                MonitoringTask.addFailedOperation(this, approxTime.now());
 
             state = MonitoringState.ABORTED;
             return true;
@@ -109,8 +111,8 @@ public abstract class MonitorableImpl implements Monitorable
     {
         if (state == MonitoringState.IN_PROGRESS)
         {
-            if (isSlow && slowTimeout > 0 && constructionTime >= 0)
-                MonitoringTask.addSlowOperation(this, ApproximateTime.currentTimeMillis());
+            if (isSlow && slowTimeoutNanos > 0 && approxCreationTimeNanos >= 0)
+                MonitoringTask.addSlowOperation(this, approxTime.now());
 
             state = MonitoringState.COMPLETED;
             return true;
@@ -121,15 +123,15 @@ public abstract class MonitorableImpl implements Monitorable
 
     private void check()
     {
-        if (constructionTime < 0 || state != MonitoringState.IN_PROGRESS)
+        if (approxCreationTimeNanos < 0 || state != MonitoringState.IN_PROGRESS)
             return;
 
-        long elapsed = ApproximateTime.currentTimeMillis() - constructionTime;
+        long minElapsedNanos = (approxTime.now() - approxCreationTimeNanos) - approxTime.error();
 
-        if (elapsed >= slowTimeout && !isSlow)
+        if (minElapsedNanos >= slowTimeoutNanos && !isSlow)
             isSlow = true;
 
-        if (elapsed >= timeout)
+        if (minElapsedNanos >= timeoutNanos)
             abort();
     }
 }
diff --git a/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java b/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
index 9426042..0f8555f 100644
--- a/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
+++ b/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
@@ -39,6 +39,8 @@ import org.apache.cassandra.config.Config;
 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;
 
 /**
  * A task for monitoring in progress operations, currently only read queries, and aborting them if they time out.
@@ -68,7 +70,7 @@ class MonitoringTask
     private final ScheduledFuture<?> reportingTask;
     private final OperationsQueue failedOperationsQueue;
     private final OperationsQueue slowOperationsQueue;
-    private long lastLogTime;
+    private long approxLastLogTimeNanos;
 
 
     @VisibleForTesting
@@ -88,10 +90,10 @@ class MonitoringTask
         this.failedOperationsQueue = new OperationsQueue(maxOperations);
         this.slowOperationsQueue = new OperationsQueue(maxOperations);
 
-        this.lastLogTime = ApproximateTime.currentTimeMillis();
+        this.approxLastLogTimeNanos = approxTime.now();
 
         logger.info("Scheduling monitoring task with report interval of {} ms, max operations {}", reportIntervalMillis, maxOperations);
-        this.reportingTask = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(() -> logOperations(ApproximateTime.currentTimeMillis()),
+        this.reportingTask = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(() -> logOperations(approxTime.now()),
                                                                                      reportIntervalMillis,
                                                                                      reportIntervalMillis,
                                                                                      TimeUnit.MILLISECONDS);
@@ -102,14 +104,14 @@ class MonitoringTask
         reportingTask.cancel(false);
     }
 
-    static void addFailedOperation(Monitorable operation, long now)
+    static void addFailedOperation(Monitorable operation, long nowNanos)
     {
-        instance.failedOperationsQueue.offer(new FailedOperation(operation, now));
+        instance.failedOperationsQueue.offer(new FailedOperation(operation, nowNanos));
     }
 
-    static void addSlowOperation(Monitorable operation, long now)
+    static void addSlowOperation(Monitorable operation, long nowNanos)
     {
-        instance.slowOperationsQueue.offer(new SlowOperation(operation, now));
+        instance.slowOperationsQueue.offer(new SlowOperation(operation, nowNanos));
     }
 
     @VisibleForTesting
@@ -131,27 +133,27 @@ class MonitoringTask
     }
 
     @VisibleForTesting
-    private void logOperations(long now)
+    private void logOperations(long approxCurrentTimeNanos)
     {
-        logSlowOperations(now);
-        logFailedOperations(now);
+        logSlowOperations(approxCurrentTimeNanos);
+        logFailedOperations(approxCurrentTimeNanos);
 
-        lastLogTime = now;
+        approxLastLogTimeNanos = approxCurrentTimeNanos;
     }
 
     @VisibleForTesting
-    boolean logFailedOperations(long now)
+    boolean logFailedOperations(long nowNanos)
     {
         AggregatedOperations failedOperations = failedOperationsQueue.popOperations();
         if (!failedOperations.isEmpty())
         {
-            long elapsed = now - lastLogTime;
+            long elapsedNanos = nowNanos - approxLastLogTimeNanos;
             noSpamLogger.warn("Some operations timed out, details available at debug level (debug.log)");
 
             if (logger.isDebugEnabled())
                 logger.debug("{} operations timed out in the last {} msecs:{}{}",
                             failedOperations.num(),
-                            elapsed,
+                             NANOSECONDS.toMillis(elapsedNanos),
                             LINE_SEPARATOR,
                             failedOperations.getLogMessage());
             return true;
@@ -161,18 +163,18 @@ class MonitoringTask
     }
 
     @VisibleForTesting
-    boolean logSlowOperations(long now)
+    boolean logSlowOperations(long approxCurrentTimeNanos)
     {
         AggregatedOperations slowOperations = slowOperationsQueue.popOperations();
         if (!slowOperations.isEmpty())
         {
-            long elapsed = now - lastLogTime;
+            long approxElapsedNanos = approxCurrentTimeNanos - approxLastLogTimeNanos;
             noSpamLogger.info("Some operations were slow, details available at debug level (debug.log)");
 
             if (logger.isDebugEnabled())
                 logger.debug("{} operations were slow in the last {} msecs:{}{}",
                              slowOperations.num(),
-                             elapsed,
+                             NANOSECONDS.toMillis(approxElapsedNanos),
                              LINE_SEPARATOR,
                              slowOperations.getLogMessage());
             return true;
@@ -314,7 +316,7 @@ class MonitoringTask
         int numTimesReported;
 
         /** The total time spent by this operation */
-        long totalTime;
+        long totalTimeNanos;
 
         /** The maximum time spent by this operation */
         long maxTime;
@@ -326,13 +328,13 @@ class MonitoringTask
          * this is set lazily as it takes time to build the query CQL */
         private String name;
 
-        Operation(Monitorable operation, long failedAt)
+        Operation(Monitorable operation, long failedAtNanos)
         {
             this.operation = operation;
             numTimesReported = 1;
-            totalTime = failedAt - operation.constructionTime();
-            minTime = totalTime;
-            maxTime = totalTime;
+            totalTimeNanos = failedAtNanos - operation.creationTimeNanos();
+            minTime = totalTimeNanos;
+            maxTime = totalTimeNanos;
         }
 
         public String name()
@@ -345,7 +347,7 @@ class MonitoringTask
         void add(Operation operation)
         {
             numTimesReported++;
-            totalTime += operation.totalTime;
+            totalTimeNanos += operation.totalTimeNanos;
             maxTime = Math.max(maxTime, operation.maxTime);
             minTime = Math.min(minTime, operation.minTime);
         }
@@ -358,9 +360,9 @@ class MonitoringTask
      */
     private final static class FailedOperation extends Operation
     {
-        FailedOperation(Monitorable operation, long failedAt)
+        FailedOperation(Monitorable operation, long failedAtNanos)
         {
-            super(operation, failedAt);
+            super(operation, failedAtNanos);
         }
 
         public String getLogMessage()
@@ -368,17 +370,17 @@ class MonitoringTask
             if (numTimesReported == 1)
                 return String.format("<%s>, total time %d msec, timeout %d %s",
                                      name(),
-                                     totalTime,
-                                     operation.timeout(),
+                                     NANOSECONDS.toMillis(totalTimeNanos),
+                                     NANOSECONDS.toMillis(operation.timeoutNanos()),
                                      operation.isCrossNode() ? "msec/cross-node" : "msec");
             else
                 return String.format("<%s> timed out %d times, avg/min/max %d/%d/%d msec, timeout %d %s",
                                      name(),
                                      numTimesReported,
-                                     totalTime / numTimesReported,
-                                     minTime,
-                                     maxTime,
-                                     operation.timeout(),
+                                     NANOSECONDS.toMillis(totalTimeNanos / numTimesReported),
+                                     NANOSECONDS.toMillis(minTime),
+                                     NANOSECONDS.toMillis(maxTime),
+                                     NANOSECONDS.toMillis(operation.timeoutNanos()),
                                      operation.isCrossNode() ? "msec/cross-node" : "msec");
         }
     }
@@ -398,17 +400,17 @@ class MonitoringTask
             if (numTimesReported == 1)
                 return String.format("<%s>, time %d msec - slow timeout %d %s",
                                      name(),
-                                     totalTime,
-                                     operation.slowTimeout(),
+                                     NANOSECONDS.toMillis(totalTimeNanos),
+                                     NANOSECONDS.toMillis(operation.slowTimeoutNanos()),
                                      operation.isCrossNode() ? "msec/cross-node" : "msec");
             else
                 return String.format("<%s>, was slow %d times: avg/min/max %d/%d/%d msec - slow timeout %d %s",
                                      name(),
                                      numTimesReported,
-                                     totalTime / numTimesReported,
-                                     minTime,
-                                     maxTime,
-                                     operation.slowTimeout(),
+                                     NANOSECONDS.toMillis(totalTimeNanos/ numTimesReported),
+                                     NANOSECONDS.toMillis(minTime),
+                                     NANOSECONDS.toMillis(maxTime),
+                                     NANOSECONDS.toMillis(operation.slowTimeoutNanos()),
                                      operation.isCrossNode() ? "msec/cross-node" : "msec");
         }
     }
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java
index eb993ff..c362d11 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java
@@ -99,8 +99,9 @@ public class CassandraCompressedStreamReader extends CassandraStreamReader
                     // when compressed, report total bytes of compressed chunks read since remoteFile.size is the sum of chunks transferred
                     session.progress(filename, ProgressInfo.Direction.IN, cis.getTotalCompressedBytesRead(), totalSize);
                 }
+                assert in.getBytesRead() == sectionLength;
             }
-            logger.debug("[Stream #{}] Finished receiving file #{} from {} readBytes = {}, totalSize = {}", session.planId(), fileSeqNum,
+            logger.trace("[Stream #{}] Finished receiving file #{} from {} readBytes = {}, totalSize = {}", session.planId(), fileSeqNum,
                          session.peer, FBUtilities.prettyPrintMemory(cis.getTotalCompressedBytesRead()), FBUtilities.prettyPrintMemory(totalSize));
             return writer;
         }
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java
index 3b971f8..efbccdc 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java
@@ -31,8 +31,7 @@ import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.ChannelProxy;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
 import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.utils.FBUtilities;
@@ -55,10 +54,9 @@ public class CassandraCompressedStreamWriter extends CassandraStreamWriter
     }
 
     @Override
-    public void write(DataOutputStreamPlus out) throws IOException
+    public void write(DataOutputStreamPlus output) throws IOException
     {
-        assert out instanceof ByteBufDataOutputStreamPlus;
-        ByteBufDataOutputStreamPlus output = (ByteBufDataOutputStreamPlus)out;
+        AsyncStreamingOutputPlus out = (AsyncStreamingOutputPlus) output;
         long totalSize = totalSize();
         logger.debug("[Stream #{}] Start streaming file {} to {}, repairedAt = {}, totalSize = {}", session.planId(),
                      sstable.getFilename(), session.peer, sstable.getSSTableMetadata().repairedAt, totalSize);
@@ -76,32 +74,24 @@ public class CassandraCompressedStreamWriter extends CassandraStreamWriter
                 // length of the section to stream
                 long length = section.upperPosition - section.lowerPosition;
 
-                logger.trace("[Stream #{}] Writing section {} with length {} to stream.", session.planId(), sectionIdx++, length);
+                logger.debug("[Stream #{}] Writing section {} with length {} to stream.", session.planId(), sectionIdx++, length);
 
                 // tracks write progress
                 long bytesTransferred = 0;
                 while (bytesTransferred < length)
                 {
-                    final int toTransfer = (int) Math.min(CHUNK_SIZE, length - bytesTransferred);
-                    limiter.acquire(toTransfer);
-
-                    ByteBuffer outBuffer = ByteBuffer.allocateDirect(toTransfer);
-                    long lastWrite;
-                    try
-                    {
-                        lastWrite = fc.read(outBuffer, section.lowerPosition + bytesTransferred);
-                        assert lastWrite == toTransfer : String.format("could not read required number of bytes from file to be streamed: read %d bytes, wanted %d bytes", lastWrite, toTransfer);
+                    int toTransfer = (int) Math.min(CHUNK_SIZE, length - bytesTransferred);
+                    long position = section.lowerPosition + bytesTransferred;
+
+                    out.writeToChannel(bufferSupplier -> {
+                        ByteBuffer outBuffer = bufferSupplier.get(toTransfer);
+                        long read = fc.read(outBuffer, position);
+                        assert read == toTransfer : String.format("could not read required number of bytes from file to be streamed: read %d bytes, wanted %d bytes", read, toTransfer);
                         outBuffer.flip();
-                        output.writeToChannel(outBuffer);
-                    }
-                    catch (IOException e)
-                    {
-                        FileUtils.clean(outBuffer);
-                        throw e;
-                    }
-
-                    bytesTransferred += lastWrite;
-                    progress += lastWrite;
+                    }, limiter);
+
+                    bytesTransferred += toTransfer;
+                    progress += toTransfer;
                     session.progress(sstable.descriptor.filenameFor(Component.DATA), ProgressInfo.Direction.OUT, progress, totalSize);
                 }
             }
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java
index 7a20110..401b20e 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java
@@ -27,7 +27,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
 import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.StreamManager;
 import org.apache.cassandra.streaming.StreamSession;
@@ -58,11 +58,11 @@ public class CassandraEntireSSTableStreamWriter
     /**
      * Stream the entire file to given channel.
      * <p>
-     *
+     * TODO: this currently requires a companion thread, but could be performed entirely asynchronously
      * @param out where this writes data to
      * @throws IOException on any I/O error
      */
-    public void write(ByteBufDataOutputStreamPlus out) throws IOException
+    public void write(AsyncStreamingOutputPlus out) throws IOException
     {
         long totalSize = manifest.totalSize();
         logger.debug("[Stream #{}] Start streaming sstable {} to {}, repairedAt = {}, totalSize = {}",
@@ -76,7 +76,7 @@ public class CassandraEntireSSTableStreamWriter
 
         for (Component component : manifest.components())
         {
-            @SuppressWarnings("resource") // this is closed after the file is transferred by ByteBufDataOutputStreamPlus
+            @SuppressWarnings("resource") // this is closed after the file is transferred by AsyncChannelOutputPlus
             FileChannel in = new RandomAccessFile(sstable.descriptor.filenameFor(component), "r").getChannel();
 
             // Total Length to transmit for this file
@@ -90,7 +90,7 @@ public class CassandraEntireSSTableStreamWriter
                          component,
                          prettyPrintMemory(length));
 
-            long bytesWritten = out.writeToChannel(in, limiter);
+            long bytesWritten = out.writeFileToChannel(in, limiter);
             progress += bytesWritten;
 
             session.progress(sstable.descriptor.filenameFor(component), ProgressInfo.Direction.OUT, bytesWritten, length);
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
index c688fdf..e8f5485 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.db.streaming;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Collection;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Objects;
@@ -41,7 +40,7 @@ import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.KeyIterator;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.streaming.OutgoingStream;
 import org.apache.cassandra.streaming.StreamOperation;
@@ -164,10 +163,10 @@ public class CassandraOutgoingFile implements OutgoingStream
         CassandraStreamHeader.serializer.serialize(header, out, version);
         out.flush();
 
-        if (shouldStreamEntireSSTable() && out instanceof ByteBufDataOutputStreamPlus)
+        if (shouldStreamEntireSSTable() && out instanceof AsyncStreamingOutputPlus)
         {
             CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((ByteBufDataOutputStreamPlus) out);
+            writer.write((AsyncStreamingOutputPlus) out);
         }
         else
         {
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java
index 43371a9..190f136 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java
@@ -28,7 +28,7 @@ import com.google.common.collect.UnmodifiableIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.exceptions.UnknownColumnException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.TrackedDataInputPlus;
 import org.apache.cassandra.schema.TableId;
@@ -47,10 +47,11 @@ import org.apache.cassandra.streaming.StreamReceiver;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.streaming.compress.StreamCompressionInputStream;
 import org.apache.cassandra.streaming.messages.StreamMessageHeader;
-import org.apache.cassandra.streaming.messages.StreamMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
+import static org.apache.cassandra.net.MessagingService.current_version;
+
 /**
  * CassandraStreamReader reads from stream and writes to SSTable.
  */
@@ -114,7 +115,7 @@ public class CassandraStreamReader implements IStreamReader
 
         StreamDeserializer deserializer = null;
         SSTableMultiWriter writer = null;
-        try (StreamCompressionInputStream streamCompressionInputStream = new StreamCompressionInputStream(inputPlus, StreamMessage.CURRENT_VERSION))
+        try (StreamCompressionInputStream streamCompressionInputStream = new StreamCompressionInputStream(inputPlus, current_version))
         {
             TrackedDataInputPlus in = new TrackedDataInputPlus(streamCompressionInputStream);
             deserializer = new StreamDeserializer(cfs.metadata(), in, inputVersion, getHeader(cfs.metadata()));
@@ -142,7 +143,7 @@ public class CassandraStreamReader implements IStreamReader
         }
     }
 
-    protected SerializationHeader getHeader(TableMetadata metadata)
+    protected SerializationHeader getHeader(TableMetadata metadata) throws UnknownColumnException
     {
         return header != null? header.toHeader(metadata) : null; //pre-3.0 sstable have no SerializationHeader
     }
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java
index c6dd9a9..ffc663d 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java
@@ -25,19 +25,25 @@ import java.util.Collection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import net.jpountz.lz4.LZ4Compressor;
+import net.jpountz.lz4.LZ4Factory;
+import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.ChannelProxy;
 import org.apache.cassandra.io.util.DataIntegrityMetadata;
 import org.apache.cassandra.io.util.DataIntegrityMetadata.ChecksumValidator;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
 import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.StreamManager;
 import org.apache.cassandra.streaming.StreamManager.StreamRateLimiter;
 import org.apache.cassandra.streaming.StreamSession;
-import org.apache.cassandra.streaming.compress.ByteBufCompressionDataOutputStreamPlus;
+import org.apache.cassandra.streaming.async.StreamCompressionSerializer;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.memory.BufferPool;
+
+import static org.apache.cassandra.net.MessagingService.current_version;
 
 /**
  * CassandraStreamWriter writes given section of the SSTable to given channel.
@@ -49,6 +55,7 @@ public class CassandraStreamWriter
     private static final Logger logger = LoggerFactory.getLogger(CassandraStreamWriter.class);
 
     protected final SSTableReader sstable;
+    private final LZ4Compressor compressor = LZ4Factory.fastestInstance().fastCompressor();
     protected final Collection<SSTableReader.PartitionPositionBounds> sections;
     protected final StreamRateLimiter limiter;
     protected final StreamSession session;
@@ -75,6 +82,7 @@ public class CassandraStreamWriter
         logger.debug("[Stream #{}] Start streaming file {} to {}, repairedAt = {}, totalSize = {}", session.planId(),
                      sstable.getFilename(), session.peer, sstable.getSSTableMetadata().repairedAt, totalSize);
 
+        AsyncStreamingOutputPlus out = (AsyncStreamingOutputPlus) output;
         try(ChannelProxy proxy = sstable.getDataChannel().sharedCopy();
             ChecksumValidator validator = new File(sstable.descriptor.filenameFor(Component.CRC)).exists()
                                           ? DataIntegrityMetadata.checksumValidator(sstable.descriptor)
@@ -85,38 +93,35 @@ public class CassandraStreamWriter
             // setting up data compression stream
             long progress = 0L;
 
-            try (DataOutputStreamPlus compressedOutput = new ByteBufCompressionDataOutputStreamPlus(output, limiter))
+            // stream each of the required sections of the file
+            for (SSTableReader.PartitionPositionBounds section : sections)
             {
-                // stream each of the required sections of the file
-                for (SSTableReader.PartitionPositionBounds section : sections)
+                long start = validator == null ? section.lowerPosition : validator.chunkStart(section.lowerPosition);
+                // if the transfer does not start on the valididator's chunk boundary, this is the number of bytes to offset by
+                int transferOffset = (int) (section.lowerPosition - start);
+                if (validator != null)
+                    validator.seek(start);
+
+                // length of the section to read
+                long length = section.upperPosition - start;
+                // tracks write progress
+                long bytesRead = 0;
+                while (bytesRead < length)
                 {
-                    long start = validator == null ? section.lowerPosition : validator.chunkStart(section.lowerPosition);
-                    // if the transfer does not start on the valididator's chunk boundary, this is the number of bytes to offset by
-                    int transferOffset = (int) (section.lowerPosition - start);
-                    if (validator != null)
-                        validator.seek(start);
-
-                    // length of the section to read
-                    long length = section.upperPosition - start;
-                    // tracks write progress
-                    long bytesRead = 0;
-                    while (bytesRead < length)
-                    {
-                        int toTransfer = (int) Math.min(bufferSize, length - bytesRead);
-                        long lastBytesRead = write(proxy, validator, compressedOutput, start, transferOffset, toTransfer, bufferSize);
-                        start += lastBytesRead;
-                        bytesRead += lastBytesRead;
-                        progress += (lastBytesRead - transferOffset);
-                        session.progress(sstable.descriptor.filenameFor(Component.DATA), ProgressInfo.Direction.OUT, progress, totalSize);
-                        transferOffset = 0;
-                    }
-
-                    // make sure that current section is sent
-                    output.flush();
+                    int toTransfer = (int) Math.min(bufferSize, length - bytesRead);
+                    long lastBytesRead = write(proxy, validator, out, start, transferOffset, toTransfer, bufferSize);
+                    start += lastBytesRead;
+                    bytesRead += lastBytesRead;
+                    progress += (lastBytesRead - transferOffset);
+                    session.progress(sstable.descriptor.filenameFor(Component.DATA), ProgressInfo.Direction.OUT, progress, totalSize);
+                    transferOffset = 0;
                 }
-                logger.debug("[Stream #{}] Finished streaming file {} to {}, bytesTransferred = {}, totalSize = {}",
-                             session.planId(), sstable.getFilename(), session.peer, FBUtilities.prettyPrintMemory(progress), FBUtilities.prettyPrintMemory(totalSize));
+
+                // make sure that current section is sent
+                out.flush();
             }
+            logger.debug("[Stream #{}] Finished streaming file {} to {}, bytesTransferred = {}, totalSize = {}",
+                         session.planId(), sstable.getFilename(), session.peer, FBUtilities.prettyPrintMemory(progress), FBUtilities.prettyPrintMemory(totalSize));
         }
     }
 
@@ -141,14 +146,14 @@ public class CassandraStreamWriter
      *
      * @throws java.io.IOException on any I/O error
      */
-    protected long write(ChannelProxy proxy, ChecksumValidator validator, DataOutputStreamPlus output, long start, int transferOffset, int toTransfer, int bufferSize) throws IOException
+    protected long write(ChannelProxy proxy, ChecksumValidator validator, AsyncStreamingOutputPlus output, long start, int transferOffset, int toTransfer, int bufferSize) throws IOException
     {
         // the count of bytes to read off disk
         int minReadable = (int) Math.min(bufferSize, proxy.size() - start);
 
         // this buffer will hold the data from disk. as it will be compressed on the fly by
-        // ByteBufCompressionDataOutputStreamPlus.write(ByteBuffer), we can release this buffer as soon as we can.
-        ByteBuffer buffer = ByteBuffer.allocateDirect(minReadable);
+        // AsyncChannelCompressedStreamWriter.write(ByteBuffer), we can release this buffer as soon as we can.
+        ByteBuffer buffer = BufferPool.get(minReadable, BufferType.OFF_HEAP);
         try
         {
             int readCount = proxy.read(buffer, start);
@@ -163,11 +168,11 @@ public class CassandraStreamWriter
 
             buffer.position(transferOffset);
             buffer.limit(transferOffset + (toTransfer - transferOffset));
-            output.write(buffer);
+            output.writeToChannel(StreamCompressionSerializer.serialize(compressor, buffer, current_version), limiter);
         }
         finally
         {
-            FileUtils.clean(buffer);
+            BufferPool.put(buffer);
         }
 
         return toTransfer;
diff --git a/src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java b/src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java
new file mode 100644
index 0000000..b0afe8f
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java
@@ -0,0 +1,135 @@
+/*
+ * 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.db.virtual;
+
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.InetAddressType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.InboundMessageHandlers;
+import org.apache.cassandra.schema.TableMetadata;
+
+public final class InternodeInboundTable extends AbstractVirtualTable
+{
+    private static final String ADDRESS = "address";
+    private static final String PORT = "port";
+    private static final String DC = "dc";
+    private static final String RACK = "rack";
+
+    private static final String USING_BYTES = "using_bytes";
+    private static final String USING_RESERVE_BYTES = "using_reserve_bytes";
+    private static final String CORRUPT_FRAMES_RECOVERED = "corrupt_frames_recovered";
+    private static final String CORRUPT_FRAMES_UNRECOVERED = "corrupt_frames_unrecovered";
+    private static final String ERROR_BYTES = "error_bytes";
+    private static final String ERROR_COUNT = "error_count";
+    private static final String EXPIRED_BYTES = "expired_bytes";
+    private static final String EXPIRED_COUNT = "expired_count";
+    private static final String SCHEDULED_BYTES = "scheduled_bytes";
+    private static final String SCHEDULED_COUNT = "scheduled_count";
+    private static final String PROCESSED_BYTES = "processed_bytes";
+    private static final String PROCESSED_COUNT = "processed_count";
+    private static final String RECEIVED_BYTES = "received_bytes";
+    private static final String RECEIVED_COUNT = "received_count";
+    private static final String THROTTLED_COUNT = "throttled_count";
+    private static final String THROTTLED_NANOS = "throttled_nanos";
+
+    InternodeInboundTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "internode_inbound")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(CompositeType.getInstance(InetAddressType.instance, Int32Type.instance)))
+                           .addPartitionKeyColumn(ADDRESS, InetAddressType.instance)
+                           .addPartitionKeyColumn(PORT, Int32Type.instance)
+                           .addClusteringColumn(DC, UTF8Type.instance)
+                           .addClusteringColumn(RACK, UTF8Type.instance)
+                           .addRegularColumn(USING_BYTES, LongType.instance)
+                           .addRegularColumn(USING_RESERVE_BYTES, LongType.instance)
+                           .addRegularColumn(CORRUPT_FRAMES_RECOVERED, LongType.instance)
+                           .addRegularColumn(CORRUPT_FRAMES_UNRECOVERED, LongType.instance)
+                           .addRegularColumn(ERROR_BYTES, LongType.instance)
+                           .addRegularColumn(ERROR_COUNT, LongType.instance)
+                           .addRegularColumn(EXPIRED_BYTES, LongType.instance)
+                           .addRegularColumn(EXPIRED_COUNT, LongType.instance)
+                           .addRegularColumn(SCHEDULED_BYTES, LongType.instance)
+                           .addRegularColumn(SCHEDULED_COUNT, LongType.instance)
+                           .addRegularColumn(PROCESSED_BYTES, LongType.instance)
+                           .addRegularColumn(PROCESSED_COUNT, LongType.instance)
+                           .addRegularColumn(RECEIVED_BYTES, LongType.instance)
+                           .addRegularColumn(RECEIVED_COUNT, LongType.instance)
+                           .addRegularColumn(THROTTLED_COUNT, LongType.instance)
+                           .addRegularColumn(THROTTLED_NANOS, LongType.instance)
+                           .build());
+    }
+
+    @Override
+    public DataSet data(DecoratedKey partitionKey)
+    {
+        ByteBuffer[] addressAndPortBytes = ((CompositeType) metadata().partitionKeyType).split(partitionKey.getKey());
+        InetAddress address = InetAddressType.instance.compose(addressAndPortBytes[0]);
+        int port = Int32Type.instance.compose(addressAndPortBytes[1]);
+        InetAddressAndPort addressAndPort = InetAddressAndPort.getByAddressOverrideDefaults(address, port);
+
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        InboundMessageHandlers handlers = MessagingService.instance().messageHandlers.get(addressAndPort);
+        if (null != handlers)
+            addRow(result, addressAndPort, handlers);
+        return result;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        MessagingService.instance()
+                        .messageHandlers
+                        .forEach((addressAndPort, handlers) -> addRow(result, addressAndPort, handlers));
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, InetAddressAndPort addressAndPort, InboundMessageHandlers handlers)
+    {
+        String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(addressAndPort);
+        String rack = DatabaseDescriptor.getEndpointSnitch().getRack(addressAndPort);
+        dataSet.row(addressAndPort.address, addressAndPort.port, dc, rack)
+               .column(USING_BYTES, handlers.usingCapacity())
+               .column(USING_RESERVE_BYTES, handlers.usingEndpointReserveCapacity())
+               .column(CORRUPT_FRAMES_RECOVERED, handlers.corruptFramesRecovered())
+               .column(CORRUPT_FRAMES_UNRECOVERED, handlers.corruptFramesUnrecovered())
+               .column(ERROR_BYTES, handlers.errorBytes())
+               .column(ERROR_COUNT, handlers.errorCount())
+               .column(EXPIRED_BYTES, handlers.expiredBytes())
+               .column(EXPIRED_COUNT, handlers.expiredCount())
+               .column(SCHEDULED_BYTES, handlers.scheduledBytes())
+               .column(SCHEDULED_COUNT, handlers.scheduledCount())
+               .column(PROCESSED_BYTES, handlers.processedBytes())
+               .column(PROCESSED_COUNT, handlers.processedCount())
+               .column(RECEIVED_BYTES, handlers.receivedBytes())
+               .column(RECEIVED_COUNT, handlers.receivedCount())
+               .column(THROTTLED_COUNT, handlers.throttledCount())
+               .column(THROTTLED_NANOS, handlers.throttledNanos());
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java b/src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java
new file mode 100644
index 0000000..87b3823
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java
@@ -0,0 +1,140 @@
+/*
+ * 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.db.virtual;
+
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.function.ToLongFunction;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.InetAddressType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.OutboundConnection;
+import org.apache.cassandra.net.OutboundConnections;
+import org.apache.cassandra.schema.TableMetadata;
+
+public final class InternodeOutboundTable extends AbstractVirtualTable
+{
+    private static final String ADDRESS = "address";
+    private static final String PORT = "port";
+    private static final String DC = "dc";
+    private static final String RACK = "rack";
+
+    private static final String USING_BYTES = "using_bytes";
+    private static final String USING_RESERVE_BYTES = "using_reserve_bytes";
+    private static final String PENDING_COUNT = "pending_count";
+    private static final String PENDING_BYTES = "pending_bytes";
+    private static final String SENT_COUNT = "sent_count";
+    private static final String SENT_BYTES = "sent_bytes";
+    private static final String EXPIRED_COUNT = "expired_count";
+    private static final String EXPIRED_BYTES = "expired_bytes";
+    private static final String ERROR_COUNT = "error_count";
+    private static final String ERROR_BYTES = "error_bytes";
+    private static final String OVERLOAD_COUNT = "overload_count";
+    private static final String OVERLOAD_BYTES = "overload_bytes";
+    private static final String ACTIVE_CONNECTION_COUNT = "active_connections";
+    private static final String CONNECTION_ATTEMPTS = "connection_attempts";
+    private static final String SUCCESSFUL_CONNECTION_ATTEMPTS = "successful_connection_attempts";
+
+    InternodeOutboundTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "internode_outbound")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(CompositeType.getInstance(InetAddressType.instance, Int32Type.instance)))
+                           .addPartitionKeyColumn(ADDRESS, InetAddressType.instance)
+                           .addPartitionKeyColumn(PORT, Int32Type.instance)
+                           .addClusteringColumn(DC, UTF8Type.instance)
+                           .addClusteringColumn(RACK, UTF8Type.instance)
+                           .addRegularColumn(USING_BYTES, LongType.instance)
+                           .addRegularColumn(USING_RESERVE_BYTES, LongType.instance)
+                           .addRegularColumn(PENDING_COUNT, LongType.instance)
+                           .addRegularColumn(PENDING_BYTES, LongType.instance)
+                           .addRegularColumn(SENT_COUNT, LongType.instance)
+                           .addRegularColumn(SENT_BYTES, LongType.instance)
+                           .addRegularColumn(EXPIRED_COUNT, LongType.instance)
+                           .addRegularColumn(EXPIRED_BYTES, LongType.instance)
+                           .addRegularColumn(ERROR_COUNT, LongType.instance)
+                           .addRegularColumn(ERROR_BYTES, LongType.instance)
+                           .addRegularColumn(OVERLOAD_COUNT, LongType.instance)
+                           .addRegularColumn(OVERLOAD_BYTES, LongType.instance)
+                           .addRegularColumn(ACTIVE_CONNECTION_COUNT, LongType.instance)
+                           .addRegularColumn(CONNECTION_ATTEMPTS, LongType.instance)
+                           .addRegularColumn(SUCCESSFUL_CONNECTION_ATTEMPTS, LongType.instance)
+                           .build());
+    }
+
+    @Override
+    public DataSet data(DecoratedKey partitionKey)
+    {
+        ByteBuffer[] addressAndPortBytes = ((CompositeType) metadata().partitionKeyType).split(partitionKey.getKey());
+        InetAddress address = InetAddressType.instance.compose(addressAndPortBytes[0]);
+        int port = Int32Type.instance.compose(addressAndPortBytes[1]);
+        InetAddressAndPort addressAndPort = InetAddressAndPort.getByAddressOverrideDefaults(address, port);
+
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        OutboundConnections connections = MessagingService.instance().channelManagers.get(addressAndPort);
+        if (null != connections)
+            addRow(result, addressAndPort, connections);
+        return result;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        MessagingService.instance()
+                        .channelManagers
+                        .forEach((addressAndPort, connections) -> addRow(result, addressAndPort, connections));
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, InetAddressAndPort addressAndPort, OutboundConnections connections)
+    {
+        String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(addressAndPort);
+        String rack = DatabaseDescriptor.getEndpointSnitch().getRack(addressAndPort);
+        long pendingBytes = sum(connections, OutboundConnection::pendingBytes);
+        dataSet.row(addressAndPort.address, addressAndPort.port, dc, rack)
+               .column(USING_BYTES, pendingBytes)
+               .column(USING_RESERVE_BYTES, connections.usingReserveBytes())
+               .column(PENDING_COUNT, sum(connections, OutboundConnection::pendingCount))
+               .column(PENDING_BYTES, pendingBytes)
+               .column(SENT_COUNT, sum(connections, OutboundConnection::sentCount))
+               .column(SENT_BYTES, sum(connections, OutboundConnection::sentBytes))
+               .column(EXPIRED_COUNT, sum(connections, OutboundConnection::expiredCount))
+               .column(EXPIRED_BYTES, sum(connections, OutboundConnection::expiredBytes))
+               .column(ERROR_COUNT, sum(connections, OutboundConnection::errorCount))
+               .column(ERROR_BYTES, sum(connections, OutboundConnection::errorBytes))
+               .column(OVERLOAD_COUNT, sum(connections, OutboundConnection::overloadedCount))
+               .column(OVERLOAD_BYTES, sum(connections, OutboundConnection::overloadedBytes))
+               .column(ACTIVE_CONNECTION_COUNT, sum(connections, c -> c.isConnected() ? 1 : 0))
+               .column(CONNECTION_ATTEMPTS, sum(connections, OutboundConnection::connectionAttempts))
+               .column(SUCCESSFUL_CONNECTION_ATTEMPTS, sum(connections, OutboundConnection::successfulConnections));
+    }
+
+    private static long sum(OutboundConnections connections, ToLongFunction<OutboundConnection> f)
+    {
+        return f.applyAsLong(connections.small) + f.applyAsLong(connections.large) + f.applyAsLong(connections.urgent);
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/virtual/SettingsTable.java b/src/java/org/apache/cassandra/db/virtual/SettingsTable.java
index 34debc6..048d4ba 100644
--- a/src/java/org/apache/cassandra/db/virtual/SettingsTable.java
+++ b/src/java/org/apache/cassandra/db/virtual/SettingsTable.java
@@ -163,7 +163,7 @@ final class SettingsTable extends AbstractVirtualTable
         result.row(f.getName() + "_enabled").column(VALUE, Boolean.toString(value.enabled));
         result.row(f.getName() + "_algorithm").column(VALUE, value.algorithm);
         result.row(f.getName() + "_protocol").column(VALUE, value.protocol);
-        result.row(f.getName() + "_cipher_suites").column(VALUE, Arrays.toString(value.cipher_suites));
+        result.row(f.getName() + "_cipher_suites").column(VALUE, value.cipher_suites.toString());
         result.row(f.getName() + "_client_auth").column(VALUE, Boolean.toString(value.require_client_auth));
         result.row(f.getName() + "_endpoint_verification").column(VALUE, Boolean.toString(value.require_endpoint_verification));
         result.row(f.getName() + "_optional").column(VALUE, Boolean.toString(value.optional));
diff --git a/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java b/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java
index f85991a..fa1ddae 100644
--- a/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java
+++ b/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java
@@ -31,6 +31,8 @@ public final class SystemViewsKeyspace extends VirtualKeyspace
                                      new ClientsTable(NAME),
                                      new SettingsTable(NAME),
                                      new SSTableTasksTable(NAME),
-                                     new ThreadPoolsTable(NAME)));
+                                     new ThreadPoolsTable(NAME),
+                                     new InternodeOutboundTable(NAME),
+                                     new InternodeInboundTable(NAME)));
     }
 }
diff --git a/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java b/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java
index dc32c8c..6db0acd 100644
--- a/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java
+++ b/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.db.virtual;
 
 import java.util.Collection;
+import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableMap;
@@ -76,9 +77,9 @@ public final class VirtualMutation implements IMutation
     }
 
     @Override
-    public long getTimeout()
+    public long getTimeout(TimeUnit unit)
     {
-        return DatabaseDescriptor.getWriteRpcTimeout();
+        return DatabaseDescriptor.getWriteRpcTimeout(unit);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java
index cef605e..f5c455c 100644
--- a/src/java/org/apache/cassandra/dht/BootStrapper.java
+++ b/src/java/org/apache/cassandra/dht/BootStrapper.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.dht;
 
-import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -28,14 +27,9 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.dht.tokenallocator.TokenAllocation;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
@@ -234,24 +228,4 @@ public class BootStrapper extends ProgressEventNotifierSupport
         logger.info("Generated random tokens. tokens are {}", tokens);
         return tokens;
     }
-
-    public static class StringSerializer implements IVersionedSerializer<String>
-    {
-        public static final StringSerializer instance = new StringSerializer();
-
-        public void serialize(String s, DataOutputPlus out, int version) throws IOException
-        {
-            out.writeUTF(s);
-        }
-
-        public String deserialize(DataInputPlus in, int version) throws IOException
-        {
-            return in.readUTF();
-        }
-
-        public long serializedSize(String s, int version)
-        {
-            return TypeSizes.sizeof(s);
-        }
-    }
 }
diff --git a/src/java/org/apache/cassandra/dht/IPartitioner.java b/src/java/org/apache/cassandra/dht/IPartitioner.java
index 5475f3c..f433f20 100644
--- a/src/java/org/apache/cassandra/dht/IPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/IPartitioner.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.dht;
 
 import java.nio.ByteBuffer;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
@@ -25,9 +26,29 @@ import java.util.Random;
 
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.service.StorageService;
 
 public interface IPartitioner
 {
+    static IPartitioner global()
+    {
+        return StorageService.instance.getTokenMetadata().partitioner;
+    }
+
+    static void validate(Collection<? extends AbstractBounds<?>> allBounds)
+    {
+        for (AbstractBounds<?> bounds : allBounds)
+            validate(bounds);
+    }
+
+    static void validate(AbstractBounds<?> bounds)
+    {
+        if (global() != bounds.left.getPartitioner())
+            throw new AssertionError(String.format("Partitioner in bounds serialization. Expected %s, was %s.",
+                                                   global().getClass().getName(),
+                                                   bounds.left.getPartitioner().getClass().getName()));
+    }
+
     /**
      * Transform key to object representation of the on-disk format.
      *
diff --git a/src/java/org/apache/cassandra/exceptions/UnknownTableException.java b/src/java/org/apache/cassandra/exceptions/IncompatibleSchemaException.java
similarity index 81%
copy from src/java/org/apache/cassandra/exceptions/UnknownTableException.java
copy to src/java/org/apache/cassandra/exceptions/IncompatibleSchemaException.java
index 2cd7aab..fe3a167 100644
--- a/src/java/org/apache/cassandra/exceptions/UnknownTableException.java
+++ b/src/java/org/apache/cassandra/exceptions/IncompatibleSchemaException.java
@@ -19,15 +19,10 @@ package org.apache.cassandra.exceptions;
 
 import java.io.IOException;
 
-import org.apache.cassandra.schema.TableId;
-
-public class UnknownTableException extends IOException
+public class IncompatibleSchemaException extends IOException
 {
-    public final TableId id;
-
-    public UnknownTableException(String msg, TableId id)
+    public IncompatibleSchemaException(String msg)
     {
         super(msg);
-        this.id = id;
     }
 }
diff --git a/src/java/org/apache/cassandra/exceptions/RequestFailureException.java b/src/java/org/apache/cassandra/exceptions/RequestFailureException.java
index 2b57a75..e982b44 100644
--- a/src/java/org/apache/cassandra/exceptions/RequestFailureException.java
+++ b/src/java/org/apache/cassandra/exceptions/RequestFailureException.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.exceptions;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.stream.Collectors;
 
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.locator.InetAddressAndPort;
@@ -32,7 +33,10 @@ public class RequestFailureException extends RequestExecutionException
 
     protected RequestFailureException(ExceptionCode code, ConsistencyLevel consistency, int received, int blockFor, Map<InetAddressAndPort, RequestFailureReason> failureReasonByEndpoint)
     {
-        super(code, String.format("Operation failed - received %d responses and %d failures", received, failureReasonByEndpoint.size()));
+        super(code, String.format("Operation failed - received %d responses and %d failures: %s",
+                                  received,
+                                  failureReasonByEndpoint.size(),
+                                  buildFailureString(failureReasonByEndpoint)));
         this.consistency = consistency;
         this.received = received;
         this.blockFor = blockFor;
@@ -45,4 +49,11 @@ public class RequestFailureException extends RequestExecutionException
         // we encode this map for transport.
         this.failureReasonByEndpoint = new HashMap<>(failureReasonByEndpoint);
     }
+
+    private static String buildFailureString(Map<InetAddressAndPort, RequestFailureReason> failures)
+    {
+        return failures.entrySet().stream()
+                       .map(e -> String.format("%s from %s", e.getValue(), e.getKey()))
+                       .collect(Collectors.joining(", "));
+    }
 }
diff --git a/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java b/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java
index 96ab7b5..1cdbdb5 100644
--- a/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java
+++ b/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java
@@ -15,37 +15,101 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.cassandra.exceptions;
 
+import java.io.IOException;
+
+import com.google.common.primitives.Ints;
+
+import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.vint.VIntCoding;
+
+import static java.lang.Math.max;
+import static org.apache.cassandra.net.MessagingService.VERSION_40;
+
 public enum RequestFailureReason
 {
-    /**
-     * The reason for the failure was none of the below reasons or was not recorded by the data node.
-     */
-    UNKNOWN                  (0x0000),
+    UNKNOWN                  (0),
+    READ_TOO_MANY_TOMBSTONES (1),
+    TIMEOUT                  (2),
+    INCOMPATIBLE_SCHEMA      (3);
 
-    /**
-     * The data node read too many tombstones when attempting to execute a read query (see tombstone_failure_threshold).
-     */
-    READ_TOO_MANY_TOMBSTONES (0x0001);
+    public static final Serializer serializer = new Serializer();
 
-    /** The code to be serialized as an unsigned 16 bit integer */
     public final int code;
-    public static final RequestFailureReason[] VALUES = values();
 
-    RequestFailureReason(final int code)
+    RequestFailureReason(int code)
     {
         this.code = code;
     }
 
-    public static RequestFailureReason fromCode(final int code)
+    private static final RequestFailureReason[] codeToReasonMap;
+
+    static
     {
-        for (RequestFailureReason reasonCode : VALUES)
+        RequestFailureReason[] reasons = values();
+
+        int max = -1;
+        for (RequestFailureReason r : reasons)
+            max = max(r.code, max);
+
+        RequestFailureReason[] codeMap = new RequestFailureReason[max + 1];
+
+        for (RequestFailureReason reason : reasons)
+        {
+            if (codeMap[reason.code] != null)
+                throw new RuntimeException("Two RequestFailureReason-s that map to the same code: " + reason.code);
+            codeMap[reason.code] = reason;
+        }
+
+        codeToReasonMap = codeMap;
+    }
+
+    public static RequestFailureReason fromCode(int code)
+    {
+        if (code < 0)
+            throw new IllegalArgumentException("RequestFailureReason code must be non-negative (got " + code + ')');
+
+        // be forgiving and return UNKNOWN if we aren't aware of the code - for forward compatibility
+        return code < codeToReasonMap.length ? codeToReasonMap[code] : UNKNOWN;
+    }
+
+    public static RequestFailureReason forException(Throwable t)
+    {
+        if (t instanceof TombstoneOverwhelmingException)
+            return READ_TOO_MANY_TOMBSTONES;
+
+        if (t instanceof IncompatibleSchemaException)
+            return INCOMPATIBLE_SCHEMA;
+
+        return UNKNOWN;
+    }
+
+    public static final class Serializer implements IVersionedSerializer<RequestFailureReason>
+    {
+        private Serializer()
+        {
+        }
+
+        public void serialize(RequestFailureReason reason, DataOutputPlus out, int version) throws IOException
+        {
+            if (version < VERSION_40)
+                out.writeShort(reason.code);
+            else
+                out.writeUnsignedVInt(reason.code);
+        }
+
+        public RequestFailureReason deserialize(DataInputPlus in, int version) throws IOException
+        {
+            return fromCode(version < VERSION_40 ? in.readUnsignedShort() : Ints.checkedCast(in.readUnsignedVInt()));
+        }
+
+        public long serializedSize(RequestFailureReason reason, int version)
         {
-            if (reasonCode.code == code)
-                return reasonCode;
+            return version < VERSION_40 ? 2 : VIntCoding.computeVIntSize(reason.code);
         }
-        throw new IllegalArgumentException("Unknown request failure reason error code: " + code);
     }
 }
diff --git a/src/java/org/apache/cassandra/locator/ILatencySubscriber.java b/src/java/org/apache/cassandra/exceptions/UnknownColumnException.java
similarity index 80%
copy from src/java/org/apache/cassandra/locator/ILatencySubscriber.java
copy to src/java/org/apache/cassandra/exceptions/UnknownColumnException.java
index f6c1c7f..93a464e 100644
--- a/src/java/org/apache/cassandra/locator/ILatencySubscriber.java
+++ b/src/java/org/apache/cassandra/exceptions/UnknownColumnException.java
@@ -15,9 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.cassandra.locator;
+package org.apache.cassandra.exceptions;
 
-public interface ILatencySubscriber
+public final class UnknownColumnException extends IncompatibleSchemaException
 {
-    public void receiveTiming(InetAddressAndPort address, long latency);
+    public UnknownColumnException(String msg)
+    {
+        super(msg);
+    }
 }
diff --git a/src/java/org/apache/cassandra/exceptions/UnknownTableException.java b/src/java/org/apache/cassandra/exceptions/UnknownTableException.java
index 2cd7aab..3e9c775 100644
--- a/src/java/org/apache/cassandra/exceptions/UnknownTableException.java
+++ b/src/java/org/apache/cassandra/exceptions/UnknownTableException.java
@@ -17,11 +17,9 @@
  */
 package org.apache.cassandra.exceptions;
 
-import java.io.IOException;
-
 import org.apache.cassandra.schema.TableId;
 
-public class UnknownTableException extends IOException
+public class UnknownTableException extends IncompatibleSchemaException
 {
     public final TableId id;
 
diff --git a/src/java/org/apache/cassandra/gms/EchoMessage.java b/src/java/org/apache/cassandra/gms/EchoMessage.java
deleted file mode 100644
index 2fee889..0000000
--- a/src/java/org/apache/cassandra/gms/EchoMessage.java
+++ /dev/null
@@ -1,56 +0,0 @@
-package org.apache.cassandra.gms;
-/*
- *
- * 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.
- *
- */
-
-
-import java.io.IOException;
-
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
-
-public final class EchoMessage
-{
-	public static final EchoMessage instance = new EchoMessage();
-
-    public static final IVersionedSerializer<EchoMessage> serializer = new EchoMessageSerializer();
-
-	private EchoMessage()
-	{
-	}
-
-    public static class EchoMessageSerializer implements IVersionedSerializer<EchoMessage>
-    {
-        public void serialize(EchoMessage t, DataOutputPlus out, int version) throws IOException
-        {
-        }
-
-        public EchoMessage deserialize(DataInputPlus in, int version) throws IOException
-        {
-            return EchoMessage.instance;
-        }
-
-        public long serializedSize(EchoMessage t, int version)
-        {
-            return 0;
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/gms/FailureDetector.java b/src/java/org/apache/cassandra/gms/FailureDetector.java
index 4a16f2a..d3a5f34 100644
--- a/src/java/org/apache/cassandra/gms/FailureDetector.java
+++ b/src/java/org/apache/cassandra/gms/FailureDetector.java
@@ -37,10 +37,11 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.utils.Clock;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.MBeanWrapper;
 
+import static org.apache.cassandra.utils.MonotonicClock.preciseTime;
+
 /**
  * This FailureDetector is an implementation of the paper titled
  * "The Phi Accrual Failure Detector" by Hayashibara.
@@ -55,7 +56,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
     private static final int DEBUG_PERCENTAGE = 80; // if the phi is larger than this percentage of the max, log a debug message
     private static final long DEFAULT_MAX_PAUSE = 5000L * 1000000L; // 5 seconds
     private static final long MAX_LOCAL_PAUSE_IN_NANOS = getMaxLocalPause();
-    private long lastInterpret = Clock.instance.nanoTime();
+    private long lastInterpret = preciseTime.now();
     private long lastPause = 0L;
 
     private static long getMaxLocalPause()
@@ -283,7 +284,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
 
     public void report(InetAddressAndPort ep)
     {
-        long now = Clock.instance.nanoTime();
+        long now = preciseTime.now();
         ArrivalWindow heartbeatWindow = arrivalSamples.get(ep);
         if (heartbeatWindow == null)
         {
@@ -310,7 +311,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
         {
             return;
         }
-        long now = Clock.instance.nanoTime();
+        long now = preciseTime.now();
         long diff = now - lastInterpret;
         lastInterpret = now;
         if (diff > MAX_LOCAL_PAUSE_IN_NANOS)
@@ -319,7 +320,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
             lastPause = now;
             return;
         }
-        if (Clock.instance.nanoTime() - lastPause < MAX_LOCAL_PAUSE_IN_NANOS)
+        if (preciseTime.now() - lastPause < MAX_LOCAL_PAUSE_IN_NANOS)
         {
             logger.debug("Still not marking nodes down due to local pause");
             return;
diff --git a/src/java/org/apache/cassandra/gms/GossipDigest.java b/src/java/org/apache/cassandra/gms/GossipDigest.java
index c7e60c4..53f6c5c 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigest.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigest.java
@@ -24,7 +24,8 @@ import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.net.CompactEndpointSerializationHelper;
+
+import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer;
 
 /**
  * Contains information about a specified list of Endpoints and the largest version
@@ -83,14 +84,14 @@ class GossipDigestSerializer implements IVersionedSerializer<GossipDigest>
 {
     public void serialize(GossipDigest gDigest, DataOutputPlus out, int version) throws IOException
     {
-        CompactEndpointSerializationHelper.instance.serialize(gDigest.endpoint, out, version);
+        inetAddressAndPortSerializer.serialize(gDigest.endpoint, out, version);
         out.writeInt(gDigest.generation);
         out.writeInt(gDigest.maxVersion);
     }
 
     public GossipDigest deserialize(DataInputPlus in, int version) throws IOException
     {
-        InetAddressAndPort endpoint = CompactEndpointSerializationHelper.instance.deserialize(in, version);
+        InetAddressAndPort endpoint = inetAddressAndPortSerializer.deserialize(in, version);
         int generation = in.readInt();
         int maxVersion = in.readInt();
         return new GossipDigest(endpoint, generation, maxVersion);
@@ -98,7 +99,7 @@ class GossipDigestSerializer implements IVersionedSerializer<GossipDigest>
 
     public long serializedSize(GossipDigest gDigest, int version)
     {
-        long size = CompactEndpointSerializationHelper.instance.serializedSize(gDigest.endpoint, version);
+        long size = inetAddressAndPortSerializer.serializedSize(gDigest.endpoint, version);
         size += TypeSizes.sizeof(gDigest.generation);
         size += TypeSizes.sizeof(gDigest.maxVersion);
         return size;
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAck.java b/src/java/org/apache/cassandra/gms/GossipDigestAck.java
index a7d5b92..26494ea 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestAck.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestAck.java
@@ -27,7 +27,8 @@ import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.net.CompactEndpointSerializationHelper;
+
+import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer;
 
 /**
  * This ack gets sent out as a result of the receipt of a GossipDigestSynMessage by an
@@ -66,7 +67,7 @@ class GossipDigestAckSerializer implements IVersionedSerializer<GossipDigestAck>
         for (Map.Entry<InetAddressAndPort, EndpointState> entry : gDigestAckMessage.epStateMap.entrySet())
         {
             InetAddressAndPort ep = entry.getKey();
-            CompactEndpointSerializationHelper.instance.serialize(ep, out, version);
+            inetAddressAndPortSerializer.serialize(ep, out, version);
             EndpointState.serializer.serialize(entry.getValue(), out, version);
         }
     }
@@ -79,7 +80,7 @@ class GossipDigestAckSerializer implements IVersionedSerializer<GossipDigestAck>
 
         for (int i = 0; i < size; ++i)
         {
-            InetAddressAndPort ep = CompactEndpointSerializationHelper.instance.deserialize(in, version);
+            InetAddressAndPort ep = inetAddressAndPortSerializer.deserialize(in, version);
             EndpointState epState = EndpointState.serializer.deserialize(in, version);
             epStateMap.put(ep, epState);
         }
@@ -91,7 +92,7 @@ class GossipDigestAckSerializer implements IVersionedSerializer<GossipDigestAck>
         int size = GossipDigestSerializationHelper.serializedSize(ack.gDigestList, version);
         size += TypeSizes.sizeof(ack.epStateMap.size());
         for (Map.Entry<InetAddressAndPort, EndpointState> entry : ack.epStateMap.entrySet())
-            size += CompactEndpointSerializationHelper.instance.serializedSize(entry.getKey(), version)
+            size += inetAddressAndPortSerializer.serializedSize(entry.getKey(), version)
                     + EndpointState.serializer.serializedSize(entry.getValue(), version);
         return size;
     }
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAck2.java b/src/java/org/apache/cassandra/gms/GossipDigestAck2.java
index a6d1d2b..0e4062b 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestAck2.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestAck2.java
@@ -26,7 +26,8 @@ import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.net.CompactEndpointSerializationHelper;
+
+import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer;
 
 /**
  * This ack gets sent out as a result of the receipt of a GossipDigestAckMessage. This the
@@ -57,7 +58,7 @@ class GossipDigestAck2Serializer implements IVersionedSerializer<GossipDigestAck
         for (Map.Entry<InetAddressAndPort, EndpointState> entry : ack2.epStateMap.entrySet())
         {
             InetAddressAndPort ep = entry.getKey();
-            CompactEndpointSerializationHelper.instance.serialize(ep, out, version);
+            inetAddressAndPortSerializer.serialize(ep, out, version);
             EndpointState.serializer.serialize(entry.getValue(), out, version);
         }
     }
@@ -69,7 +70,7 @@ class GossipDigestAck2Serializer implements IVersionedSerializer<GossipDigestAck
 
         for (int i = 0; i < size; ++i)
         {
-            InetAddressAndPort ep = CompactEndpointSerializationHelper.instance.deserialize(in, version);
+            InetAddressAndPort ep = inetAddressAndPortSerializer.deserialize(in, version);
             EndpointState epState = EndpointState.serializer.deserialize(in, version);
             epStateMap.put(ep, epState);
         }
@@ -80,7 +81,7 @@ class GossipDigestAck2Serializer implements IVersionedSerializer<GossipDigestAck
     {
         long size = TypeSizes.sizeof(ack2.epStateMap.size());
         for (Map.Entry<InetAddressAndPort, EndpointState> entry : ack2.epStateMap.entrySet())
-            size += CompactEndpointSerializationHelper.instance.serializedSize(entry.getKey(), version)
+            size += inetAddressAndPortSerializer.serializedSize(entry.getKey(), version)
                     + EndpointState.serializer.serializedSize(entry.getValue(), version);
         return size;
     }
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAck2VerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestAck2VerbHandler.java
index fd5d487..58c1589 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestAck2VerbHandler.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestAck2VerbHandler.java
@@ -23,18 +23,19 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
 
-public class GossipDigestAck2VerbHandler implements IVerbHandler<GossipDigestAck2>
+public class GossipDigestAck2VerbHandler extends GossipVerbHandler<GossipDigestAck2>
 {
+    public static final GossipDigestAck2VerbHandler instance = new GossipDigestAck2VerbHandler();
+
     private static final Logger logger = LoggerFactory.getLogger(GossipDigestAck2VerbHandler.class);
 
-    public void doVerb(MessageIn<GossipDigestAck2> message, int id)
+    public void doVerb(Message<GossipDigestAck2> message)
     {
         if (logger.isTraceEnabled())
         {
-            InetAddressAndPort from = message.from;
+            InetAddressAndPort from = message.from();
             logger.trace("Received a GossipDigestAck2Message from {}", from);
         }
         if (!Gossiper.instance.isEnabled())
@@ -47,5 +48,7 @@ public class GossipDigestAck2VerbHandler implements IVerbHandler<GossipDigestAck
         /* Notify the Failure Detector */
         Gossiper.instance.notifyFailureDetector(remoteEpStateMap);
         Gossiper.instance.applyStateLocally(remoteEpStateMap);
+
+        super.doVerb(message);
     }
 }
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java
index 2a12b7c..1e8604b 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java
@@ -25,18 +25,20 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
 
-public class GossipDigestAckVerbHandler implements IVerbHandler<GossipDigestAck>
+import static org.apache.cassandra.net.Verb.GOSSIP_DIGEST_ACK2;
+
+public class GossipDigestAckVerbHandler extends GossipVerbHandler<GossipDigestAck>
 {
+    public static final GossipDigestAckVerbHandler instance = new GossipDigestAckVerbHandler();
+
     private static final Logger logger = LoggerFactory.getLogger(GossipDigestAckVerbHandler.class);
 
-    public void doVerb(MessageIn<GossipDigestAck> message, int id)
+    public void doVerb(Message<GossipDigestAck> message)
     {
-        InetAddressAndPort from = message.from;
+        InetAddressAndPort from = message.from();
         if (logger.isTraceEnabled())
             logger.trace("Received a GossipDigestAckMessage from {}", from);
         if (!Gossiper.instance.isEnabled() && !Gossiper.instance.isInShadowRound())
@@ -88,11 +90,11 @@ public class GossipDigestAckVerbHandler implements IVerbHandler<GossipDigestAck>
                 deltaEpStateMap.put(addr, localEpStatePtr);
         }
 
-        MessageOut<GossipDigestAck2> gDigestAck2Message = new MessageOut<GossipDigestAck2>(MessagingService.Verb.GOSSIP_DIGEST_ACK2,
-                                                                                           new GossipDigestAck2(deltaEpStateMap),
-                                                                                           GossipDigestAck2.serializer);
+        Message<GossipDigestAck2> gDigestAck2Message = Message.out(GOSSIP_DIGEST_ACK2, new GossipDigestAck2(deltaEpStateMap));
         if (logger.isTraceEnabled())
             logger.trace("Sending a GossipDigestAck2Message to {}", from);
-        MessagingService.instance().sendOneWay(gDigestAck2Message, from);
+        MessagingService.instance().send(gDigestAck2Message, from);
+
+        super.doVerb(message);
     }
 }
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java
index b06c24d..520dbec 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java
@@ -24,18 +24,20 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
 
-public class GossipDigestSynVerbHandler implements IVerbHandler<GossipDigestSyn>
+import static org.apache.cassandra.net.Verb.*;
+
+public class GossipDigestSynVerbHandler extends GossipVerbHandler<GossipDigestSyn>
 {
+    public static final GossipDigestSynVerbHandler instance = new GossipDigestSynVerbHandler();
+
     private static final Logger logger = LoggerFactory.getLogger(GossipDigestSynVerbHandler.class);
 
-    public void doVerb(MessageIn<GossipDigestSyn> message, int id)
+    public void doVerb(Message<GossipDigestSyn> message)
     {
-        InetAddressAndPort from = message.from;
+        InetAddressAndPort from = message.from();
         if (logger.isTraceEnabled())
             logger.trace("Received a GossipDigestSynMessage from {}", from);
         if (!Gossiper.instance.isEnabled() && !Gossiper.instance.isInShadowRound())
@@ -79,10 +81,8 @@ public class GossipDigestSynVerbHandler implements IVerbHandler<GossipDigestSyn>
             logger.debug("Received a shadow round syn from {}. Gossip is disabled but " +
                          "currently also in shadow round, responding with a minimal ack", from);
             MessagingService.instance()
-                            .sendOneWay(new MessageOut<>(MessagingService.Verb.GOSSIP_DIGEST_ACK,
-                                                         new GossipDigestAck(new ArrayList<>(), new HashMap<>()),
-                                                         GossipDigestAck.serializer),
-                                        from);
+                            .send(Message.out(GOSSIP_DIGEST_ACK, new GossipDigestAck(Collections.emptyList(), Collections.emptyMap())),
+                                  from);
             return;
         }
 
@@ -101,11 +101,11 @@ public class GossipDigestSynVerbHandler implements IVerbHandler<GossipDigestSyn>
         Map<InetAddressAndPort, EndpointState> deltaEpStateMap = new HashMap<InetAddressAndPort, EndpointState>();
         Gossiper.instance.examineGossiper(gDigestList, deltaGossipDigestList, deltaEpStateMap);
         logger.trace("sending {} digests and {} deltas", deltaGossipDigestList.size(), deltaEpStateMap.size());
-        MessageOut<GossipDigestAck> gDigestAckMessage = new MessageOut<GossipDigestAck>(MessagingService.Verb.GOSSIP_DIGEST_ACK,
-                                                                                        new GossipDigestAck(deltaGossipDigestList, deltaEpStateMap),
-                                                                                        GossipDigestAck.serializer);
+        Message<GossipDigestAck> gDigestAckMessage = Message.out(GOSSIP_DIGEST_ACK, new GossipDigestAck(deltaGossipDigestList, deltaEpStateMap));
         if (logger.isTraceEnabled())
             logger.trace("Sending a GossipDigestAckMessage to {}", from);
-        MessagingService.instance().sendOneWay(gDigestAckMessage, from);
+        MessagingService.instance().send(gDigestAckMessage, from);
+
+        super.doVerb(message);
     }
 }
diff --git a/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java
index 1691107..83c8568 100644
--- a/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java
+++ b/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java
@@ -18,23 +18,25 @@
 package org.apache.cassandra.gms;
 
 import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class GossipShutdownVerbHandler implements IVerbHandler
 {
+    public static final GossipShutdownVerbHandler instance = new GossipShutdownVerbHandler();
+
     private static final Logger logger = LoggerFactory.getLogger(GossipShutdownVerbHandler.class);
 
-    public void doVerb(MessageIn message, int id)
+    public void doVerb(Message message)
     {
         if (!Gossiper.instance.isEnabled())
         {
-            logger.debug("Ignoring shutdown message from {} because gossip is disabled", message.from);
+            logger.debug("Ignoring shutdown message from {} because gossip is disabled", message.from());
             return;
         }
-        Gossiper.instance.markAsShutdown(message.from);
+        Gossiper.instance.markAsShutdown(message.from());
     }
 
 }
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipVerbHandler.java
similarity index 75%
copy from src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java
copy to src/java/org/apache/cassandra/gms/GossipVerbHandler.java
index 3c3fcec..02aeaf4 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java
+++ b/src/java/org/apache/cassandra/gms/GossipVerbHandler.java
@@ -15,17 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.cassandra.batchlog;
 
-import java.util.UUID;
+package org.apache.cassandra.gms;
 
 import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
 
-public final class BatchRemoveVerbHandler implements IVerbHandler<UUID>
+public class GossipVerbHandler<T> implements IVerbHandler<T>
 {
-    public void doVerb(MessageIn<UUID> message, int id)
+    public void doVerb(Message<T> message)
     {
-        BatchlogManager.remove(message.payload);
+        Gossiper.instance.setLastProcessedMessageAt(message.creationTimeMillis());
     }
 }
diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java
index 8955bf9..a6c9be7 100644
--- a/src/java/org/apache/cassandra/gms/Gossiper.java
+++ b/src/java/org/apache/cassandra/gms/Gossiper.java
@@ -39,6 +39,8 @@ import com.google.common.util.concurrent.ListenableFutureTask;
 import com.google.common.util.concurrent.Uninterruptibles;
 
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.NoPayload;
+import org.apache.cassandra.net.Verb;
 import org.apache.cassandra.utils.CassandraVersion;
 import io.netty.util.concurrent.FastThreadLocal;
 import org.apache.cassandra.utils.MBeanWrapper;
@@ -53,14 +55,17 @@ import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.net.IAsyncCallback;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.RequestCallback;
+import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
+import static org.apache.cassandra.net.NoPayload.noPayload;
+import static org.apache.cassandra.net.Verb.ECHO_REQ;
+import static org.apache.cassandra.net.Verb.GOSSIP_DIGEST_SYN;
+
 /**
  * This module is responsible for Gossiping information for the local endpoint. This abstraction
  * maintains the list of live and dead endpoints. Periodically i.e. every 1 second this module
@@ -236,9 +241,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
                     GossipDigestSyn digestSynMessage = new GossipDigestSyn(DatabaseDescriptor.getClusterName(),
                                                                            DatabaseDescriptor.getPartitionerName(),
                                                                            gDigests);
-                    MessageOut<GossipDigestSyn> message = new MessageOut<GossipDigestSyn>(MessagingService.Verb.GOSSIP_DIGEST_SYN,
-                                                                                          digestSynMessage,
-                                                                                          GossipDigestSyn.serializer);
+                    Message<GossipDigestSyn> message = Message.out(GOSSIP_DIGEST_SYN, digestSynMessage);
                     /* Gossip to some random live member */
                     boolean gossipedToSeed = doGossipToLiveMember(message);
 
@@ -545,11 +548,11 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
 
         liveEndpoints.remove(endpoint);
         unreachableEndpoints.remove(endpoint);
-        MessagingService.instance().resetVersion(endpoint);
+        MessagingService.instance().versions.reset(endpoint);
         quarantineEndpoint(endpoint);
-        MessagingService.instance().destroyConnectionPool(endpoint);
-        if (logger.isDebugEnabled())
-            logger.debug("removing endpoint {}", endpoint);
+        MessagingService.instance().closeOutbound(endpoint);
+        MessagingService.instance().removeInbound(endpoint);
+        logger.debug("removing endpoint {}", endpoint);
         GossiperDiagnostics.removedEndpoint(this, endpoint);
     }
 
@@ -777,7 +780,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
      * @param epSet   a set of endpoint from which a random endpoint is chosen.
      * @return true if the chosen endpoint is also a seed.
      */
-    private boolean sendGossip(MessageOut<GossipDigestSyn> message, Set<InetAddressAndPort> epSet)
+    private boolean sendGossip(Message<GossipDigestSyn> message, Set<InetAddressAndPort> epSet)
     {
         List<InetAddressAndPort> liveEndpoints = ImmutableList.copyOf(epSet);
 
@@ -791,7 +794,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
             logger.trace("Sending a GossipDigestSyn to {} ...", to);
         if (firstSynSendAt == 0)
             firstSynSendAt = System.nanoTime();
-        MessagingService.instance().sendOneWay(message, to);
+        MessagingService.instance().send(message, to);
 
         boolean isSeed = seeds.contains(to);
         GossiperDiagnostics.sendGossipDigestSyn(this, to);
@@ -799,7 +802,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     }
 
     /* Sends a Gossip message to a live member and returns true if the recipient was a seed */
-    private boolean doGossipToLiveMember(MessageOut<GossipDigestSyn> message)
+    private boolean doGossipToLiveMember(Message<GossipDigestSyn> message)
     {
         int size = liveEndpoints.size();
         if (size == 0)
@@ -808,7 +811,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     }
 
     /* Sends a Gossip message to an unreachable member */
-    private void maybeGossipToUnreachableMember(MessageOut<GossipDigestSyn> message)
+    private void maybeGossipToUnreachableMember(Message<GossipDigestSyn> message)
     {
         double liveEndpointCount = liveEndpoints.size();
         double unreachableEndpointCount = unreachableEndpoints.size();
@@ -823,7 +826,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     }
 
     /* Possibly gossip to a seed for facilitating partition healing */
-    private void maybeGossipToSeed(MessageOut<GossipDigestSyn> prod)
+    private void maybeGossipToSeed(Message<GossipDigestSyn> prod)
     {
         int size = seeds.size();
         if (size > 0)
@@ -1145,23 +1148,15 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
     {
         localState.markDead();
 
-        MessageOut<EchoMessage> echoMessage = new MessageOut<EchoMessage>(MessagingService.Verb.ECHO, EchoMessage.instance, EchoMessage.serializer);
-        logger.trace("Sending a EchoMessage to {}", addr);
-        IAsyncCallback echoHandler = new IAsyncCallback()
+        Message<NoPayload> echoMessage = Message.out(ECHO_REQ, noPayload);
+        logger.trace("Sending ECHO_REQ to {}", addr);
+        RequestCallback echoHandler = msg ->
         {
-            public boolean isLatencyForSnitch()
-            {
-                return false;
-            }
-
-            public void response(MessageIn msg)
-            {
-                // force processing of the echo response onto the gossip stage, as it comes in on the REQUEST_RESPONSE stage
-                runInGossipStageBlocking(() -> realMarkAlive(addr, localState));
-            }
+            // force processing of the echo response onto the gossip stage, as it comes in on the REQUEST_RESPONSE stage
+            runInGossipStageBlocking(() -> realMarkAlive(addr, localState));
         };
 
-        MessagingService.instance().sendRR(echoMessage, addr, echoHandler);
+        MessagingService.instance().sendWithCallback(echoMessage, addr, echoHandler);
 
         GossiperDiagnostics.markedAlive(this, addr, localState);
     }
@@ -1447,7 +1442,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         if (gDigestList.size() == 0)
         {
            /* we've been sent a *completely* empty syn, which should normally never happen since an endpoint will at least send a syn with itself.
-              If this is happening then the node is attempting shadow gossip, and we should reply with everything we know.
+              If this is happening then the node is attempting shadow gossip, and we should respond with everything we know.
             */
             logger.debug("Shadow request received, adding all states");
             for (Map.Entry<InetAddressAndPort, EndpointState> entry : endpointStateMap.entrySet())
@@ -1582,9 +1577,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         GossipDigestSyn digestSynMessage = new GossipDigestSyn(DatabaseDescriptor.getClusterName(),
                 DatabaseDescriptor.getPartitionerName(),
                 gDigests);
-        MessageOut<GossipDigestSyn> message = new MessageOut<GossipDigestSyn>(MessagingService.Verb.GOSSIP_DIGEST_SYN,
-                digestSynMessage,
-                GossipDigestSyn.serializer);
+        Message<GossipDigestSyn> message = Message.out(GOSSIP_DIGEST_SYN, digestSynMessage);
 
         inShadowRound = true;
         boolean includePeers = false;
@@ -1598,14 +1591,14 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
                     logger.trace("Sending shadow round GOSSIP DIGEST SYN to seeds {}", seeds);
 
                     for (InetAddressAndPort seed : seeds)
-                        MessagingService.instance().sendOneWay(message, seed);
+                        MessagingService.instance().send(message, seed);
 
                     // Send to any peers we already know about, but only if a seed didn't respond.
                     if (includePeers)
                     {
                         logger.trace("Sending shadow round GOSSIP DIGEST SYN to known peers {}", peers);
                         for (InetAddressAndPort peer : peers)
-                            MessagingService.instance().sendOneWay(message, peer);
+                            MessagingService.instance().send(message, peer);
                     }
                     includePeers = true;
                 }
@@ -1800,9 +1793,9 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
             logger.info("Announcing shutdown");
             addLocalApplicationState(ApplicationState.STATUS_WITH_PORT, StorageService.instance.valueFactory.shutdown(true));
             addLocalApplicationState(ApplicationState.STATUS, StorageService.instance.valueFactory.shutdown(true));
-            MessageOut message = new MessageOut(MessagingService.Verb.GOSSIP_SHUTDOWN);
+            Message message = Message.out(Verb.GOSSIP_SHUTDOWN, noPayload);
             for (InetAddressAndPort ep : liveEndpoints)
-                MessagingService.instance().sendOneWay(message, ep);
+                MessagingService.instance().send(message, ep);
             Uninterruptibles.sleepUninterruptibly(Integer.getInteger("cassandra.shutdown_announce_in_ms", 2000), TimeUnit.MILLISECONDS);
         }
         else
diff --git a/src/java/org/apache/cassandra/hints/EncodedHintMessage.java b/src/java/org/apache/cassandra/hints/EncodedHintMessage.java
deleted file mode 100644
index 50d1302..0000000
--- a/src/java/org/apache/cassandra/hints/EncodedHintMessage.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.hints;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.UUID;
-
-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.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.UUIDSerializer;
-
-/**
- * A specialized version of {@link HintMessage} that takes an already encoded in a bytebuffer hint and sends it verbatim.
- *
- * An optimization for when dispatching a hint file of the current messaging version to a node of the same messaging version,
- * which is the most common case. Saves on extra ByteBuffer allocations one redundant hint deserialization-serialization cycle.
- *
- * Never deserialized as an EncodedHintMessage - the receiving side will always deserialize the message as vanilla
- * {@link HintMessage}.
- */
-final class EncodedHintMessage
-{
-    private static final IVersionedSerializer<EncodedHintMessage> serializer = new Serializer();
-
-    private final UUID hostId;
-    private final ByteBuffer hint;
-    private final int version;
-
-    EncodedHintMessage(UUID hostId, ByteBuffer hint, int version)
-    {
-        this.hostId = hostId;
-        this.hint = hint;
-        this.version = version;
-    }
-
-    MessageOut<EncodedHintMessage> createMessageOut()
-    {
-        return new MessageOut<>(MessagingService.Verb.HINT, this, serializer);
-    }
-
-    public long getHintCreationTime()
-    {
-        return Hint.serializer.getHintCreationTime(hint, version);
-    }
-
-    private static class Serializer implements IVersionedSerializer<EncodedHintMessage>
-    {
-        public long serializedSize(EncodedHintMessage message, int version)
-        {
-            if (version != message.version)
-                throw new IllegalArgumentException("serializedSize() called with non-matching version " + version);
-
-            long size = UUIDSerializer.serializer.serializedSize(message.hostId, version);
-            size += TypeSizes.sizeofUnsignedVInt(message.hint.remaining());
-            size += message.hint.remaining();
-            return size;
-        }
-
-        public void serialize(EncodedHintMessage message, DataOutputPlus out, int version) throws IOException
-        {
-            if (version != message.version)
-                throw new IllegalArgumentException("serialize() called with non-matching version " + version);
-
-            UUIDSerializer.serializer.serialize(message.hostId, out, version);
-            out.writeUnsignedVInt(message.hint.remaining());
-            out.write(message.hint);
-        }
-
-        public EncodedHintMessage deserialize(DataInputPlus in, int version) throws IOException
-        {
-            throw new UnsupportedOperationException();
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/hints/HintMessage.java b/src/java/org/apache/cassandra/hints/HintMessage.java
index 683b894..333af84 100644
--- a/src/java/org/apache/cassandra/hints/HintMessage.java
+++ b/src/java/org/apache/cassandra/hints/HintMessage.java
@@ -19,6 +19,7 @@
 package org.apache.cassandra.hints;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Objects;
 import java.util.UUID;
 
@@ -28,11 +29,9 @@ import com.google.common.primitives.Ints;
 
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.exceptions.UnknownTableException;
-import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.IVersionedAsymmetricSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.io.util.TrackedDataInputPlus;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.UUIDSerializer;
@@ -49,9 +48,9 @@ import org.apache.cassandra.utils.UUIDSerializer;
  * Scenario (2) means that we got a hint from a node that's going through decommissioning and is streaming its hints
  * elsewhere first.
  */
-public final class HintMessage
+public final class HintMessage implements SerializableHintMessage
 {
-    public static final IVersionedSerializer<HintMessage> serializer = new Serializer();
+    public static final IVersionedAsymmetricSerializer<SerializableHintMessage, HintMessage> serializer = new Serializer();
 
     final UUID hostId;
 
@@ -75,37 +74,72 @@ public final class HintMessage
         this.unknownTableID = unknownTableID;
     }
 
-    public MessageOut<HintMessage> createMessageOut()
+    public static class Serializer implements IVersionedAsymmetricSerializer<SerializableHintMessage, HintMessage>
     {
-        return new MessageOut<>(MessagingService.Verb.HINT, this, serializer);
-    }
-
-    public static class Serializer implements IVersionedSerializer<HintMessage>
-    {
-        public long serializedSize(HintMessage message, int version)
+        public long serializedSize(SerializableHintMessage obj, int version)
         {
-            long size = UUIDSerializer.serializer.serializedSize(message.hostId, version);
+            if (obj instanceof HintMessage)
+            {
+                HintMessage message = (HintMessage) obj;
+                long size = UUIDSerializer.serializer.serializedSize(message.hostId, version);
+
+                long hintSize = Hint.serializer.serializedSize(message.hint, version);
+                size += TypeSizes.sizeofUnsignedVInt(hintSize);
+                size += hintSize;
+
+                return size;
+            }
+            else if (obj instanceof Encoded)
+            {
+                Encoded message = (Encoded) obj;
 
-            long hintSize = Hint.serializer.serializedSize(message.hint, version);
-            size += TypeSizes.sizeofUnsignedVInt(hintSize);
-            size += hintSize;
+                if (version != message.version)
+                    throw new IllegalArgumentException("serializedSize() called with non-matching version " + version);
 
-            return size;
+                long size = UUIDSerializer.serializer.serializedSize(message.hostId, version);
+                size += TypeSizes.sizeofUnsignedVInt(message.hint.remaining());
+                size += message.hint.remaining();
+                return size;
+            }
+            else
+            {
+                throw new IllegalStateException("Unexpected type: " + obj);
+            }
         }
 
-        public void serialize(HintMessage message, DataOutputPlus out, int version) throws IOException
+        public void serialize(SerializableHintMessage obj, DataOutputPlus out, int version) throws IOException
         {
-            Objects.requireNonNull(message.hint); // we should never *send* a HintMessage with null hint
+            if (obj instanceof HintMessage)
+            {
+                HintMessage message = (HintMessage) obj;
 
-            UUIDSerializer.serializer.serialize(message.hostId, out, version);
+                Objects.requireNonNull(message.hint); // we should never *send* a HintMessage with null hint
 
-            /*
-             * We are serializing the hint size so that the receiver of the message could gracefully handle
-             * deserialize failure when a table had been dropped, by simply skipping the unread bytes.
-             */
-            out.writeUnsignedVInt(Hint.serializer.serializedSize(message.hint, version));
+                UUIDSerializer.serializer.serialize(message.hostId, out, version);
 
-            Hint.serializer.serialize(message.hint, out, version);
+                /*
+                 * We are serializing the hint size so that the receiver of the message could gracefully handle
+                 * deserialize failure when a table had been dropped, by simply skipping the unread bytes.
+                 */
+                out.writeUnsignedVInt(Hint.serializer.serializedSize(message.hint, version));
+
+                Hint.serializer.serialize(message.hint, out, version);
+            }
+            else if (obj instanceof Encoded)
+            {
+                Encoded message = (Encoded) obj;
+
+                if (version != message.version)
+                    throw new IllegalArgumentException("serialize() called with non-matching version " + version);
+
+                UUIDSerializer.serializer.serialize(message.hostId, out, version);
+                out.writeUnsignedVInt(message.hint.remaining());
+                out.write(message.hint);
+            }
+            else
+            {
+                throw new IllegalStateException("Unexpected type: " + obj);
+            }
         }
 
         /*
@@ -130,4 +164,32 @@ public final class HintMessage
             }
         }
     }
+
+    /**
+     * A specialized version of {@link HintMessage} that takes an already encoded in a bytebuffer hint and sends it verbatim.
+     *
+     * An optimization for when dispatching a hint file of the current messaging version to a node of the same messaging version,
+     * which is the most common case. Saves on extra ByteBuffer allocations one redundant hint deserialization-serialization cycle.
+     *
+     * Never deserialized as an HintMessage.Encoded - the receiving side will always deserialize the message as vanilla
+     * {@link HintMessage}.
+     */
+    static final class Encoded implements SerializableHintMessage
+    {
+        private final UUID hostId;
+        private final ByteBuffer hint;
+        private final int version;
+
+        Encoded(UUID hostId, ByteBuffer hint, int version)
+        {
+            this.hostId = hostId;
+            this.hint = hint;
+            this.version = version;
+        }
+
+        public long getHintCreationTime()
+        {
+            return Hint.serializer.getHintCreationTime(hint, version);
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/hints/HintResponse.java b/src/java/org/apache/cassandra/hints/HintResponse.java
deleted file mode 100644
index 8aa888f..0000000
--- a/src/java/org/apache/cassandra/hints/HintResponse.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.cassandra.hints;
-
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-
-/**
- * An empty successful response to a HintMessage.
- */
-public final class HintResponse
-{
-    public static final IVersionedSerializer<HintResponse> serializer = new Serializer();
-
-    static final HintResponse instance = new HintResponse();
-    static final MessageOut<HintResponse> message =
-        new MessageOut<>(MessagingService.Verb.REQUEST_RESPONSE, instance, serializer);
-
-    private HintResponse()
-    {
-    }
-
-    private static final class Serializer implements IVersionedSerializer<HintResponse>
-    {
-        public long serializedSize(HintResponse response, int version)
-        {
-            return 0;
-        }
-
-        public void serialize(HintResponse response, DataOutputPlus out, int version)
-        {
-        }
-
-        public HintResponse deserialize(DataInputPlus in, int version)
-        {
-            return instance;
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/hints/HintVerbHandler.java b/src/java/org/apache/cassandra/hints/HintVerbHandler.java
index cec6f0b..2fbe475 100644
--- a/src/java/org/apache/cassandra/hints/HintVerbHandler.java
+++ b/src/java/org/apache/cassandra/hints/HintVerbHandler.java
@@ -26,7 +26,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.service.StorageProxy;
@@ -41,9 +41,11 @@ import org.apache.cassandra.service.StorageService;
  */
 public final class HintVerbHandler implements IVerbHandler<HintMessage>
 {
+    public static final HintVerbHandler instance = new HintVerbHandler();
+
     private static final Logger logger = LoggerFactory.getLogger(HintVerbHandler.class);
 
-    public void doVerb(MessageIn<HintMessage> message, int id)
+    public void doVerb(Message<HintMessage> message)
     {
         UUID hostId = message.payload.hostId;
         Hint hint = message.payload.hint;
@@ -59,7 +61,7 @@ public final class HintVerbHandler implements IVerbHandler<HintMessage>
                          address,
                          hostId,
                          message.payload.unknownTableID);
-            reply(id, message.from);
+            respond(message);
             return;
         }
 
@@ -71,7 +73,7 @@ public final class HintVerbHandler implements IVerbHandler<HintMessage>
         catch (MarshalException e)
         {
             logger.warn("Failed to validate a hint for {}: {} - skipped", address, hostId);
-            reply(id, message.from);
+            respond(message);
             return;
         }
 
@@ -80,24 +82,24 @@ public final class HintVerbHandler implements IVerbHandler<HintMessage>
             // the node is not the final destination of the hint (must have gotten it from a decommissioning node),
             // so just store it locally, to be delivered later.
             HintsService.instance.write(hostId, hint);
-            reply(id, message.from);
+            respond(message);
         }
         else if (!StorageProxy.instance.appliesLocally(hint.mutation))
         {
             // the topology has changed, and we are no longer a replica of the mutation - since we don't know which node(s)
             // it has been handed over to, re-address the hint to all replicas; see CASSANDRA-5902.
             HintsService.instance.writeForAllReplicas(hint);
-            reply(id, message.from);
+            respond(message);
         }
         else
         {
             // the common path - the node is both the destination and a valid replica for the hint.
-            hint.applyFuture().thenAccept(o -> reply(id, message.from)).exceptionally(e -> {logger.debug("Failed to apply hint", e); return null;});
+            hint.applyFuture().thenAccept(o -> respond(message)).exceptionally(e -> {logger.debug("Failed to apply hint", e); return null;});
         }
     }
 
-    private static void reply(int id, InetAddressAndPort to)
+    private static void respond(Message<HintMessage> respondTo)
     {
-        MessagingService.instance().sendReply(HintResponse.message, id, to);
+        MessagingService.instance().send(respondTo.emptyResponse(), respondTo.from());
     }
 }
diff --git a/src/java/org/apache/cassandra/hints/HintsDispatcher.java b/src/java/org/apache/cassandra/hints/HintsDispatcher.java
index 2cff186..39e4b25 100644
--- a/src/java/org/apache/cassandra/hints/HintsDispatcher.java
+++ b/src/java/org/apache/cassandra/hints/HintsDispatcher.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.hints;
 import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.TimeUnit;
 import java.util.function.BooleanSupplier;
 import java.util.function.Function;
 
@@ -28,19 +27,21 @@ import com.google.common.util.concurrent.RateLimiter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.db.monitoring.ApproximateTime;
+import org.apache.cassandra.net.RequestCallback;
 import org.apache.cassandra.exceptions.RequestFailureReason;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.HintsServiceMetrics;
-import org.apache.cassandra.net.IAsyncCallbackWithFailure;
-import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.concurrent.SimpleCondition;
 
+import static org.apache.cassandra.net.Verb.HINT_REQ;
+import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+
 /**
  * Dispatches a single hints file to a specified node in a batched manner.
  *
- * Uses either {@link EncodedHintMessage} - when dispatching hints into a node with the same messaging version as the hints file,
+ * Uses either {@link HintMessage.Encoded} - when dispatching hints into a node with the same messaging version as the hints file,
  * or {@link HintMessage}, when conversion is required.
  */
 final class HintsDispatcher implements AutoCloseable
@@ -70,7 +71,7 @@ final class HintsDispatcher implements AutoCloseable
 
     static HintsDispatcher create(File file, RateLimiter rateLimiter, InetAddressAndPort address, UUID hostId, BooleanSupplier abortRequested)
     {
-        int messagingVersion = MessagingService.instance().getVersion(address);
+        int messagingVersion = MessagingService.instance().versions.get(address);
         HintsDispatcher dispatcher = new HintsDispatcher(HintsReader.open(file, rateLimiter), hostId, address, messagingVersion, abortRequested);
         HintDiagnostics.dispatcherCreated(dispatcher);
         return dispatcher;
@@ -187,8 +188,8 @@ final class HintsDispatcher implements AutoCloseable
     private Callback sendHint(Hint hint)
     {
         Callback callback = new Callback(hint.creationTime);
-        HintMessage message = new HintMessage(hostId, hint);
-        MessagingService.instance().sendRRWithFailure(message.createMessageOut(), address, callback);
+        Message<?> message = Message.out(HINT_REQ, new HintMessage(hostId, hint));
+        MessagingService.instance().sendWithCallback(message, address, callback);
         return callback;
     }
 
@@ -198,34 +199,32 @@ final class HintsDispatcher implements AutoCloseable
 
     private Callback sendEncodedHint(ByteBuffer hint)
     {
-        EncodedHintMessage message = new EncodedHintMessage(hostId, hint, messagingVersion);
+        HintMessage.Encoded message = new HintMessage.Encoded(hostId, hint, messagingVersion);
         Callback callback = new Callback(message.getHintCreationTime());
-        MessagingService.instance().sendRRWithFailure(message.createMessageOut(), address, callback);
+        MessagingService.instance().sendWithCallback(Message.out(HINT_REQ, message), address, callback);
         return callback;
     }
 
-    private static final class Callback implements IAsyncCallbackWithFailure
+    private static final class Callback implements RequestCallback
     {
         enum Outcome { SUCCESS, TIMEOUT, FAILURE, INTERRUPTED }
 
-        private final long start = System.nanoTime();
+        private final long start = approxTime.now();
         private final SimpleCondition condition = new SimpleCondition();
         private volatile Outcome outcome;
-        private final long hintCreationTime;
+        private final long hintCreationNanoTime;
 
-        private Callback(long hintCreationTime)
+        private Callback(long hintCreationTimeMillisSinceEpoch)
         {
-            this.hintCreationTime = hintCreationTime;
+            this.hintCreationNanoTime = approxTime.translate().fromMillisSinceEpoch(hintCreationTimeMillisSinceEpoch);
         }
 
         Outcome await()
         {
-            long timeout = TimeUnit.MILLISECONDS.toNanos(MessagingService.Verb.HINT.getTimeout()) - (System.nanoTime() - start);
             boolean timedOut;
-
             try
             {
-                timedOut = !condition.await(timeout, TimeUnit.NANOSECONDS);
+                timedOut = !condition.awaitUntil(HINT_REQ.expiresAtNanos(start));
             }
             catch (InterruptedException e)
             {
@@ -236,24 +235,27 @@ final class HintsDispatcher implements AutoCloseable
             return timedOut ? Outcome.TIMEOUT : outcome;
         }
 
+        @Override
+        public boolean invokeOnFailure()
+        {
+            return true;
+        }
+
+        @Override
         public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason)
         {
             outcome = Outcome.FAILURE;
             condition.signalAll();
         }
 
-        public void response(MessageIn msg)
+        @Override
+        public void onResponse(Message msg)
         {
-            HintsServiceMetrics.updateDelayMetrics(msg.from, ApproximateTime.currentTimeMillis() - this.hintCreationTime);
+            HintsServiceMetrics.updateDelayMetrics(msg.from(), approxTime.now() - this.hintCreationNanoTime);
             outcome = Outcome.SUCCESS;
             condition.signalAll();
         }
 
-        public boolean isLatencyForSnitch()
-        {
-            return false;
-        }
-
         @Override
         public boolean supportsBackPressure()
         {
diff --git a/src/java/org/apache/cassandra/locator/ILatencySubscriber.java b/src/java/org/apache/cassandra/hints/SerializableHintMessage.java
similarity index 84%
copy from src/java/org/apache/cassandra/locator/ILatencySubscriber.java
copy to src/java/org/apache/cassandra/hints/SerializableHintMessage.java
index f6c1c7f..43c289c 100644
--- a/src/java/org/apache/cassandra/locator/ILatencySubscriber.java
+++ b/src/java/org/apache/cassandra/hints/SerializableHintMessage.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.cassandra.locator;
 
-public interface ILatencySubscriber
+package org.apache.cassandra.hints;
+
+public interface SerializableHintMessage
 {
-    public void receiveTiming(InetAddressAndPort address, long latency);
 }
diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index ec54a65..b37251a 100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@ -71,11 +71,15 @@ import org.apache.cassandra.schema.Indexes;
 import org.apache.cassandra.service.pager.SinglePartitionPager;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ExecutorUtils;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.Refs;
 
+import static org.apache.cassandra.utils.ExecutorUtils.awaitTermination;
+import static org.apache.cassandra.utils.ExecutorUtils.shutdown;
+
 /**
  * Handles the core maintenance functionality associated with indexes: adding/removing them to or from
  * a table, (re)building during bootstrap or other streaming operations, flushing, reloading metadata
@@ -1487,12 +1491,9 @@ public class SecondaryIndexManager implements IndexRegistry, INotificationConsum
     }
 
     @VisibleForTesting
-    public static void shutdownExecutors() throws InterruptedException
+    public static void shutdownAndWait(long timeout, TimeUnit units) throws InterruptedException, TimeoutException
     {
-        ExecutorService[] executors = new ExecutorService[]{ asyncExecutor, blockingExecutor };
-        for (ExecutorService executor : executors)
-            executor.shutdown();
-        for (ExecutorService executor : executors)
-            executor.awaitTermination(60, TimeUnit.SECONDS);
+        shutdown(asyncExecutor, blockingExecutor);
+        awaitTermination(timeout, units, asyncExecutor, blockingExecutor);
     }
 }
diff --git a/src/java/org/apache/cassandra/index/sasi/SASIIndex.java b/src/java/org/apache/cassandra/index/sasi/SASIIndex.java
index 19c09cc..07327ea 100644
--- a/src/java/org/apache/cassandra/index/sasi/SASIIndex.java
+++ b/src/java/org/apache/cassandra/index/sasi/SASIIndex.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.index.sasi;
 
 import java.util.*;
 import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
 import java.util.function.BiFunction;
 
 import com.googlecode.concurrenttrees.common.Iterables;
@@ -60,6 +61,8 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
 public class SASIIndex implements Index, INotificationConsumer
 {
     public final static String USAGE_WARNING = "SASI indexes are experimental and are not recommended for production use.";
@@ -295,7 +298,7 @@ public class SASIIndex implements Index, INotificationConsumer
     {
         TableMetadata config = command.metadata();
         ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(config.id);
-        return controller -> new QueryPlan(cfs, command, DatabaseDescriptor.getRangeRpcTimeout()).execute(controller);
+        return controller -> new QueryPlan(cfs, command, DatabaseDescriptor.getRangeRpcTimeout(MILLISECONDS)).execute(controller);
     }
 
     public SSTableFlushObserver getFlushObserver(Descriptor descriptor, OperationType opType)
diff --git a/src/java/org/apache/cassandra/io/DummyByteVersionedSerializer.java b/src/java/org/apache/cassandra/io/DummyByteVersionedSerializer.java
deleted file mode 100644
index d82ff7d..0000000
--- a/src/java/org/apache/cassandra/io/DummyByteVersionedSerializer.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.cassandra.io;
-
-import java.io.IOException;
-
-import com.google.common.base.Preconditions;
-
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessagingService;
-
-/**
- * Serializes a dummy byte that can't be set. Will always write 0 and return 0 in a correctly formed message.
- */
-public class DummyByteVersionedSerializer implements IVersionedSerializer<byte[]>
-{
-    public static final DummyByteVersionedSerializer instance = new DummyByteVersionedSerializer();
-
-    private DummyByteVersionedSerializer() {}
-
-    public void serialize(byte[] bytes, DataOutputPlus out, int version) throws IOException
-    {
-        Preconditions.checkArgument(bytes == MessagingService.ONE_BYTE);
-        out.write(0);
-    }
-
-    public byte[] deserialize(DataInputPlus in, int version) throws IOException
-    {
-        assert(0 == in.readByte());
-        return MessagingService.ONE_BYTE;
-    }
-
-    public long serializedSize(byte[] bytes, int version)
-    {
-        //Payload
-        return 1;
-    }
-}
diff --git a/src/java/org/apache/cassandra/io/IVersionedSerializer.java b/src/java/org/apache/cassandra/io/IVersionedAsymmetricSerializer.java
similarity index 84%
copy from src/java/org/apache/cassandra/io/IVersionedSerializer.java
copy to src/java/org/apache/cassandra/io/IVersionedAsymmetricSerializer.java
index e555573..8ad2c28 100644
--- a/src/java/org/apache/cassandra/io/IVersionedSerializer.java
+++ b/src/java/org/apache/cassandra/io/IVersionedAsymmetricSerializer.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
-public interface IVersionedSerializer<T>
+public interface IVersionedAsymmetricSerializer<In, Out>
 {
     /**
      * Serialize the specified type into the specified DataOutputStream instance.
@@ -30,9 +30,9 @@ public interface IVersionedSerializer<T>
      * @param t type that needs to be serialized
      * @param out DataOutput into which serialization needs to happen.
      * @param version protocol version
-     * @throws java.io.IOException if serialization fails
+     * @throws IOException if serialization fails
      */
-    public void serialize(T t, DataOutputPlus out, int version) throws IOException;
+    public void serialize(In t, DataOutputPlus out, int version) throws IOException;
 
     /**
      * Deserialize into the specified DataInputStream instance.
@@ -41,7 +41,7 @@ public interface IVersionedSerializer<T>
      * @return the type that was deserialized
      * @throws IOException if deserialization fails
      */
-    public T deserialize(DataInputPlus in, int version) throws IOException;
+    public Out deserialize(DataInputPlus in, int version) throws IOException;
 
     /**
      * Calculate serialized size of object without actually serializing.
@@ -49,5 +49,5 @@ public interface IVersionedSerializer<T>
      * @param version protocol version
      * @return serialized size of object t
      */
-    public long serializedSize(T t, int version);
+    public long serializedSize(In t, int version);
 }
diff --git a/src/java/org/apache/cassandra/io/IVersionedSerializer.java b/src/java/org/apache/cassandra/io/IVersionedSerializer.java
index e555573..6730ec0 100644
--- a/src/java/org/apache/cassandra/io/IVersionedSerializer.java
+++ b/src/java/org/apache/cassandra/io/IVersionedSerializer.java
@@ -17,37 +17,6 @@
  */
 package org.apache.cassandra.io;
 
-import java.io.IOException;
-
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
-
-public interface IVersionedSerializer<T>
+public interface IVersionedSerializer<T> extends IVersionedAsymmetricSerializer<T, T>
 {
-    /**
-     * Serialize the specified type into the specified DataOutputStream instance.
-     *
-     * @param t type that needs to be serialized
-     * @param out DataOutput into which serialization needs to happen.
-     * @param version protocol version
-     * @throws java.io.IOException if serialization fails
-     */
-    public void serialize(T t, DataOutputPlus out, int version) throws IOException;
-
-    /**
-     * Deserialize into the specified DataInputStream instance.
-     * @param in DataInput from which deserialization needs to happen.
-     * @param version protocol version
-     * @return the type that was deserialized
-     * @throws IOException if deserialization fails
-     */
-    public T deserialize(DataInputPlus in, int version) throws IOException;
-
-    /**
-     * Calculate serialized size of object without actually serializing.
-     * @param t object to calculate serialized size
-     * @param version protocol version
-     * @return serialized size of object t
-     */
-    public long serializedSize(T t, int version);
 }
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 e3059c8..b545d51 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -56,6 +56,7 @@ import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.UnknownColumnException;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.*;
@@ -431,13 +432,22 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         long fileLength = new File(descriptor.filenameFor(Component.DATA)).length();
         if (logger.isDebugEnabled())
             logger.debug("Opening {} ({})", descriptor, FBUtilities.prettyPrintMemory(fileLength));
-        SSTableReader sstable = internalOpen(descriptor,
-                                             components,
-                                             metadata,
-                                             System.currentTimeMillis(),
-                                             statsMetadata,
-                                             OpenReason.NORMAL,
-                                             header.toHeader(metadata.get()));
+
+        final SSTableReader sstable;
+        try
+        {
+            sstable = internalOpen(descriptor,
+                                   components,
+                                   metadata,
+                                   System.currentTimeMillis(),
+                                   statsMetadata,
+                                   OpenReason.NORMAL,
+                                   header.toHeader(metadata.get()));
+        }
+        catch (UnknownColumnException e)
+        {
+            throw new IllegalStateException(e);
+        }
 
         try(FileHandle.Builder ibuilder = new FileHandle.Builder(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX))
                                                      .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
@@ -522,13 +532,22 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         long fileLength = new File(descriptor.filenameFor(Component.DATA)).length();
         if (logger.isDebugEnabled())
             logger.debug("Opening {} ({})", descriptor, FBUtilities.prettyPrintMemory(fileLength));
-        SSTableReader sstable = internalOpen(descriptor,
-                                             components,
-                                             metadata,
-                                             System.currentTimeMillis(),
-                                             statsMetadata,
-                                             OpenReason.NORMAL,
-                                             header.toHeader(metadata.get()));
+
+        final SSTableReader sstable;
+        try
+        {
+            sstable = internalOpen(descriptor,
+                                   components,
+                                   metadata,
+                                   System.currentTimeMillis(),
+                                   statsMetadata,
+                                   OpenReason.NORMAL,
+                                   header.toHeader(metadata.get()));
+        }
+        catch (UnknownColumnException e)
+        {
+            throw new IllegalStateException(e);
+        }
 
         try
         {
@@ -2475,12 +2494,14 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         return reader;
     }
 
-    public static void shutdownBlocking() throws InterruptedException
+    public static void shutdownBlocking(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
     {
         if (syncExecutor != null)
         {
             syncExecutor.shutdownNow();
-            syncExecutor.awaitTermination(0, TimeUnit.SECONDS);
+            syncExecutor.awaitTermination(timeout, unit);
+            if (!syncExecutor.isTerminated())
+                throw new TimeoutException();
         }
         resetTidying();
     }
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java
index 8826381..f05ea94 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.io.sstable.format.big;
 
+import java.io.EOFException;
 import java.io.File;
 import java.io.IOException;
 import java.util.Collection;
@@ -31,7 +32,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.compress.BufferType;
@@ -43,7 +43,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.SequentialWriter;
 import org.apache.cassandra.io.util.SequentialWriterOption;
-import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus;
+import org.apache.cassandra.net.AsyncStreamingInputPlus;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadataRef;
 
@@ -202,22 +202,25 @@ public class BigTableZeroCopyWriter extends SSTable implements SSTableMultiWrite
     {
         logger.info("Writing component {} to {} length {}", type, componentWriters.get(type).getPath(), prettyPrintMemory(size));
 
-        if (in instanceof RebufferingByteBufDataInputPlus)
-            write((RebufferingByteBufDataInputPlus) in, size, componentWriters.get(type));
+        if (in instanceof AsyncStreamingInputPlus)
+            write((AsyncStreamingInputPlus) in, size, componentWriters.get(type));
         else
             write(in, size, componentWriters.get(type));
     }
 
-    private void write(RebufferingByteBufDataInputPlus in, long size, SequentialWriter writer)
+    private void write(AsyncStreamingInputPlus in, long size, SequentialWriter writer)
     {
         logger.info("Block Writing component to {} length {}", writer.getPath(), prettyPrintMemory(size));
 
         try
         {
-            long bytesWritten = in.consumeUntil(writer, size);
-
-            if (bytesWritten != size)
-                throw new IOException(format("Failed to read correct number of bytes from channel %s", writer));
+            in.consume(writer::writeDirectlyToChannel, size);
+            writer.sync();
+        }
+        // FIXME: handle ACIP exceptions properly
+        catch (EOFException | AsyncStreamingInputPlus.InputTimeoutException e)
+        {
+            in.close();
         }
         catch (IOException e)
         {
diff --git a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java
index 56d88f7..7d1e91d 100644
--- a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java
+++ b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java
@@ -28,8 +28,8 @@ import com.google.common.base.Preconditions;
 
 import net.nicoulaj.compilecommand.annotations.DontInline;
 import org.apache.cassandra.config.Config;
+import org.apache.cassandra.utils.FastByteOperations;
 import org.apache.cassandra.utils.memory.MemoryUtil;
-import org.apache.cassandra.utils.vint.VIntCoding;
 
 /**
  * An implementation of the DataOutputStreamPlus interface using a ByteBuffer to stage writes
@@ -43,15 +43,6 @@ public class BufferedDataOutputStreamPlus extends DataOutputStreamPlus
 
     protected ByteBuffer buffer;
 
-    //Allow derived classes to specify writing to the channel
-    //directly shouldn't happen because they intercept via doFlush for things
-    //like compression or checksumming
-    //Another hack for this value is that it also indicates that flushing early
-    //should not occur, flushes aligned with buffer size are desired
-    //Unless... it's the last flush. Compression and checksum formats
-    //expect block (same as buffer size) alignment for everything except the last block
-    protected boolean strictFlushing = false;
-
     public BufferedDataOutputStreamPlus(RandomAccessFile ras)
     {
         this(ras.getChannel());
@@ -132,9 +123,6 @@ public class BufferedDataOutputStreamPlus extends DataOutputStreamPlus
         }
     }
 
-    // ByteBuffer to use for defensive copies
-    private final ByteBuffer hollowBuffer = MemoryUtil.getHollowDirectByteBuffer();
-
     /*
      * Makes a defensive copy of the incoming ByteBuffer and don't modify the position or limit
      * even temporarily so it is thread-safe WRT to the incoming buffer
@@ -142,48 +130,20 @@ public class BufferedDataOutputStreamPlus extends DataOutputStreamPlus
      * @see org.apache.cassandra.io.util.DataOutputPlus#write(java.nio.ByteBuffer)
      */
     @Override
-    public void write(ByteBuffer toWrite) throws IOException
-    {
-        if (toWrite.hasArray())
-        {
-            write(toWrite.array(), toWrite.arrayOffset() + toWrite.position(), toWrite.remaining());
-        }
-        else
-        {
-            assert toWrite.isDirect();
-            MemoryUtil.duplicateDirectByteBuffer(toWrite, hollowBuffer);
-            int toWriteRemaining = toWrite.remaining();
-
-            if (toWriteRemaining > buffer.remaining())
-            {
-                if (strictFlushing)
-                {
-                    writeExcessSlow();
-                }
-                else
-                {
-                    doFlush(toWriteRemaining - buffer.remaining());
-                    while (hollowBuffer.remaining() > buffer.capacity())
-                        channel.write(hollowBuffer);
-                }
-            }
-
-            buffer.put(hollowBuffer);
-        }
-    }
-
-    // writes anything we can't fit into the buffer
-    @DontInline
-    private void writeExcessSlow() throws IOException
+    public void write(ByteBuffer src) throws IOException
     {
-        int originalLimit = hollowBuffer.limit();
-        while (originalLimit - hollowBuffer.position() > buffer.remaining())
+        int srcPos = src.position();
+        int srcCount;
+        int trgAvailable;
+        while ((srcCount = src.limit() - srcPos) > (trgAvailable = buffer.remaining()))
         {
-            hollowBuffer.limit(hollowBuffer.position() + buffer.remaining());
-            buffer.put(hollowBuffer);
-            doFlush(originalLimit - hollowBuffer.position());
+            FastByteOperations.copy(src, srcPos, buffer, buffer.position(), trgAvailable);
+            buffer.position(buffer.position() + trgAvailable);
+            srcPos += trgAvailable;
+            doFlush(src.limit() - srcPos);
         }
-        hollowBuffer.limit(originalLimit);
+        FastByteOperations.copy(src, srcPos, buffer, buffer.position(), srcCount);
+        buffer.position(buffer.position() + srcCount);
     }
 
     @Override
@@ -242,25 +202,6 @@ public class BufferedDataOutputStreamPlus extends DataOutputStreamPlus
     }
 
     @Override
-    public void writeVInt(long value) throws IOException
-    {
-        writeUnsignedVInt(VIntCoding.encodeZigZag64(value));
-    }
-
-    @Override
-    public void writeUnsignedVInt(long value) throws IOException
-    {
-        int size = VIntCoding.computeUnsignedVIntSize(value);
-        if (size == 1)
-        {
-            write((int) value);
-            return;
-        }
-
-        write(VIntCoding.encodeVInt(value, size), 0, size);
-    }
-
-    @Override
     public void writeFloat(float v) throws IOException
     {
         writeInt(Float.floatToRawIntBits(v));
@@ -302,13 +243,6 @@ public class BufferedDataOutputStreamPlus extends DataOutputStreamPlus
         UnbufferedDataOutputStreamPlus.writeUTF(s, this);
     }
 
-    @Override
-    public void write(Memory memory, long offset, long length) throws IOException
-    {
-        for (ByteBuffer buffer : memory.asByteBuffers(offset, length))
-            write(buffer);
-    }
-
     /*
      * Count is the number of bytes remaining to write ignoring already remaining capacity
      */
@@ -338,16 +272,6 @@ public class BufferedDataOutputStreamPlus extends DataOutputStreamPlus
         buffer = null;
     }
 
-    @Override
-    public <R> R applyToChannel(CheckedFunction<WritableByteChannel, R, IOException> f) throws IOException
-    {
-        if (strictFlushing)
-            throw new UnsupportedOperationException();
-        //Don't allow writes to the underlying channel while data is buffered
-        flush();
-        return f.apply(channel);
-    }
-
     public BufferedDataOutputStreamPlus order(ByteOrder order)
     {
         this.buffer.order(order);
diff --git a/src/java/org/apache/cassandra/io/util/DataOutputPlus.java b/src/java/org/apache/cassandra/io/util/DataOutputPlus.java
index 16be42f..b94d097 100644
--- a/src/java/org/apache/cassandra/io/util/DataOutputPlus.java
+++ b/src/java/org/apache/cassandra/io/util/DataOutputPlus.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.io.util;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.nio.channels.WritableByteChannel;
 
 import org.apache.cassandra.utils.vint.VIntCoding;
 
@@ -33,13 +32,11 @@ public interface DataOutputPlus extends DataOutput
     // write the buffer without modifying its position
     void write(ByteBuffer buffer) throws IOException;
 
-    void write(Memory memory, long offset, long length) throws IOException;
-
-    /**
-     * Safe way to operate against the underlying channel. Impossible to stash a reference to the channel
-     * and forget to flush
-     */
-    <R> R applyToChannel(CheckedFunction<WritableByteChannel, R, IOException> c) throws IOException;
+    default void write(Memory memory, long offset, long length) throws IOException
+    {
+        for (ByteBuffer buffer : memory.asByteBuffers(offset, length))
+            write(buffer);
+    }
 
     default void writeVInt(long i) throws IOException
     {
diff --git a/src/java/org/apache/cassandra/io/util/DataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/DataOutputStreamPlus.java
index 4adb6d2..e931899 100644
--- a/src/java/org/apache/cassandra/io/util/DataOutputStreamPlus.java
+++ b/src/java/org/apache/cassandra/io/util/DataOutputStreamPlus.java
@@ -119,7 +119,7 @@ public abstract class DataOutputStreamPlus extends OutputStream implements DataO
                 {
                     int toWriteThisTime = Math.min(buf.length, toWrite - totalWritten);
 
-                    ByteBufferUtil.arrayCopy(src, src.position() + totalWritten, buf, 0, toWriteThisTime);
+                    ByteBufferUtil.copyBytes(src, src.position() + totalWritten, buf, 0, toWriteThisTime);
 
                     DataOutputStreamPlus.this.write(buf, 0, toWriteThisTime);
 
diff --git a/src/java/org/apache/cassandra/io/util/FastByteArrayInputStream.java b/src/java/org/apache/cassandra/io/util/FastByteArrayInputStream.java
deleted file mode 100644
index f61546c..0000000
--- a/src/java/org/apache/cassandra/io/util/FastByteArrayInputStream.java
+++ /dev/null
@@ -1,249 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.io.util;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-/*
- * This file has been modified from Apache Harmony's ByteArrayInputStream
- * implementation. The synchronized methods of the original have been
- * replaced by non-synchronized methods. This makes this certain operations
- * FASTer, but also *not thread-safe*.
- *
- * This file remains formatted the same as the Apache Harmony original to
- * make patching easier if any bug fixes are made to the Harmony version.
- */
-
-/**
- * A specialized {@link InputStream } for reading the contents of a byte array.
- *
- * @see ByteArrayInputStream
- */
-public class FastByteArrayInputStream extends InputStream
-{
-    /**
-     * The {@code byte} array containing the bytes to stream over.
-     */
-    protected byte[] buf;
-
-    /**
-     * The current position within the byte array.
-     */
-    protected int pos;
-
-    /**
-     * The current mark position. Initially set to 0 or the <code>offset</code>
-     * parameter within the constructor.
-     */
-    protected int mark;
-
-    /**
-     * The total number of bytes initially available in the byte array
-     * {@code buf}.
-     */
-    protected int count;
-
-    /**
-     * Constructs a new {@code ByteArrayInputStream} on the byte array
-     * {@code buf}.
-     *
-     * @param buf
-     *            the byte array to stream over.
-     */
-    public FastByteArrayInputStream(byte buf[])
-    {
-        this.mark = 0;
-        this.buf = buf;
-        this.count = buf.length;
-    }
-
-    /**
-     * Constructs a new {@code ByteArrayInputStream} on the byte array
-     * {@code buf} with the initial position set to {@code offset} and the
-     * number of bytes available set to {@code offset} + {@code length}.
-     *
-     * @param buf
-     *            the byte array to stream over.
-     * @param offset
-     *            the initial position in {@code buf} to start streaming from.
-     * @param length
-     *            the number of bytes available for streaming.
-     */
-    public FastByteArrayInputStream(byte buf[], int offset, int length)
-    {
-        this.buf = buf;
-        pos = offset;
-        mark = offset;
-        count = offset + length > buf.length ? buf.length : offset + length;
-    }
-
-    /**
-     * Returns the number of bytes that are available before this stream will
-     * block. This method returns the number of bytes yet to be read from the
-     * source byte array.
-     *
-     * @return the number of bytes available before blocking.
-     */
-    @Override
-    public int available()
-    {
-        return count - pos;
-    }
-
-    /**
-     * Closes this stream and frees resources associated with this stream.
-     *
-     * @throws IOException
-     *             if an I/O error occurs while closing this stream.
-     */
-    @Override
-    public void close() throws IOException
-    {
-        // Do nothing on close, this matches JDK behaviour.
-    }
-
-    /**
-     * Sets a mark position in this ByteArrayInputStream. The parameter
-     * {@code readlimit} is ignored. Sending {@code reset()} will reposition the
-     * stream back to the marked position.
-     *
-     * @param readlimit
-     *            ignored.
-     * @see #markSupported()
-     * @see #reset()
-     */
-    @Override
-    public void mark(int readlimit)
-    {
-        mark = pos;
-    }
-
-    /**
-     * Indicates whether this stream supports the {@code mark()} and
-     * {@code reset()} methods. Returns {@code true} since this class supports
-     * these methods.
-     *
-     * @return always {@code true}.
-     * @see #mark(int)
-     * @see #reset()
-     */
-    @Override
-    public boolean markSupported()
-    {
-        return true;
-    }
-
-    /**
-     * Reads a single byte from the source byte array and returns it as an
-     * integer in the range from 0 to 255. Returns -1 if the end of the source
-     * array has been reached.
-     *
-     * @return the byte read or -1 if the end of this stream has been reached.
-     */
-    @Override
-    public int read()
-    {
-        return pos < count ? buf[pos++] & 0xFF : -1;
-    }
-
-    /**
-     * Reads at most {@code len} bytes from this stream and stores
-     * them in byte array {@code b} starting at {@code offset}. This
-     * implementation reads bytes from the source byte array.
-     *
-     * @param b
-     *            the byte array in which to store the bytes read.
-     * @param offset
-     *            the initial position in {@code b} to store the bytes read from
-     *            this stream.
-     * @param length
-     *            the maximum number of bytes to store in {@code b}.
-     * @return the number of bytes actually read or -1 if no bytes were read and
-     *         the end of the stream was encountered.
-     * @throws IndexOutOfBoundsException
-     *             if {@code offset < 0} or {@code length < 0}, or if
-     *             {@code offset + length} is greater than the size of
-     *             {@code b}.
-     * @throws NullPointerException
-     *             if {@code b} is {@code null}.
-     */
-    @Override
-    public int read(byte b[], int offset, int length)
-    {
-        if (b == null) {
-            throw new NullPointerException();
-        }
-        // avoid int overflow
-        if (offset < 0 || offset > b.length || length < 0
-                || length > b.length - offset)
-        {
-            throw new IndexOutOfBoundsException();
-        }
-        // Are there any bytes available?
-        if (this.pos >= this.count)
-        {
-            return -1;
-        }
-        if (length == 0)
-        {
-            return 0;
-        }
-
-        int copylen = this.count - pos < length ? this.count - pos : length;
-        System.arraycopy(buf, pos, b, offset, copylen);
-        pos += copylen;
-        return copylen;
-    }
-
-    /**
-     * Resets this stream to the last marked location. This implementation
-     * resets the position to either the marked position, the start position
-     * supplied in the constructor or 0 if neither has been provided.
-     *
-     * @see #mark(int)
-     */
-    @Override
-    public void reset()
-    {
-        pos = mark;
-    }
-
-    /**
-     * Skips {@code count} number of bytes in this InputStream. Subsequent
-     * {@code read()}s will not return these bytes unless {@code reset()} is
-     * used. This implementation skips {@code count} number of bytes in the
-     * target stream. It does nothing and returns 0 if {@code n} is negative.
-     *
-     * @param n
-     *            the number of bytes to skip.
-     * @return the number of bytes actually skipped.
-     */
-    @Override
-    public long skip(long n)
-    {
-        if (n <= 0)
-        {
-            return 0;
-        }
-        int temp = pos;
-        pos = this.count - pos < n ? this.count : (int) (pos + n);
-        return pos - temp;
-    }
-}
diff --git a/src/java/org/apache/cassandra/io/util/Memory.java b/src/java/org/apache/cassandra/io/util/Memory.java
index 0ca6aa2..eaa6e91 100644
--- a/src/java/org/apache/cassandra/io/util/Memory.java
+++ b/src/java/org/apache/cassandra/io/util/Memory.java
@@ -417,7 +417,7 @@ public class Memory implements AutoCloseable
     public void setByteBuffer(ByteBuffer buffer, long offset, int length)
     {
         checkBounds(offset, offset + length);
-        MemoryUtil.setByteBuffer(buffer, peer + offset, length);
+        MemoryUtil.setDirectByteBuffer(buffer, peer + offset, length);
     }
 
     public String toString()
diff --git a/src/java/org/apache/cassandra/io/util/SequentialWriter.java b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
index 3eb1a7d..9ad944b 100644
--- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.io.util;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.file.StandardOpenOption;
 
@@ -43,6 +44,15 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr
 
     protected final FileChannel fchannel;
 
+    //Allow derived classes to specify writing to the channel
+    //directly shouldn't happen because they intercept via doFlush for things
+    //like compression or checksumming
+    //Another hack for this value is that it also indicates that flushing early
+    //should not occur, flushes aligned with buffer size are desired
+    //Unless... it's the last flush. Compression and checksum formats
+    //expect block (same as buffer size) alignment for everything except the last block
+    private final boolean strictFlushing;
+
     // whether to do trickling fsync() to avoid sudden bursts of dirty buffer flushing by kernel causing read
     // latency spikes
     private final SequentialWriterOption option;
@@ -388,6 +398,15 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr
             txnProxy.close();
     }
 
+    public int writeDirectlyToChannel(ByteBuffer buf) throws IOException
+    {
+        if (strictFlushing)
+            throw new UnsupportedOperationException();
+        // Don't allow writes to the underlying channel while data is buffered
+        flush();
+        return channel.write(buf);
+    }
+
     public final void finish()
     {
         txnProxy.finish();
diff --git a/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java
index d9ef010..3d83212 100644
--- a/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java
+++ b/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java
@@ -371,15 +371,4 @@ public abstract class UnbufferedDataOutputStreamPlus extends DataOutputStreamPlu
         }
     }
 
-    public void write(Memory memory, long offset, long length) throws IOException
-    {
-        for (ByteBuffer buffer : memory.asByteBuffers(offset, length))
-            write(buffer);
-    }
-
-    @Override
-    public <R> R applyToChannel(CheckedFunction<WritableByteChannel, R, IOException> f) throws IOException
-    {
-        return f.apply(channel);
-    }
 }
diff --git a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
index ddc8fba..0b241ce 100644
--- a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
+++ b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
@@ -35,6 +35,7 @@ import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.EndpointState;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.gms.VersionedValue;
+import org.apache.cassandra.net.LatencySubscribers;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
@@ -43,7 +44,7 @@ import org.apache.cassandra.utils.MBeanWrapper;
 /**
  * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
  */
-public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
+public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements LatencySubscribers.Subscriber, DynamicEndpointSnitchMBean
 {
     private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
 
@@ -253,7 +254,7 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
         throw new UnsupportedOperationException("You shouldn't wrap the DynamicEndpointSnitch (within itself or otherwise)");
     }
 
-    public void receiveTiming(InetAddressAndPort host, long latency) // this is cheap
+    public void receiveTiming(InetAddressAndPort host, long latency, TimeUnit unit) // this is cheap
     {
         ExponentiallyDecayingReservoir sample = samples.get(host);
         if (sample == null)
@@ -263,7 +264,7 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
             if (sample == null)
                 sample = maybeNewSample;
         }
-        sample.update(latency);
+        sample.update(unit.toMillis(latency));
     }
 
     private void updateScores() // this is expensive
@@ -274,7 +275,7 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
         {
             if (MessagingService.instance() != null)
             {
-                MessagingService.instance().register(this);
+                MessagingService.instance().latencySubscribers.subscribe(this);
                 registered = true;
             }
 
diff --git a/src/java/org/apache/cassandra/locator/Endpoints.java b/src/java/org/apache/cassandra/locator/Endpoints.java
index a2bad6c..c1a9282 100644
--- a/src/java/org/apache/cassandra/locator/Endpoints.java
+++ b/src/java/org/apache/cassandra/locator/Endpoints.java
@@ -21,11 +21,15 @@ package org.apache.cassandra.locator;
 import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict;
 import org.apache.cassandra.utils.FBUtilities;
 
+import java.util.AbstractList;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 
+import com.google.common.collect.Lists;
+
 /**
  * A collection of Endpoints for a given ring position.  This will typically reside in a ReplicaLayout,
  * representing some subset of the endpoints for the Token or Range
@@ -52,6 +56,22 @@ public abstract class Endpoints<E extends Endpoints<E>> extends AbstractReplicaC
         return byEndpoint().keySet();
     }
 
+    public List<InetAddressAndPort> endpointList()
+    {
+        return new AbstractList<InetAddressAndPort>()
+        {
+            public InetAddressAndPort get(int index)
+            {
+                return list.get(index).endpoint();
+            }
+
+            public int size()
+            {
+                return list.size;
+            }
+        };
+    }
+
     public Map<InetAddressAndPort, Replica> byEndpoint()
     {
         ReplicaMap<InetAddressAndPort> map = byEndpoint;
diff --git a/src/java/org/apache/cassandra/locator/InetAddressAndPort.java b/src/java/org/apache/cassandra/locator/InetAddressAndPort.java
index a47c72a..6821f13 100644
--- a/src/java/org/apache/cassandra/locator/InetAddressAndPort.java
+++ b/src/java/org/apache/cassandra/locator/InetAddressAndPort.java
@@ -15,16 +15,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.cassandra.locator;
 
+import java.io.IOException;
 import java.io.Serializable;
+import java.net.Inet4Address;
+import java.net.Inet6Address;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
 
 import com.google.common.base.Preconditions;
 import com.google.common.net.HostAndPort;
 
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.FastByteOperations;
 
@@ -41,6 +49,7 @@ import org.apache.cassandra.utils.FastByteOperations;
  * need to sometimes return a port and sometimes not.
  *
  */
+@SuppressWarnings("UnstableApiUsage")
 public final class InetAddressAndPort implements Comparable<InetAddressAndPort>, Serializable
 {
     private static final long serialVersionUID = 0;
@@ -65,6 +74,11 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
         this.addressBytes = addressBytes;
     }
 
+    public InetAddressAndPort withPort(int port)
+    {
+        return new InetAddressAndPort(address, addressBytes, port);
+    }
+
     private static void validatePortRange(int port)
     {
         if (port < 0 | port > 65535)
@@ -127,7 +141,7 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
     {
         if (withPort)
         {
-            return HostAndPort.fromParts(address.getHostAddress(), port).toString();
+            return toString(address, port);
         }
         else
         {
@@ -135,6 +149,11 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
         }
     }
 
+    public static String toString(InetAddress address, int port)
+    {
+        return HostAndPort.fromParts(address.getHostAddress(), port).toString();
+    }
+
     public static InetAddressAndPort getByName(String name) throws UnknownHostException
     {
         return getByNameOverrideDefaults(name, null);
@@ -144,8 +163,6 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
      *
      * @param name Hostname + optional ports string
      * @param port Port to connect on, overridden by values in hostname string, defaults to DatabaseDescriptor default if not specified anywhere.
-     * @return
-     * @throws UnknownHostException
      */
     public static InetAddressAndPort getByNameOverrideDefaults(String name, Integer port) throws UnknownHostException
     {
@@ -201,4 +218,114 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
     {
         defaultPort = port;
     }
+
+    static int getDefaultPort()
+    {
+        return defaultPort;
+    }
+
+    /*
+     * As of version 4.0 the endpoint description includes a port number as an unsigned short
+     */
+    public static final class Serializer implements IVersionedSerializer<InetAddressAndPort>
+    {
+        public static final int MAXIMUM_SIZE = 19;
+
+        // We put the static instance here, to avoid complexity with dtests.
+        // InetAddressAndPort is one of the only classes we share between instances, which is possible cleanly
+        // because it has no type-dependencies in its public API, however Serializer requires DataOutputPlus, which requires...
+        // and the chain becomes quite unwieldy
+        public static final Serializer inetAddressAndPortSerializer = new Serializer();
+
+        private Serializer() {}
+
+        public void serialize(InetAddressAndPort endpoint, DataOutputPlus out, int version) throws IOException
+        {
+            byte[] buf = endpoint.addressBytes;
+
+            if (version >= MessagingService.VERSION_40)
+            {
+                out.writeByte(buf.length + 2);
+                out.write(buf);
+                out.writeShort(endpoint.port);
+            }
+            else
+            {
+                out.writeByte(buf.length);
+                out.write(buf);
+            }
+        }
+
+        public InetAddressAndPort deserialize(DataInputPlus in, int version) throws IOException
+        {
+            int size = in.readByte() & 0xFF;
+            switch(size)
+            {
+                //The original pre-4.0 serialiation of just an address
+                case 4:
+                case 16:
+                {
+                    byte[] bytes = new byte[size];
+                    in.readFully(bytes, 0, bytes.length);
+                    return getByAddress(bytes);
+                }
+                //Address and one port
+                case 6:
+                case 18:
+                {
+                    byte[] bytes = new byte[size - 2];
+                    in.readFully(bytes);
+
+                    int port = in.readShort() & 0xFFFF;
+                    return getByAddressOverrideDefaults(InetAddress.getByAddress(bytes), bytes, port);
+                }
+                default:
+                    throw new AssertionError("Unexpected size " + size);
+
+            }
+        }
+
+        /**
+         * Extract {@link InetAddressAndPort} from the provided {@link ByteBuffer} without altering its state.
+         */
+        public InetAddressAndPort extract(ByteBuffer buf, int position) throws IOException
+        {
+            int size = buf.get(position++) & 0xFF;
+            if (size == 4 || size == 16)
+            {
+                byte[] bytes = new byte[size];
+                ByteBufferUtil.copyBytes(buf, position, bytes, 0, size);
+                return getByAddress(bytes);
+            }
+            else if (size == 6 || size == 18)
+            {
+                byte[] bytes = new byte[size - 2];
+                ByteBufferUtil.copyBytes(buf, position, bytes, 0, size - 2);
+                position += (size - 2);
+                int port = buf.getShort(position) & 0xFFFF;
+                return getByAddressOverrideDefaults(InetAddress.getByAddress(bytes), bytes, port);
+            }
+
+            throw new AssertionError("Unexpected pre-4.0 InetAddressAndPort size " + size);
+        }
+
+        public long serializedSize(InetAddressAndPort from, int version)
+        {
+            //4.0 includes a port number
+            if (version >= MessagingService.VERSION_40)
+            {
+                if (from.address instanceof Inet4Address)
+                    return 1 + 4 + 2;
+                assert from.address instanceof Inet6Address;
+                return 1 + 16 + 2;
+            }
+            else
+            {
+                if (from.address instanceof Inet4Address)
+                    return 1 + 4;
+                assert from.address instanceof Inet6Address;
+                return 1 + 16;
+            }
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
index 5479010..dea8c76 100644
--- a/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
+++ b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
@@ -22,13 +22,16 @@ import java.net.UnknownHostException;
 
 import com.google.common.annotations.VisibleForTesting;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.gms.*;
+import org.apache.cassandra.net.ConnectionCategory;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.OutboundConnectionSettings;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.cassandra.net.ConnectionType.SMALL_MESSAGES;
+
 /**
  * Sidekick helper for snitches that want to reconnect from one IP addr for a node to another.
  * Typically, this is for situations like EC2 where a node will have a public address and a private address,
@@ -63,16 +66,15 @@ public class ReconnectableSnitchHelper implements IEndpointStateChangeSubscriber
     @VisibleForTesting
     static void reconnect(InetAddressAndPort publicAddress, InetAddressAndPort localAddress, IEndpointSnitch snitch, String localDc)
     {
-        if (!DatabaseDescriptor.getInternodeAuthenticator().authenticate(publicAddress.address, MessagingService.instance().portFor(publicAddress)))
+        if (!new OutboundConnectionSettings(publicAddress, localAddress).withDefaults(ConnectionCategory.MESSAGING).authenticate())
         {
             logger.debug("InternodeAuthenticator said don't reconnect to {} on {}", publicAddress, localAddress);
             return;
         }
 
-        if (snitch.getDatacenter(publicAddress).equals(localDc)
-                && !MessagingService.instance().getCurrentEndpoint(publicAddress).equals(localAddress))
+        if (snitch.getDatacenter(publicAddress).equals(localDc))
         {
-            MessagingService.instance().reconnectWithNewIp(publicAddress, localAddress);
+            MessagingService.instance().maybeReconnectWithNewIp(publicAddress, localAddress);
             logger.debug("Initiated reconnect to an Internal IP {} for the {}", localAddress, publicAddress);
         }
     }
diff --git a/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java b/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
deleted file mode 100644
index 3655a40..0000000
--- a/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.metrics;
-
-import com.codahale.metrics.Gauge;
-import com.codahale.metrics.Meter;
-import org.apache.cassandra.net.async.OutboundMessagingPool;
-
-import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
-
-import org.apache.cassandra.locator.InetAddressAndPort;
-
-/**
- * Metrics for internode connections.
- */
-public class ConnectionMetrics
-{
-    public static final String TYPE_NAME = "Connection";
-
-    /** Total number of timeouts happened on this node */
-    public static final Meter totalTimeouts = Metrics.meter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalTimeouts", null));
-
-    public final String address;
-    /** Pending tasks for large message TCP Connections */
-    public final Gauge<Integer> largeMessagePendingTasks;
-    /** Completed tasks for large message TCP Connections */
-    public final Gauge<Long> largeMessageCompletedTasks;
-    /** Dropped tasks for large message TCP Connections */
-    public final Gauge<Long> largeMessageDroppedTasks;
-    /** Pending tasks for small message TCP Connections */
-    public final Gauge<Integer> smallMessagePendingTasks;
-    /** Completed tasks for small message TCP Connections */
-    public final Gauge<Long> smallMessageCompletedTasks;
-    /** Dropped tasks for small message TCP Connections */
-    public final Gauge<Long> smallMessageDroppedTasks;
-    /** Pending tasks for gossip message TCP Connections */
-    public final Gauge<Integer> gossipMessagePendingTasks;
-    /** Completed tasks for gossip message TCP Connections */
-    public final Gauge<Long> gossipMessageCompletedTasks;
-    /** Dropped tasks for gossip message TCP Connections */
-    public final Gauge<Long> gossipMessageDroppedTasks;
-
-    /** Number of timeouts for specific IP */
-    public final Meter timeouts;
-
-    private final MetricNameFactory factory;
-
-    /**
-     * Create metrics for given connection pool.
-     *
-     * @param ip IP address to use for metrics label
-     */
-    public ConnectionMetrics(InetAddressAndPort ip, final OutboundMessagingPool messagingPool)
-    {
-        // ipv6 addresses will contain colons, which are invalid in a JMX ObjectName
-        address = ip.toString().replace(':', '.');
-
-        factory = new DefaultNameFactory("Connection", address);
-
-        largeMessagePendingTasks = Metrics.register(factory.createMetricName("LargeMessagePendingTasks"), new Gauge<Integer>()
-        {
-            public Integer getValue()
-            {
-                return messagingPool.largeMessageChannel.getPendingMessages();
-            }
-        });
-        largeMessageCompletedTasks = Metrics.register(factory.createMetricName("LargeMessageCompletedTasks"), new Gauge<Long>()
-        {
-            public Long getValue()
-            {
-                return messagingPool.largeMessageChannel.getCompletedMessages();
-            }
-        });
-        largeMessageDroppedTasks = Metrics.register(factory.createMetricName("LargeMessageDroppedTasks"), new Gauge<Long>()
-        {
-            public Long getValue()
-            {
-                return messagingPool.largeMessageChannel.getDroppedMessages();
-            }
-        });
-        smallMessagePendingTasks = Metrics.register(factory.createMetricName("SmallMessagePendingTasks"), new Gauge<Integer>()
-        {
-            public Integer getValue()
-            {
-                return messagingPool.smallMessageChannel.getPendingMessages();
-            }
-        });
-        smallMessageCompletedTasks = Metrics.register(factory.createMetricName("SmallMessageCompletedTasks"), new Gauge<Long>()
-        {
-            public Long getValue()
-            {
-                return messagingPool.smallMessageChannel.getCompletedMessages();
-            }
-        });
-        smallMessageDroppedTasks = Metrics.register(factory.createMetricName("SmallMessageDroppedTasks"), new Gauge<Long>()
-        {
-            public Long getValue()
-            {
-                return messagingPool.smallMessageChannel.getDroppedMessages();
-            }
-        });
-        gossipMessagePendingTasks = Metrics.register(factory.createMetricName("GossipMessagePendingTasks"), new Gauge<Integer>()
-        {
-            public Integer getValue()
-            {
-                return messagingPool.gossipChannel.getPendingMessages();
-            }
-        });
-        gossipMessageCompletedTasks = Metrics.register(factory.createMetricName("GossipMessageCompletedTasks"), new Gauge<Long>()
-        {
-            public Long getValue()
-            {
-                return messagingPool.gossipChannel.getCompletedMessages();
-            }
-        });
-        gossipMessageDroppedTasks = Metrics.register(factory.createMetricName("GossipMessageDroppedTasks"), new Gauge<Long>()
-        {
-            public Long getValue()
-            {
-                return messagingPool.gossipChannel.getDroppedMessages();
-            }
-        });
-        timeouts = Metrics.meter(factory.createMetricName("Timeouts"));
-    }
-
-    public void release()
-    {
-        Metrics.remove(factory.createMetricName("LargeMessagePendingTasks"));
-        Metrics.remove(factory.createMetricName("LargeMessageCompletedTasks"));
-        Metrics.remove(factory.createMetricName("LargeMessageDroppedTasks"));
-        Metrics.remove(factory.createMetricName("SmallMessagePendingTasks"));
-        Metrics.remove(factory.createMetricName("SmallMessageCompletedTasks"));
-        Metrics.remove(factory.createMetricName("SmallMessageDroppedTasks"));
-        Metrics.remove(factory.createMetricName("GossipMessagePendingTasks"));
-        Metrics.remove(factory.createMetricName("GossipMessageCompletedTasks"));
-        Metrics.remove(factory.createMetricName("GossipMessageDroppedTasks"));
-        Metrics.remove(factory.createMetricName("Timeouts"));
-    }
-}
diff --git a/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java b/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java
index 794fa9c..8c22778 100644
--- a/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java
@@ -21,6 +21,7 @@ import com.codahale.metrics.Meter;
 import com.codahale.metrics.Timer;
 
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.Verb;
 
 import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
 
@@ -38,7 +39,7 @@ public class DroppedMessageMetrics
     /** The cross node dropped latency */
     public final Timer crossNodeDroppedLatency;
 
-    public DroppedMessageMetrics(MessagingService.Verb verb)
+    public DroppedMessageMetrics(Verb verb)
     {
         this(new DefaultNameFactory("DroppedMessage", verb.toString()));
     }
diff --git a/src/java/org/apache/cassandra/metrics/FrequencySampler.java b/src/java/org/apache/cassandra/metrics/FrequencySampler.java
index c094347..8a8918b 100644
--- a/src/java/org/apache/cassandra/metrics/FrequencySampler.java
+++ b/src/java/org/apache/cassandra/metrics/FrequencySampler.java
@@ -26,6 +26,8 @@ import org.slf4j.LoggerFactory;
 
 import com.clearspring.analytics.stream.StreamSummary;
 
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
 /**
  * Find the most frequent sample. A sample adds to the sum of its key ie
  * <p>add("x", 10); and add("x", 20); will result in "x" = 30</p> This uses StreamSummary to only store the
@@ -37,7 +39,7 @@ import com.clearspring.analytics.stream.StreamSummary;
 public abstract class FrequencySampler<T> extends Sampler<T>
 {
     private static final Logger logger = LoggerFactory.getLogger(FrequencySampler.class);
-    private long endTimeMillis = -1;
+    private long endTimeNanos = -1;
 
     private StreamSummary<T> summary;
 
@@ -51,10 +53,10 @@ public abstract class FrequencySampler<T> extends Sampler<T>
      */
     public synchronized void beginSampling(int capacity, int durationMillis)
     {
-        if (endTimeMillis == -1 || clock.currentTimeMillis() > endTimeMillis)
+        if (endTimeNanos == -1 || clock.now() > endTimeNanos)
         {
-            summary = new StreamSummary<T>(capacity);
-            endTimeMillis = clock.currentTimeMillis() + durationMillis;
+            summary = new StreamSummary<>(capacity);
+            endTimeNanos = clock.now() + MILLISECONDS.toNanos(durationMillis);
         }
         else
             throw new RuntimeException("Sampling already in progress");
@@ -67,9 +69,9 @@ public abstract class FrequencySampler<T> extends Sampler<T>
     public synchronized List<Sample<T>> finishSampling(int count)
     {
         List<Sample<T>> results = Collections.emptyList();
-        if (endTimeMillis != -1)
+        if (endTimeNanos != -1)
         {
-            endTimeMillis = -1;
+            endTimeNanos = -1;
             results = summary.topK(count)
                              .stream()
                              .map(c -> new Sample<T>(c.getItem(), c.getCount(), c.getError()))
@@ -82,7 +84,7 @@ public abstract class FrequencySampler<T> extends Sampler<T>
     {
         // samplerExecutor is single threaded but still need
         // synchronization against jmx calls to finishSampling
-        if (value > 0 && clock.currentTimeMillis() <= endTimeMillis)
+        if (value > 0 && clock.now() <= endTimeNanos)
         {
             try
             {
@@ -96,7 +98,7 @@ public abstract class FrequencySampler<T> extends Sampler<T>
 
     public boolean isEnabled()
     {
-        return endTimeMillis != -1 && clock.currentTimeMillis() <= endTimeMillis;
+        return endTimeNanos != -1 && clock.now() <= endTimeNanos;
     }
 
 }
diff --git a/src/java/org/apache/cassandra/metrics/InternodeInboundMetrics.java b/src/java/org/apache/cassandra/metrics/InternodeInboundMetrics.java
new file mode 100644
index 0000000..cc3c1c0
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/InternodeInboundMetrics.java
@@ -0,0 +1,98 @@
+/*
+ * 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.metrics;
+
+import com.codahale.metrics.Gauge;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.InboundMessageHandlers;
+import org.apache.cassandra.metrics.CassandraMetricsRegistry.MetricName;
+
+/**
+ * Metrics for internode connections.
+ */
+public class InternodeInboundMetrics
+{
+    private final MetricName corruptFramesRecovered;
+    private final MetricName corruptFramesUnrecovered;
+    private final MetricName errorBytes;
+    private final MetricName errorCount;
+    private final MetricName expiredBytes;
+    private final MetricName expiredCount;
+    private final MetricName pendingBytes;
+    private final MetricName pendingCount;
+    private final MetricName processedBytes;
+    private final MetricName processedCount;
+    private final MetricName receivedBytes;
+    private final MetricName receivedCount;
+    private final MetricName throttledCount;
+    private final MetricName throttledNanos;
+
+    /**
+     * Create metrics for given inbound message handlers.
+     *
+     * @param peer IP address and port to use for metrics label
+     */
+    public InternodeInboundMetrics(InetAddressAndPort peer, InboundMessageHandlers handlers)
+    {
+        // ipv6 addresses will contain colons, which are invalid in a JMX ObjectName
+        MetricNameFactory factory = new DefaultNameFactory("InboundConnection", peer.toString().replace(':', '_'));
+
+        register(corruptFramesRecovered = factory.createMetricName("CorruptFramesRecovered"), handlers::corruptFramesRecovered);
+        register(corruptFramesUnrecovered = factory.createMetricName("CorruptFramesUnrecovered"), handlers::corruptFramesUnrecovered);
+        register(errorBytes = factory.createMetricName("ErrorBytes"), handlers::errorBytes);
+        register(errorCount = factory.createMetricName("ErrorCount"), handlers::errorCount);
+        register(expiredBytes = factory.createMetricName("ExpiredBytes"), handlers::expiredBytes);
+        register(expiredCount = factory.createMetricName("ExpiredCount"), handlers::expiredCount);
+        register(pendingBytes = factory.createMetricName("ScheduledBytes"), handlers::scheduledBytes);
+        register(pendingCount = factory.createMetricName("ScheduledCount"), handlers::scheduledCount);
+        register(processedBytes = factory.createMetricName("ProcessedBytes"), handlers::processedBytes);
+        register(processedCount = factory.createMetricName("ProcessedCount"), handlers::processedCount);
+        register(receivedBytes = factory.createMetricName("ReceivedBytes"), handlers::receivedBytes);
+        register(receivedCount = factory.createMetricName("ReceivedCount"), handlers::receivedCount);
+        register(throttledCount = factory.createMetricName("ThrottledCount"), handlers::throttledCount);
+        register(throttledNanos = factory.createMetricName("ThrottledNanos"), handlers::throttledNanos);
+    }
+
+    public void release()
+    {
+        remove(corruptFramesRecovered);
+        remove(corruptFramesUnrecovered);
+        remove(errorBytes);
+        remove(errorCount);
+        remove(expiredBytes);
+        remove(expiredCount);
+        remove(pendingBytes);
+        remove(pendingCount);
+        remove(processedBytes);
+        remove(processedCount);
+        remove(receivedBytes);
+        remove(receivedCount);
+        remove(throttledCount);
+        remove(throttledNanos);
+    }
+
+    private static void register(MetricName name, Gauge gauge)
+    {
+        CassandraMetricsRegistry.Metrics.register(name, gauge);
+    }
+
+    private static void remove(MetricName name)
+    {
+        CassandraMetricsRegistry.Metrics.remove(name);
+    }
+}
diff --git a/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java b/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java
new file mode 100644
index 0000000..4e64f66
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java
@@ -0,0 +1,205 @@
+/*
+ * 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.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Meter;
+import org.apache.cassandra.net.OutboundConnections;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
+import org.apache.cassandra.locator.InetAddressAndPort;
+
+/**
+ * Metrics for internode connections.
+ */
+public class InternodeOutboundMetrics
+{
+    public static final String TYPE_NAME = "Connection";
+
+    /** Total number of callbacks that were not completed successfully for messages that were sent to this node
+     * TODO this was always broken, as it never counted those messages without callbacks? So perhaps we can redefine it. */
+    public static final Meter totalExpiredCallbacks = Metrics.meter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalTimeouts", null));
+
+    /** Number of timeouts for specific IP */
+    public final Meter expiredCallbacks;
+
+    public final String address;
+    /** Pending tasks for large message TCP Connections */
+    public final Gauge<Integer> largeMessagePendingTasks;
+    /** Pending bytes for large message TCP Connections */
+    public final Gauge<Long> largeMessagePendingBytes;
+    /** Completed tasks for large message TCP Connections */
+    public final Gauge<Long> largeMessageCompletedTasks;
+    /** Completed bytes for large message TCP Connections */
+    public final Gauge<Long> largeMessageCompletedBytes;
+    /** Dropped tasks for large message TCP Connections */
+    public final Gauge<Long> largeMessageDropped;
+    /** Dropped tasks because of timeout for large message TCP Connections */
+    public final Gauge<Long> largeMessageDroppedTasksDueToTimeout;
+    /** Dropped bytes because of timeout for large message TCP Connections */
+    public final Gauge<Long> largeMessageDroppedBytesDueToTimeout;
+    /** Dropped tasks because of overload for large message TCP Connections */
+    public final Gauge<Long> largeMessageDroppedTasksDueToOverload;
+    /** Dropped bytes because of overload for large message TCP Connections */
+    public final Gauge<Long> largeMessageDroppedBytesDueToOverload;
+    /** Dropped tasks because of error for large message TCP Connections */
+    public final Gauge<Long> largeMessageDroppedTasksDueToError;
+    /** Dropped bytes because of error for large message TCP Connections */
+    public final Gauge<Long> largeMessageDroppedBytesDueToError;
+    /** Pending tasks for small message TCP Connections */
+    public final Gauge<Integer> smallMessagePendingTasks;
+    /** Pending bytes for small message TCP Connections */
+    public final Gauge<Long> smallMessagePendingBytes;
+    /** Completed tasks for small message TCP Connections */
+    public final Gauge<Long> smallMessageCompletedTasks;
+    /** Completed bytes for small message TCP Connections */
+    public final Gauge<Long> smallMessageCompletedBytes;
+    /** Dropped tasks for small message TCP Connections */
+    public final Gauge<Long> smallMessageDroppedTasks;
+    /** Dropped tasks because of timeout for small message TCP Connections */
+    public final Gauge<Long> smallMessageDroppedTasksDueToTimeout;
+    /** Dropped bytes because of timeout for small message TCP Connections */
+    public final Gauge<Long> smallMessageDroppedBytesDueToTimeout;
+    /** Dropped tasks because of overload for small message TCP Connections */
+    public final Gauge<Long> smallMessageDroppedTasksDueToOverload;
+    /** Dropped bytes because of overload for small message TCP Connections */
+    public final Gauge<Long> smallMessageDroppedBytesDueToOverload;
+    /** Dropped tasks because of error for small message TCP Connections */
+    public final Gauge<Long> smallMessageDroppedTasksDueToError;
+    /** Dropped bytes because of error for small message TCP Connections */
+    public final Gauge<Long> smallMessageDroppedBytesDueToError;
+    /** Pending tasks for small message TCP Connections */
+    public final Gauge<Integer> urgentMessagePendingTasks;
+    /** Pending bytes for urgent message TCP Connections */
+    public final Gauge<Long> urgentMessagePendingBytes;
+    /** Completed tasks for urgent message TCP Connections */
+    public final Gauge<Long> urgentMessageCompletedTasks;
+    /** Completed bytes for urgent message TCP Connections */
+    public final Gauge<Long> urgentMessageCompletedBytes;
+    /** Dropped tasks for urgent message TCP Connections */
+    public final Gauge<Long> urgentMessageDroppedTasks;
+    /** Dropped tasks because of timeout for urgent message TCP Connections */
+    public final Gauge<Long> urgentMessageDroppedTasksDueToTimeout;
+    /** Dropped bytes because of timeout for urgent message TCP Connections */
+    public final Gauge<Long> urgentMessageDroppedBytesDueToTimeout;
+    /** Dropped tasks because of overload for urgent message TCP Connections */
+    public final Gauge<Long> urgentMessageDroppedTasksDueToOverload;
+    /** Dropped bytes because of overload for urgent message TCP Connections */
+    public final Gauge<Long> urgentMessageDroppedBytesDueToOverload;
+    /** Dropped tasks because of error for urgent message TCP Connections */
+    public final Gauge<Long> urgentMessageDroppedTasksDueToError;
+    /** Dropped bytes because of error for urgent message TCP Connections */
+    public final Gauge<Long> urgentMessageDroppedBytesDueToError;
+    
+    private final MetricNameFactory factory;
+
+    /**
+     * Create metrics for given connection pool.
+     *
+     * @param ip IP address to use for metrics label
+     */
+    public InternodeOutboundMetrics(InetAddressAndPort ip, final OutboundConnections messagingPool)
+    {
+        // ipv6 addresses will contain colons, which are invalid in a JMX ObjectName
+        address = ip.toString().replace(':', '_');
+
+        factory = new DefaultNameFactory("Connection", address);
+
+        largeMessagePendingTasks = Metrics.register(factory.createMetricName("LargeMessagePendingTasks"), messagingPool.large::pendingCount);
+        largeMessagePendingBytes = Metrics.register(factory.createMetricName("LargeMessagePendingBytes"), messagingPool.large::pendingBytes);
+        largeMessageCompletedTasks = Metrics.register(factory.createMetricName("LargeMessageCompletedTasks"),messagingPool.large::sentCount);
+        largeMessageCompletedBytes = Metrics.register(factory.createMetricName("LargeMessageCompletedBytes"),messagingPool.large::sentBytes);
+        largeMessageDropped = Metrics.register(factory.createMetricName("LargeMessageDroppedTasks"), messagingPool.large::dropped);
+        largeMessageDroppedTasksDueToOverload = Metrics.register(factory.createMetricName("LargeMessageDroppedTasksDueToOverload"), messagingPool.large::overloadedBytes);
+        largeMessageDroppedBytesDueToOverload = Metrics.register(factory.createMetricName("LargeMessageDroppedBytesDueToOverload"), messagingPool.large::overloadedCount);
+        largeMessageDroppedTasksDueToTimeout = Metrics.register(factory.createMetricName("LargeMessageDroppedTasksDueToTimeout"), messagingPool.large::expiredCount);
+        largeMessageDroppedBytesDueToTimeout = Metrics.register(factory.createMetricName("LargeMessageDroppedBytesDueToTimeout"), messagingPool.large::expiredBytes);
+        largeMessageDroppedTasksDueToError = Metrics.register(factory.createMetricName("LargeMessageDroppedTasksDueToError"), messagingPool.large::errorCount);
+        largeMessageDroppedBytesDueToError = Metrics.register(factory.createMetricName("LargeMessageDroppedBytesDueToError"), messagingPool.large::errorBytes);
+        smallMessagePendingTasks = Metrics.register(factory.createMetricName("SmallMessagePendingTasks"), messagingPool.small::pendingCount);
+        smallMessagePendingBytes = Metrics.register(factory.createMetricName("SmallMessagePendingBytes"), messagingPool.small::pendingBytes);
+        smallMessageCompletedTasks = Metrics.register(factory.createMetricName("SmallMessageCompletedTasks"), messagingPool.small::sentCount);
+        smallMessageCompletedBytes = Metrics.register(factory.createMetricName("SmallMessageCompletedBytes"),messagingPool.small::sentBytes);
+        smallMessageDroppedTasks = Metrics.register(factory.createMetricName("SmallMessageDroppedTasks"), messagingPool.small::dropped);
+        smallMessageDroppedTasksDueToOverload = Metrics.register(factory.createMetricName("SmallMessageDroppedTasksDueToOverload"), messagingPool.small::overloadedCount);
+        smallMessageDroppedBytesDueToOverload = Metrics.register(factory.createMetricName("SmallMessageDroppedBytesDueToOverload"), messagingPool.small::overloadedBytes);
+        smallMessageDroppedTasksDueToTimeout = Metrics.register(factory.createMetricName("SmallMessageDroppedTasksDueToTimeout"), messagingPool.small::expiredCount);
+        smallMessageDroppedBytesDueToTimeout = Metrics.register(factory.createMetricName("SmallMessageDroppedBytesDueToTimeout"), messagingPool.small::expiredBytes);
+        smallMessageDroppedTasksDueToError = Metrics.register(factory.createMetricName("SmallMessageDroppedTasksDueToError"), messagingPool.small::errorCount);
+        smallMessageDroppedBytesDueToError = Metrics.register(factory.createMetricName("SmallMessageDroppedBytesDueToError"), messagingPool.small::errorBytes);
+        urgentMessagePendingTasks = Metrics.register(factory.createMetricName("UrgentMessagePendingTasks"), messagingPool.urgent::pendingCount);
+        urgentMessagePendingBytes = Metrics.register(factory.createMetricName("UrgentMessagePendingBytes"), messagingPool.urgent::pendingBytes);
+        urgentMessageCompletedTasks = Metrics.register(factory.createMetricName("UrgentMessageCompletedTasks"), messagingPool.urgent::sentCount);
+        urgentMessageCompletedBytes = Metrics.register(factory.createMetricName("UrgentMessageCompletedBytes"),messagingPool.urgent::sentBytes);
+        urgentMessageDroppedTasks = Metrics.register(factory.createMetricName("UrgentMessageDroppedTasks"), messagingPool.urgent::dropped);
+        urgentMessageDroppedTasksDueToOverload = Metrics.register(factory.createMetricName("UrgentMessageDroppedTasksDueToOverload"), messagingPool.urgent::overloadedBytes);
+        urgentMessageDroppedBytesDueToOverload = Metrics.register(factory.createMetricName("UrgentMessageDroppedBytesDueToOverload"), messagingPool.urgent::overloadedCount);
+        urgentMessageDroppedTasksDueToTimeout = Metrics.register(factory.createMetricName("UrgentMessageDroppedTasksDueToTimeout"), messagingPool.urgent::expiredCount);
+        urgentMessageDroppedBytesDueToTimeout = Metrics.register(factory.createMetricName("UrgentMessageDroppedBytesDueToTimeout"), messagingPool.urgent::expiredBytes);
+        urgentMessageDroppedTasksDueToError = Metrics.register(factory.createMetricName("UrgentMessageDroppedTasksDueToError"), messagingPool.urgent::errorCount);
+        urgentMessageDroppedBytesDueToError = Metrics.register(factory.createMetricName("UrgentMessageDroppedBytesDueToError"), messagingPool.urgent::errorBytes);
+        expiredCallbacks = Metrics.meter(factory.createMetricName("Timeouts"));
+
+        // deprecated
+        Metrics.register(factory.createMetricName("GossipMessagePendingTasks"), (Gauge<Integer>) messagingPool.urgent::pendingCount);
+        Metrics.register(factory.createMetricName("GossipMessageCompletedTasks"), (Gauge<Long>) messagingPool.urgent::sentCount);
+        Metrics.register(factory.createMetricName("GossipMessageDroppedTasks"), (Gauge<Long>) messagingPool.urgent::dropped);
+    }
+
+    public void release()
+    {
+        Metrics.remove(factory.createMetricName("LargeMessagePendingTasks"));
+        Metrics.remove(factory.createMetricName("LargeMessagePendingBytes"));
+        Metrics.remove(factory.createMetricName("LargeMessageCompletedTasks"));
+        Metrics.remove(factory.createMetricName("LargeMessageCompletedBytes"));
+        Metrics.remove(factory.createMetricName("LargeMessageDroppedTasks"));
+        Metrics.remove(factory.createMetricName("LargeMessageDroppedTasksDueToTimeout"));
+        Metrics.remove(factory.createMetricName("LargeMessageDroppedBytesDueToTimeout"));
+        Metrics.remove(factory.createMetricName("LargeMessageDroppedTasksDueToOverload"));
+        Metrics.remove(factory.createMetricName("LargeMessageDroppedBytesDueToOverload"));
+        Metrics.remove(factory.createMetricName("LargeMessageDroppedTasksDueToError"));
+        Metrics.remove(factory.createMetricName("LargeMessageDroppedBytesDueToError"));
+        Metrics.remove(factory.createMetricName("SmallMessagePendingTasks"));
+        Metrics.remove(factory.createMetricName("SmallMessagePendingBytes"));
+        Metrics.remove(factory.createMetricName("SmallMessageCompletedTasks"));
+        Metrics.remove(factory.createMetricName("SmallMessageCompletedBytes"));
+        Metrics.remove(factory.createMetricName("SmallMessageDroppedTasks"));
+        Metrics.remove(factory.createMetricName("SmallMessageDroppedTasksDueToTimeout"));
+        Metrics.remove(factory.createMetricName("SmallMessageDroppedBytesDueToTimeout"));
+        Metrics.remove(factory.createMetricName("SmallMessageDroppedTasksDueToOverload"));
+        Metrics.remove(factory.createMetricName("SmallMessageDroppedBytesDueToOverload"));
+        Metrics.remove(factory.createMetricName("SmallMessageDroppedTasksDueToError"));
+        Metrics.remove(factory.createMetricName("SmallMessageDroppedBytesDueToError"));
+        Metrics.remove(factory.createMetricName("GossipMessagePendingTasks"));
+        Metrics.remove(factory.createMetricName("GossipMessageCompletedTasks"));
+        Metrics.remove(factory.createMetricName("GossipMessageDroppedTasks"));
+        Metrics.remove(factory.createMetricName("UrgentMessagePendingTasks"));
+        Metrics.remove(factory.createMetricName("UrgentMessagePendingBytes"));
+        Metrics.remove(factory.createMetricName("UrgentMessageCompletedTasks"));
+        Metrics.remove(factory.createMetricName("UrgentMessageCompletedBytes"));
+        Metrics.remove(factory.createMetricName("UrgentMessageDroppedTasks"));
+        Metrics.remove(factory.createMetricName("UrgentMessageDroppedTasksDueToTimeout"));
+        Metrics.remove(factory.createMetricName("UrgentMessageDroppedBytesDueToTimeout"));
+        Metrics.remove(factory.createMetricName("UrgentMessageDroppedTasksDueToOverload"));
+        Metrics.remove(factory.createMetricName("UrgentMessageDroppedBytesDueToOverload"));
+        Metrics.remove(factory.createMetricName("UrgentMessageDroppedTasksDueToError"));
+        Metrics.remove(factory.createMetricName("UrgentMessageDroppedBytesDueToError"));
+        Metrics.remove(factory.createMetricName("Timeouts"));
+    }
+}
diff --git a/src/java/org/apache/cassandra/metrics/MaxSampler.java b/src/java/org/apache/cassandra/metrics/MaxSampler.java
index f4fb873..df24bb9 100644
--- a/src/java/org/apache/cassandra/metrics/MaxSampler.java
+++ b/src/java/org/apache/cassandra/metrics/MaxSampler.java
@@ -24,23 +24,25 @@ import java.util.List;
 
 import com.google.common.collect.MinMaxPriorityQueue;
 
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
 public abstract class MaxSampler<T> extends Sampler<T>
 {
     private int capacity;
     private MinMaxPriorityQueue<Sample<T>> queue;
-    private long endTimeMillis = -1;
+    private long endTimeNanos = -1;
     private final Comparator<Sample<T>> comp = Collections.reverseOrder(Comparator.comparing(p -> p.count));
 
     public boolean isEnabled()
     {
-        return endTimeMillis != -1 && clock.currentTimeMillis() <= endTimeMillis;
+        return endTimeNanos != -1 && clock.now() <= endTimeNanos;
     }
 
     public synchronized void beginSampling(int capacity, int durationMillis)
     {
-        if (endTimeMillis == -1 || clock.currentTimeMillis() > endTimeMillis)
+        if (endTimeNanos == -1 || clock.now() > endTimeNanos)
         {
-            endTimeMillis = clock.currentTimeMillis() + durationMillis;
+            endTimeNanos = clock.now() + MILLISECONDS.toNanos(durationMillis);
             queue = MinMaxPriorityQueue
                     .orderedBy(comp)
                     .maximumSize(Math.max(1, capacity))
@@ -54,9 +56,9 @@ public abstract class MaxSampler<T> extends Sampler<T>
     public synchronized List<Sample<T>> finishSampling(int count)
     {
         List<Sample<T>> result = new ArrayList<>(count);
-        if (endTimeMillis != -1)
+        if (endTimeNanos != -1)
         {
-            endTimeMillis = -1;
+            endTimeNanos = -1;
             Sample<T> next;
             while ((next = queue.poll()) != null && result.size() <= count)
                 result.add(next);
@@ -67,7 +69,7 @@ public abstract class MaxSampler<T> extends Sampler<T>
     @Override
     protected synchronized void insert(T item, long value)
     {
-        if (value > 0 && clock.currentTimeMillis() <= endTimeMillis
+        if (value > 0 && clock.now() <= endTimeNanos
                 && (queue.isEmpty() || queue.size() < capacity || queue.peekLast().count < value))
             queue.add(new Sample<T>(item, value, 0));
     }
diff --git a/src/java/org/apache/cassandra/metrics/MessagingMetrics.java b/src/java/org/apache/cassandra/metrics/MessagingMetrics.java
index 2f096f6..0ea2e10 100644
--- a/src/java/org/apache/cassandra/metrics/MessagingMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/MessagingMetrics.java
@@ -17,59 +17,215 @@
  */
 package org.apache.cassandra.metrics;
 
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
+import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.config.DatabaseDescriptor;
+
 import com.codahale.metrics.Timer;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.Message;
+import org.apache.cassandra.net.Verb;
+import org.apache.cassandra.net.InboundMessageHandlers;
+import org.apache.cassandra.net.LatencyConsumer;
+import org.apache.cassandra.utils.StatusLogger;
 
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
 
 /**
  * Metrics for messages
  */
-public class MessagingMetrics
+public class MessagingMetrics implements InboundMessageHandlers.GlobalMetricCallbacks
 {
-    private static Logger logger = LoggerFactory.getLogger(MessagingMetrics.class);
     private static final MetricNameFactory factory = new DefaultNameFactory("Messaging");
-    public final Timer crossNodeLatency;
-    public final ConcurrentHashMap<String, Timer> dcLatency;
-    public final ConcurrentHashMap<String, Timer> queueWaitLatency;
+    private static final Logger logger = LoggerFactory.getLogger(MessagingMetrics.class);
+    private static final int LOG_DROPPED_INTERVAL_IN_MS = 5000;
+    
+    public static class DCLatencyRecorder implements LatencyConsumer
+    {
+        public final Timer dcLatency;
+        public final Timer allLatency;
+
+        DCLatencyRecorder(Timer dcLatency, Timer allLatency)
+        {
+            this.dcLatency = dcLatency;
+            this.allLatency = allLatency;
+        }
+
+        public void accept(long timeTaken, TimeUnit units)
+        {
+            if (timeTaken > 0)
+            {
+                dcLatency.update(timeTaken, units);
+                allLatency.update(timeTaken, units);
+            }
+        }
+    }
+
+    private static final class DroppedForVerb
+    {
+        final DroppedMessageMetrics metrics;
+        final AtomicInteger droppedFromSelf;
+        final AtomicInteger droppedFromPeer;
+
+        DroppedForVerb(Verb verb)
+        {
+            this(new DroppedMessageMetrics(verb));
+        }
+
+        DroppedForVerb(DroppedMessageMetrics metrics)
+        {
+            this.metrics = metrics;
+            this.droppedFromSelf = new AtomicInteger(0);
+            this.droppedFromPeer = new AtomicInteger(0);
+        }
+    }
+
+    private final Timer allLatency;
+    public final ConcurrentHashMap<String, DCLatencyRecorder> dcLatency;
+    public final EnumMap<Verb, Timer> internalLatency;
+
+    // total dropped message counts for server lifetime
+    private final Map<Verb, DroppedForVerb> droppedMessages = new EnumMap<>(Verb.class);
 
     public MessagingMetrics()
     {
-        crossNodeLatency = Metrics.timer(factory.createMetricName("CrossNodeLatency"));
+        allLatency = Metrics.timer(factory.createMetricName("CrossNodeLatency"));
         dcLatency = new ConcurrentHashMap<>();
-        queueWaitLatency = new ConcurrentHashMap<>();
+        internalLatency = new EnumMap<>(Verb.class);
+        for (Verb verb : Verb.VERBS)
+            internalLatency.put(verb, Metrics.timer(factory.createMetricName(verb + "-WaitLatency")));
+        for (Verb verb : Verb.values())
+            droppedMessages.put(verb, new DroppedForVerb(verb));
+    }
+
+    public DCLatencyRecorder internodeLatencyRecorder(InetAddressAndPort from)
+    {
+        String dcName = DatabaseDescriptor.getEndpointSnitch().getDatacenter(from);
+        DCLatencyRecorder dcUpdater = dcLatency.get(dcName);
+        if (dcUpdater == null)
+            dcUpdater = dcLatency.computeIfAbsent(dcName, k -> new DCLatencyRecorder(Metrics.timer(factory.createMetricName(dcName + "-Latency")), allLatency));
+        return dcUpdater;
+    }
+
+    public void recordInternalLatency(Verb verb, long timeTaken, TimeUnit units)
+    {
+        if (timeTaken > 0)
+            internalLatency.get(verb).update(timeTaken, units);
+    }
+
+    public void recordSelfDroppedMessage(Verb verb)
+    {
+        recordDroppedMessage(droppedMessages.get(verb), false);
     }
 
-    public void addTimeTaken(InetAddressAndPort from, long timeTaken)
+    public void recordSelfDroppedMessage(Verb verb, long timeElapsed, TimeUnit timeUnit)
     {
-        String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(from);
-        Timer timer = dcLatency.get(dc);
-        if (timer == null)
+        recordDroppedMessage(verb, timeElapsed, timeUnit, false);
+    }
+
+    public void recordInternodeDroppedMessage(Verb verb, long timeElapsed, TimeUnit timeUnit)
+    {
+        recordDroppedMessage(verb, timeElapsed, timeUnit, true);
+    }
+
+    public void recordDroppedMessage(Message<?> message, long timeElapsed, TimeUnit timeUnit)
+    {
+        recordDroppedMessage(message.verb(), timeElapsed, timeUnit, message.isCrossNode());
+    }
+
+    public void recordDroppedMessage(Verb verb, long timeElapsed, TimeUnit timeUnit, boolean isCrossNode)
+    {
+        recordDroppedMessage(droppedMessages.get(verb), timeElapsed, timeUnit, isCrossNode);
+    }
+
+    private static void recordDroppedMessage(DroppedForVerb droppedMessages, long timeTaken, TimeUnit units, boolean isCrossNode)
+    {
+        if (isCrossNode)
+            droppedMessages.metrics.crossNodeDroppedLatency.update(timeTaken, units);
+        else
+            droppedMessages.metrics.internalDroppedLatency.update(timeTaken, units);
+        recordDroppedMessage(droppedMessages, isCrossNode);
+    }
+
+    private static void recordDroppedMessage(DroppedForVerb droppedMessages, boolean isCrossNode)
+    {
+        droppedMessages.metrics.dropped.mark();
+        if (isCrossNode)
+            droppedMessages.droppedFromPeer.incrementAndGet();
+        else
+            droppedMessages.droppedFromSelf.incrementAndGet();
+    }
+
+    public void scheduleLogging()
+    {
+        ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::logDroppedMessages,
+                                                                 LOG_DROPPED_INTERVAL_IN_MS,
+                                                                 LOG_DROPPED_INTERVAL_IN_MS,
+                                                                 MILLISECONDS);
+    }
+
+    public Map<String, Integer> getDroppedMessages()
+    {
+        Map<String, Integer> map = new HashMap<>(droppedMessages.size());
+        for (Map.Entry<Verb, DroppedForVerb> entry : droppedMessages.entrySet())
+            map.put(entry.getKey().toString(), (int) entry.getValue().metrics.dropped.getCount());
+        return map;
+    }
+
+    private void logDroppedMessages()
+    {
+        if (resetAndConsumeDroppedErrors(logger::info) > 0)
+            StatusLogger.log();
+    }
+
+    @VisibleForTesting
+    public int resetAndConsumeDroppedErrors(Consumer<String> messageConsumer)
+    {
+        int count = 0;
+        for (Map.Entry<Verb, DroppedForVerb> entry : droppedMessages.entrySet())
         {
-            timer = dcLatency.computeIfAbsent(dc, k -> Metrics.timer(factory.createMetricName(dc + "-Latency")));
+            Verb verb = entry.getKey();
+            DroppedForVerb droppedForVerb = entry.getValue();
+
+            int droppedInternal = droppedForVerb.droppedFromSelf.getAndSet(0);
+            int droppedCrossNode = droppedForVerb.droppedFromPeer.getAndSet(0);
+            if (droppedInternal > 0 || droppedCrossNode > 0)
+            {
+                messageConsumer.accept(String.format("%s messages were dropped in last %d ms: %d internal and %d cross node."
+                                      + " Mean internal dropped latency: %d ms and Mean cross-node dropped latency: %d ms",
+                                      verb,
+                                      LOG_DROPPED_INTERVAL_IN_MS,
+                                      droppedInternal,
+                                      droppedCrossNode,
+                                      TimeUnit.NANOSECONDS.toMillis((long) droppedForVerb.metrics.internalDroppedLatency.getSnapshot().getMean()),
+                                      TimeUnit.NANOSECONDS.toMillis((long) droppedForVerb.metrics.crossNodeDroppedLatency.getSnapshot().getMean())));
+                ++count;
+            }
         }
-        timer.update(timeTaken, TimeUnit.MILLISECONDS);
-        crossNodeLatency.update(timeTaken, TimeUnit.MILLISECONDS);
+        return count;
     }
 
-    public void addQueueWaitTime(String verb, long timeTaken)
+    @VisibleForTesting
+    public void resetDroppedMessages(String scope)
     {
-        if (timeTaken < 0)
-            // the measurement is not accurate, ignore the negative timeTaken
-            return;
-
-        Timer timer = queueWaitLatency.get(verb);
-        if (timer == null)
+        for (Verb verb : droppedMessages.keySet())
         {
-            timer = queueWaitLatency.computeIfAbsent(verb, k -> Metrics.timer(factory.createMetricName(verb + "-WaitLatency")));
+            droppedMessages.put(verb, new DroppedForVerb(new DroppedMessageMetrics(metricName ->
+                                                                                      new CassandraMetricsRegistry.MetricName("DroppedMessages", metricName, scope)
+            )));
         }
-        timer.update(timeTaken, TimeUnit.MILLISECONDS);
     }
+
 }
diff --git a/src/java/org/apache/cassandra/metrics/Sampler.java b/src/java/org/apache/cassandra/metrics/Sampler.java
index 4bff332..cfe3f3b 100644
--- a/src/java/org/apache/cassandra/metrics/Sampler.java
+++ b/src/java/org/apache/cassandra/metrics/Sampler.java
@@ -26,8 +26,8 @@ import java.util.concurrent.TimeUnit;
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.net.MessagingService.Verb;
-import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.net.Verb;
+import org.apache.cassandra.utils.MonotonicClock;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -39,7 +39,7 @@ public abstract class Sampler<T>
     }
 
     @VisibleForTesting
-    Clock clock = Clock.instance;
+    MonotonicClock clock = MonotonicClock.approxTime;
 
     @VisibleForTesting
     static final ThreadPoolExecutor samplerExecutor = new JMXEnabledThreadPoolExecutor(1, 1,
@@ -52,7 +52,7 @@ public abstract class Sampler<T>
     {
         samplerExecutor.setRejectedExecutionHandler((runnable, executor) ->
         {
-            MessagingService.instance().incrementDroppedMessages(Verb._SAMPLE);
+            MessagingService.instance().metrics.recordSelfDroppedMessage(Verb._SAMPLE);
         });
     }
 
diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java
index c854c43..beb6d2e 100644
--- a/src/java/org/apache/cassandra/metrics/TableMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java
@@ -202,7 +202,7 @@ public class TableMetrics
     /** Time spent waiting for free memtable space, either on- or off-heap */
     public final Histogram waitingOnFreeMemtableSpace;
 
-    /** Dropped Mutations Count */
+    @Deprecated
     public final Counter droppedMutations;
 
     private final MetricNameFactory factory;
diff --git a/test/unit/org/apache/cassandra/net/Matcher.java b/src/java/org/apache/cassandra/net/AcceptVersions.java
similarity index 65%
copy from test/unit/org/apache/cassandra/net/Matcher.java
copy to src/java/org/apache/cassandra/net/AcceptVersions.java
index 27b685f..61ae049 100644
--- a/test/unit/org/apache/cassandra/net/Matcher.java
+++ b/src/java/org/apache/cassandra/net/AcceptVersions.java
@@ -17,16 +17,26 @@
  */
 package org.apache.cassandra.net;
 
-import org.apache.cassandra.locator.InetAddressAndPort;
-
 /**
- * Predicate based on intercepted, outgoing messange and the message's destination address.
+ * Encapsulates minimum and maximum messaging versions that a node accepts.
  */
-public interface Matcher<T>
+class AcceptVersions
 {
-    /**
-     * @param obj intercepted outgoing message
-     * @param to  destination address
-     */
-    public boolean matches(MessageOut<T> obj, InetAddressAndPort to);
+    final int min, max;
+
+    AcceptVersions(int min, int max)
+    {
+        this.min = min;
+        this.max = max;
+    }
+
+    @Override
+    public boolean equals(Object that)
+    {
+        if (!(that instanceof AcceptVersions))
+            return false;
+
+        return min == ((AcceptVersions) that).min
+            && max == ((AcceptVersions) that).max;
+    }
 }
diff --git a/src/java/org/apache/cassandra/net/AsyncChannelOutputPlus.java b/src/java/org/apache/cassandra/net/AsyncChannelOutputPlus.java
new file mode 100644
index 0000000..163981c
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/AsyncChannelOutputPlus.java
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.net;
+
+import java.io.IOException;
+import java.nio.channels.WritableByteChannel;
+import java.util.concurrent.locks.LockSupport;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelPromise;
+import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+
+import static java.lang.Math.max;
+
+/**
+ * A {@link DataOutputStreamPlus} that writes ASYNCHRONOUSLY to a Netty Channel.
+ *
+ * The close() and flush() methods synchronously wait for pending writes, and will propagate any exceptions
+ * encountered in writing them to the wire.
+ *
+ * The correctness of this class depends on the ChannelPromise we create against a Channel always being completed,
+ * which appears to be a guarantee provided by Netty so long as the event loop is running.
+ *
+ * There are two logical threads accessing the state in this class: the eventLoop of the channel, and the writer
+ * (the writer thread may change, so long as only one utilises the class at any time).
+ * Each thread has exclusive write access to certain state in the class, with the other thread only viewing the state,
+ * simplifying concurrency considerations.
+ */
+public abstract class AsyncChannelOutputPlus extends BufferedDataOutputStreamPlus
+{
+    public static class FlushException extends IOException
+    {
+        public FlushException(String message)
+        {
+            super(message);
+        }
+
+        public FlushException(String message, Throwable cause)
+        {
+            super(message, cause);
+        }
+    }
+
+    final Channel channel;
+
+    /** the number of bytes we have begun flushing; updated only by writer */
+    private volatile long flushing;
+    /** the number of bytes we have finished flushing, successfully or otherwise; updated only by eventLoop */
+    private volatile long flushed;
+    /** the number of bytes we have finished flushing to the network; updated only by eventLoop */
+    private          long flushedToNetwork;
+    /** any error that has been thrown during a flush; updated only by eventLoop */
+    private volatile Throwable flushFailed;
+
+    /**
+     * state for pausing until flushing has caught up - store the number of bytes we need to be flushed before
+     * we should be signalled, and store ourselves in {@link #waiting}; once the flushing thread exceeds this many
+     * total bytes flushed, any Thread stored in waiting will be signalled.
+     *
+     * This works exactly like using a WaitQueue, except that we only need to manage a single waiting thread.
+     */
+    private volatile long signalWhenFlushed; // updated only by writer
+    private volatile Thread waiting; // updated only by writer
+
+    public AsyncChannelOutputPlus(Channel channel)
+    {
+        super(null, null);
+        this.channel = channel;
+    }
+
+    /**
+     * Create a ChannelPromise for a flush of the given size.
+     * <p>
+     * This method will not return until the write is permitted by the provided watermarks and in flight bytes,
+     * and on its completion will mark the requested bytes flushed.
+     * <p>
+     * If this method returns normally, the ChannelPromise MUST be writtenAndFlushed, or else completed exceptionally.
+     */
+    protected ChannelPromise beginFlush(int byteCount, int lowWaterMark, int highWaterMark) throws IOException
+    {
+        waitForSpace(byteCount, lowWaterMark, highWaterMark);
+
+        return AsyncChannelPromise.withListener(channel, future -> {
+            if (future.isSuccess() && null == flushFailed)
+            {
+                flushedToNetwork += byteCount;
+                releaseSpace(byteCount);
+            }
+            else if (null == flushFailed)
+            {
+                Throwable cause = future.cause();
+                if (cause == null)
+                {
+                    cause = new FlushException("Flush failed for unknown reason");
+                    cause.fillInStackTrace();
+                }
+                flushFailed = cause;
+                releaseSpace(flushing - flushed);
+            }
+            else
+            {
+                assert flushing == flushed;
+            }
+        });
+    }
+
+    /**
+     * Imposes our lowWaterMark/highWaterMark constraints, and propagates any exceptions thrown by prior flushes.
+     *
+     * If we currently have lowWaterMark or fewer bytes flushing, we are good to go.
+     * If our new write will not take us over our highWaterMark, we are good to go.
+     * Otherwise we wait until either of these conditions are met.
+     *
+     * This may only be invoked by the writer thread, never by the eventLoop.
+     *
+     * @throws IOException if a prior asynchronous flush failed
+     */
+    private void waitForSpace(int bytesToWrite, int lowWaterMark, int highWaterMark) throws IOException
+    {
+        // decide when we would be willing to carry on writing
+        // we are always writable if we have lowWaterMark or fewer bytes, no matter how many bytes we are flushing
+        // our callers should not be supplying more than (highWaterMark - lowWaterMark) bytes, but we must work correctly if they do
+        int wakeUpWhenFlushing = highWaterMark - bytesToWrite;
+        waitUntilFlushed(max(lowWaterMark, wakeUpWhenFlushing), lowWaterMark);
+        flushing += bytesToWrite;
+    }
+
+    /**
+     * Implementation of waitForSpace, which calculates what flushed points we need to wait for,
+     * parks if necessary and propagates flush failures.
+     *
+     * This may only be invoked by the writer thread, never by the eventLoop.
+     */
+    void waitUntilFlushed(int wakeUpWhenExcessBytesWritten, int signalWhenExcessBytesWritten) throws IOException
+    {
+        // we assume that we are happy to wake up at least as early as we will be signalled; otherwise we will never exit
+        assert signalWhenExcessBytesWritten <= wakeUpWhenExcessBytesWritten;
+        // flushing shouldn't change during this method invocation, so our calculations for signal and flushed are consistent
+        long wakeUpWhenFlushed = flushing - wakeUpWhenExcessBytesWritten;
+        if (flushed < wakeUpWhenFlushed)
+            parkUntilFlushed(wakeUpWhenFlushed, flushing - signalWhenExcessBytesWritten);
+        propagateFailedFlush();
+    }
+
+    /**
+     * Utility method for waitUntilFlushed, which actually parks the current thread until the necessary
+     * number of bytes have been flushed
+     *
+     * This may only be invoked by the writer thread, never by the eventLoop.
+     */
+    protected void parkUntilFlushed(long wakeUpWhenFlushed, long signalWhenFlushed)
+    {
+        assert wakeUpWhenFlushed <= signalWhenFlushed;
+        assert waiting == null;
+        this.waiting = Thread.currentThread();
+        this.signalWhenFlushed = signalWhenFlushed;
+
+        while (flushed < wakeUpWhenFlushed)
+            LockSupport.park();
+        waiting = null;
+    }
+
+    /**
+     * Update our flushed count, and signal any waiters.
+     *
+     * This may only be invoked by the eventLoop, never by the writer thread.
+     */
+    protected void releaseSpace(long bytesFlushed)
+    {
+        long newFlushed = flushed + bytesFlushed;
+        flushed = newFlushed;
+
+        Thread thread = waiting;
+        if (thread != null && signalWhenFlushed <= newFlushed)
+            LockSupport.unpark(thread);
+    }
+
+    private void propagateFailedFlush() throws IOException
+    {
+        Throwable t = flushFailed;
+        if (t != null)
+        {
+            if (SocketFactory.isCausedByConnectionReset(t))
+                throw new FlushException("The channel this output stream was writing to has been closed", t);
+            throw new FlushException("This output stream is in an unsafe state after an asynchronous flush failed", t);
+        }
+    }
+
+    @Override
+    abstract protected void doFlush(int count) throws IOException;
+
+    abstract public long position();
+
+    public long flushed()
+    {
+        // external flushed (that which has had flush() invoked implicitly or otherwise) == internal flushing
+        return flushing;
+    }
+
+    public long flushedToNetwork()
+    {
+        return flushedToNetwork;
+    }
+
+    /**
+     * Perform an asynchronous flush, then waits until all outstanding flushes have completed
+     *
+     * @throws IOException if any flush fails
+     */
+    @Override
+    public void flush() throws IOException
+    {
+        doFlush(0);
+        waitUntilFlushed(0, 0);
+    }
+
+    /**
+     * Flush any remaining writes, and release any buffers.
+     *
+     * The channel is not closed, as it is assumed to be managed externally.
+     *
+     * WARNING: This method requires mutual exclusivity with all other producer methods to run safely.
+     * It should only be invoked by the owning thread, never the eventLoop; the eventLoop should propagate
+     * errors to {@link #flushFailed}, which will propagate them to the producer thread no later than its
+     * final invocation to {@link #close()} or {@link #flush()} (that must not be followed by any further writes).
+     */
+    @Override
+    public void close() throws IOException
+    {
+        try
+        {
+            flush();
+        }
+        finally
+        {
+            discard();
+        }
+    }
+
+    /**
+     * Discard any buffered data, and the buffers that contain it.
+     * May be invoked instead of {@link #close()} if we terminate exceptionally.
+     */
+    public abstract void discard();
+
+    @Override
+    protected WritableByteChannel newDefaultChannel()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/net/AsyncChannelPromise.java b/src/java/org/apache/cassandra/net/AsyncChannelPromise.java
new file mode 100644
index 0000000..d2c9d0b
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/AsyncChannelPromise.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.net;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelPromise;
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.GenericFutureListener;
+
+/**
+ * See {@link AsyncPromise} and {@link io.netty.channel.ChannelPromise}
+ *
+ * This class is all boiler plate, just ensuring we return ourselves and invoke the correct Promise method.
+ */
+public class AsyncChannelPromise extends AsyncPromise<Void> implements ChannelPromise
+{
+    private final Channel channel;
+
+    @SuppressWarnings("unused")
+    public AsyncChannelPromise(Channel channel)
+    {
+        super(channel.eventLoop());
+        this.channel = channel;
+    }
+
+    AsyncChannelPromise(Channel channel, GenericFutureListener<? extends Future<? super Void>> listener)
+    {
+        super(channel.eventLoop(), listener);
+        this.channel = channel;
+    }
+
+    public static AsyncChannelPromise withListener(ChannelHandlerContext context, GenericFutureListener<? extends Future<? super Void>> listener)
+    {
+        return withListener(context.channel(), listener);
+    }
+
+    public static AsyncChannelPromise withListener(Channel channel, GenericFutureListener<? extends Future<? super Void>> listener)
+    {
+        return new AsyncChannelPromise(channel, listener);
+    }
+
+    public static ChannelFuture writeAndFlush(ChannelHandlerContext context, Object message, GenericFutureListener<? extends Future<? super Void>> listener)
+    {
+        return context.writeAndFlush(message, withListener(context.channel(), listener));
+    }
+
+    public static ChannelFuture writeAndFlush(Channel channel, Object message, GenericFutureListener<? extends Future<? super Void>> listener)
+    {
+        return channel.writeAndFlush(message, withListener(channel, listener));
+    }
+
+    public static ChannelFuture writeAndFlush(ChannelHandlerContext context, Object message)
+    {
+        return context.writeAndFlush(message, new AsyncChannelPromise(context.channel()));
+    }
+
+    public static ChannelFuture writeAndFlush(Channel channel, Object message)
+    {
+        return channel.writeAndFlush(message, new AsyncChannelPromise(channel));
+    }
+
+    public Channel channel()
+    {
+        return channel;
+    }
+
+    public boolean isVoid()
+    {
+        return false;
+    }
+
+    public ChannelPromise setSuccess()
+    {
+        return setSuccess(null);
+    }
+
+    public ChannelPromise setSuccess(Void v)
+    {
+        super.setSuccess(v);
+        return this;
+    }
+
+    public boolean trySuccess()
+    {
+        return trySuccess(null);
+    }
+
+    public ChannelPromise setFailure(Throwable throwable)
+    {
+        super.setFailure(throwable);
+        return this;
+    }
+
+    public ChannelPromise sync() throws InterruptedException
+    {
+        super.sync();
+        return this;
+    }
+
+    public ChannelPromise syncUninterruptibly()
+    {
+        super.syncUninterruptibly();
+        return this;
+    }
+
+    public ChannelPromise await() throws InterruptedException
+    {
+        super.await();
+        return this;
+    }
+
+    public ChannelPromise awaitUninterruptibly()
+    {
+        super.awaitUninterruptibly();
+        return this;
+    }
+
+    public ChannelPromise addListener(GenericFutureListener<? extends Future<? super Void>> listener)
+    {
+        super.addListener(listener);
+        return this;
+    }
+
+    public ChannelPromise addListeners(GenericFutureListener<? extends Future<? super Void>>... listeners)
+    {
+        super.addListeners(listeners);
+        return this;
+    }
+
+    public ChannelPromise removeListener(GenericFutureListener<? extends Future<? super Void>> listener)
+    {
+        super.removeListener(listener);
+        return this;
+    }
+
+    public ChannelPromise removeListeners(GenericFutureListener<? extends Future<? super Void>>... listeners)
+    {
+        super.removeListeners(listeners);
+        return this;
+    }
+
+    public ChannelPromise unvoid()
+    {
+        return this;
+    }
+}
diff --git a/src/java/org/apache/cassandra/net/AsyncMessageOutputPlus.java b/src/java/org/apache/cassandra/net/AsyncMessageOutputPlus.java
new file mode 100644
index 0000000..8ef0a8f
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/AsyncMessageOutputPlus.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.net;
+
+import java.io.IOException;
+import java.nio.channels.ClosedChannelException;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.WriteBufferWaterMark;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+
+/**
+ * A {@link DataOutputStreamPlus} that writes ASYNCHRONOUSLY to a Netty Channel.
+ *
+ * Intended as single use, to write one (large) message.
+ *
+ * The close() and flush() methods synchronously wait for pending writes, and will propagate any exceptions
+ * encountered in writing them to the wire.
+ *
+ * The correctness of this class depends on the ChannelPromise we create against a Channel always being completed,
+ * which appears to be a guarantee provided by Netty so long as the event loop is running.
+ */
+public class AsyncMessageOutputPlus extends AsyncChannelOutputPlus
+{
+    /**
+     * the maximum {@link #highWaterMark} and minimum {@link #lowWaterMark} number of bytes we have flushing
+     * during which we should still be writing to the channel.
+     *
+     * i.e., if we are at or below the {@link #lowWaterMark} we should definitely start writing again;
+     *       if we are at or above the {@link #highWaterMark} we should definitely stop writing;
+     *       if we are inbetween, it is OK to either write or not write
+     *
+     * note that we consider the bytes we are about to write to our high water mark, but not our low.
+     * i.e., we will not begin a write that would take us over our high water mark, unless not doing so would
+     * take us below our low water mark.
+     *
+     * This is somewhat arbitrary accounting, and a meaningless distinction for flushes of a consistent size.
+     */
+    @SuppressWarnings("JavaDoc")
+    private final int highWaterMark;
+    private final int lowWaterMark;
+    private final int bufferSize;
+    private final int messageSize;
+    private boolean closing;
+
+    private final FrameEncoder.PayloadAllocator payloadAllocator;
+    private volatile FrameEncoder.Payload payload;
+
+    AsyncMessageOutputPlus(Channel channel, int bufferSize, int messageSize, FrameEncoder.PayloadAllocator payloadAllocator)
+    {
+        super(channel);
+        WriteBufferWaterMark waterMark = channel.config().getWriteBufferWaterMark();
+        this.lowWaterMark = waterMark.low();
+        this.highWaterMark = waterMark.high();
+        this.messageSize = messageSize;
+        this.bufferSize = Math.min(messageSize, bufferSize);
+        this.payloadAllocator = payloadAllocator;
+        allocateBuffer();
+    }
+
+    private void allocateBuffer()
+    {
+        payload = payloadAllocator.allocate(false, bufferSize);
+        buffer = payload.buffer;
+    }
+
+    @Override
+    protected void doFlush(int count) throws IOException
+    {
+        if (!channel.isOpen())
+            throw new ClosedChannelException();
+
+        // flush the current backing write buffer only if there's any pending data
+        FrameEncoder.Payload flush = payload;
+        int byteCount = flush.length();
+        if (byteCount == 0)
+            return;
+
+        if (byteCount + flushed() > (closing ? messageSize : messageSize - 1))
+            throw new InvalidSerializedSizeException(messageSize, byteCount + flushed());
+
+        flush.finish();
+        ChannelPromise promise = beginFlush(byteCount, lowWaterMark, highWaterMark);
+        channel.writeAndFlush(flush, promise);
+        allocateBuffer();
+    }
+
+    public void close() throws IOException
+    {
+        closing = true;
+        if (flushed() == 0 && payload != null)
+            payload.setSelfContained(true);
+        super.close();
+    }
+
+    public long position()
+    {
+        return flushed() + payload.length();
+    }
+
+    /**
+     * Discard any buffered data, and the buffers that contain it.
+     * May be invoked instead of {@link #close()} if we terminate exceptionally.
+     */
+    public void discard()
+    {
+        if (payload != null)
+        {
+            payload.release();
+            payload = null;
+            buffer = null;
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/net/AsyncOneResponse.java b/src/java/org/apache/cassandra/net/AsyncOneResponse.java
index 3fe0a2a..ba83c84 100644
--- a/src/java/org/apache/cassandra/net/AsyncOneResponse.java
+++ b/src/java/org/apache/cassandra/net/AsyncOneResponse.java
@@ -17,54 +17,31 @@
  */
 package org.apache.cassandra.net;
 
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.AbstractFuture;
+
+import io.netty.util.concurrent.ImmediateEventExecutor;
 
 /**
  * A callback specialized for returning a value from a single target; that is, this is for messages
  * that we only send to one recipient.
  */
-public class AsyncOneResponse<T> extends AbstractFuture<T> implements IAsyncCallback<T>
+public class AsyncOneResponse<T> extends AsyncPromise<T> implements RequestCallback<T>
 {
-    private final long start = System.nanoTime();
-
-    public void response(MessageIn<T> response)
-    {
-        set(response.payload);
-    }
-
-    public boolean isLatencyForSnitch()
+    public AsyncOneResponse()
     {
-        return false;
+        super(ImmediateEventExecutor.INSTANCE);
     }
 
-    @Override
-    public T get(long timeout, TimeUnit unit) throws TimeoutException
+    public void onResponse(Message<T> response)
     {
-        long adjustedTimeout = unit.toNanos(timeout) - (System.nanoTime() - start);
-        if (adjustedTimeout <= 0)
-        {
-            throw new TimeoutException("Operation timed out.");
-        }
-        try
-        {
-            return super.get(adjustedTimeout, TimeUnit.NANOSECONDS);
-        }
-        catch (InterruptedException | ExecutionException e)
-        {
-            throw new AssertionError(e);
-        }
+        setSuccess(response.payload);
     }
 
     @VisibleForTesting
     public static <T> AsyncOneResponse<T> immediate(T value)
     {
         AsyncOneResponse<T> response = new AsyncOneResponse<>();
-        response.set(value);
+        response.setSuccess(value);
         return response;
     }
 }
diff --git a/src/java/org/apache/cassandra/net/AsyncPromise.java b/src/java/org/apache/cassandra/net/AsyncPromise.java
new file mode 100644
index 0000000..36bc304
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/AsyncPromise.java
@@ -0,0 +1,488 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.net;
+
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.util.concurrent.EventExecutor;
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.GenericFutureListener;
+import io.netty.util.concurrent.Promise;
+import io.netty.util.internal.PlatformDependent;
+import io.netty.util.internal.ThrowableUtil;
+import org.apache.cassandra.utils.concurrent.WaitQueue;
+
+import static java.util.concurrent.atomic.AtomicReferenceFieldUpdater.*;
+
+/**
+ * Netty's DefaultPromise uses a mutex to coordinate notifiers AND waiters between the eventLoop and the other threads.
+ * Since we register cross-thread listeners, this has the potential to block internode messaging for an unknown
+ * number of threads for an unknown period of time, if we are unlucky with the scheduler (which will certainly
+ * happen, just with some unknown but low periodicity)
+ *
+ * At the same time, we manage some other efficiencies:
+ *  - We save some space when registering listeners, especially if there is only one listener, as we perform no
+ *    extra allocations in this case.
+ *  - We permit efficient initial state declaration, avoiding unnecessary CAS or lock acquisitions when mutating
+ *    a Promise we are ourselves constructing (and can easily add more; only those we use have been added)
+ *
+ * We can also make some guarantees about our behaviour here, although we primarily mirror Netty.
+ * Specifically, we can guarantee that notifiers are always invoked in the order they are added (which may be true
+ * for netty, but was unclear and is not declared).  This is useful for ensuring the correctness of some of our
+ * behaviours in OutboundConnection without having to jump through extra hoops.
+ *
+ * The implementation loosely follows that of Netty's DefaultPromise, with some slight changes; notably that we have
+ * no synchronisation on our listeners, instead using a CoW list that is cleared each time we notify listeners.
+ *
+ * We handle special values slightly differently.  We do not use a special value for null, instead using
+ * a special value to indicate the result has not been set yet.  This means that once isSuccess() holds,
+ * the result must be a correctly typed object (modulo generics pitfalls).
+ * All special values are also instances of FailureHolder, which simplifies a number of the logical conditions.
+ *
+ * @param <V>
+ */
+public class AsyncPromise<V> implements Promise<V>
+{
+    private static final Logger logger = LoggerFactory.getLogger(AsyncPromise.class);
+
+    private final EventExecutor executor;
+    private volatile Object result;
+    private volatile GenericFutureListener<? extends Future<? super V>> listeners;
+    private volatile WaitQueue waiting;
+    private static final AtomicReferenceFieldUpdater<AsyncPromise, Object> resultUpdater = newUpdater(AsyncPromise.class, Object.class, "result");
+    private static final AtomicReferenceFieldUpdater<AsyncPromise, GenericFutureListener> listenersUpdater = newUpdater(AsyncPromise.class, GenericFutureListener.class, "listeners");
+    private static final AtomicReferenceFieldUpdater<AsyncPromise, WaitQueue> waitingUpdater = newUpdater(AsyncPromise.class, WaitQueue.class, "waiting");
+
+    private static final FailureHolder UNSET = new FailureHolder(null);
+    private static final FailureHolder UNCANCELLABLE = new FailureHolder(null);
+    private static final FailureHolder CANCELLED = new FailureHolder(ThrowableUtil.unknownStackTrace(new CancellationException(), AsyncPromise.class, "cancel(...)"));
+
+    private static final DeferredGenericFutureListener NOTIFYING = future -> {};
+    private static interface DeferredGenericFutureListener<F extends Future<?>> extends GenericFutureListener<F> {}
+
+    private static final class FailureHolder
+    {
+        final Throwable cause;
+        private FailureHolder(Throwable cause)
+        {
+            this.cause = cause;
+        }
+    }
+
+    public AsyncPromise(EventExecutor executor)
+    {
+        this(executor, UNSET);
+    }
+
+    private AsyncPromise(EventExecutor executor, FailureHolder initialState)
+    {
+        this.executor = executor;
+        this.result = initialState;
+    }
+
+    public AsyncPromise(EventExecutor executor, GenericFutureListener<? extends Future<? super V>> listener)
+    {
+        this(executor);
+        this.listeners = listener;
+    }
+
+    AsyncPromise(EventExecutor executor, FailureHolder initialState, GenericFutureListener<? extends Future<? super V>> listener)
+    {
+        this(executor, initialState);
+        this.listeners = listener;
+    }
+
+    public static <V> AsyncPromise<V> uncancellable(EventExecutor executor)
+    {
+        return new AsyncPromise<>(executor, UNCANCELLABLE);
+    }
+
+    public static <V> AsyncPromise<V> uncancellable(EventExecutor executor, GenericFutureListener<? extends Future<? super V>> listener)
+    {
+        return new AsyncPromise<>(executor, UNCANCELLABLE);
+    }
+
+    public Promise<V> setSuccess(V v)
+    {
+        if (!trySuccess(v))
+            throw new IllegalStateException("complete already: " + this);
+        return this;
+    }
+
+    public Promise<V> setFailure(Throwable throwable)
+    {
+        if (!tryFailure(throwable))
+            throw new IllegalStateException("complete already: " + this);
+        return this;
+    }
+
+    public boolean trySuccess(V v)
+    {
+        return trySet(v);
+    }
+
+    public boolean tryFailure(Throwable throwable)
+    {
+        return trySet(new FailureHolder(throwable));
+    }
+
+    public boolean setUncancellable()
+    {
+        if (trySet(UNCANCELLABLE))
+            return true;
+        return result == UNCANCELLABLE;
+    }
+
+    public boolean cancel(boolean b)
+    {
+        return trySet(CANCELLED);
+    }
+
+    /**
+     * Shared implementation of various promise completion methods.
+     * Updates the result if it is possible to do so, returning success/failure.
+     *
+     * If the promise is UNSET the new value will succeed;
+     *          if it is UNCANCELLABLE it will succeed only if the new value is not CANCELLED
+     *          otherwise it will fail, as isDone() is implied
+     *
+     * If the update succeeds, and the new state implies isDone(), any listeners and waiters will be notified
+     */
+    private boolean trySet(Object v)
+    {
+        while (true)
+        {
+            Object current = result;
+            if (isDone(current) || (current == UNCANCELLABLE && v == CANCELLED))
+                return false;
+            if (resultUpdater.compareAndSet(this, current, v))
+            {
+                if (v != UNCANCELLABLE)
+                {
+                    notifyListeners();
+                    notifyWaiters();
+                }
+                return true;
+            }
+        }
+    }
+
+    public boolean isSuccess()
+    {
+        return isSuccess(result);
+    }
+
+    private static boolean isSuccess(Object result)
+    {
+        return !(result instanceof FailureHolder);
+    }
+
+    public boolean isCancelled()
+    {
+        return isCancelled(result);
+    }
+
+    private static boolean isCancelled(Object result)
+    {
+        return result == CANCELLED;
+    }
+
+    public boolean isDone()
+    {
+        return isDone(result);
+    }
+
+    private static boolean isDone(Object result)
+    {
+        return result != UNSET && result != UNCANCELLABLE;
+    }
+
+    public boolean isCancellable()
+    {
+        Object result = this.result;
+        return result == UNSET;
+    }
+
+    public Throwable cause()
+    {
+        Object result = this.result;
+        if (result instanceof FailureHolder)
+            return ((FailureHolder) result).cause;
+        return null;
+    }
+
+    /**
+     * if isSuccess(), returns the value, otherwise returns null
+     */
+    @SuppressWarnings("unchecked")
+    public V getNow()
+    {
+        Object result = this.result;
+        if (isSuccess(result))
+            return (V) result;
+        return null;
+    }
+
+    public V get() throws InterruptedException, ExecutionException
+    {
+        await();
+        return getWhenDone();
+    }
+
+    public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+    {
+        if (!await(timeout, unit))
+            throw new TimeoutException();
+        return getWhenDone();
+    }
+
+    /**
+     * Shared implementation of get() after suitable await(); assumes isDone(), and returns
+     * either the success result or throws the suitable exception under failure
+     */
+    @SuppressWarnings("unchecked")
+    private V getWhenDone() throws ExecutionException
+    {
+        Object result = this.result;
+        if (isSuccess(result))
+            return (V) result;
+        if (result == CANCELLED)
+            throw new CancellationException();
+        throw new ExecutionException(((FailureHolder) result).cause);
+    }
+
+    /**
+     * waits for completion; in case of failure rethrows the original exception without a new wrapping exception
+     * so may cause problems for reporting stack traces
+     */
+    public Promise<V> sync() throws InterruptedException
+    {
+        await();
+        rethrowIfFailed();
+        return this;
+    }
+
+    /**
+     * waits for completion; in case of failure rethrows the original exception without a new wrapping exception
+     * so may cause problems for reporting stack traces
+     */
+    public Promise<V> syncUninterruptibly()
+    {
+        awaitUninterruptibly();
+        rethrowIfFailed();
+        return this;
+    }
+
+    private void rethrowIfFailed()
+    {
+        Throwable cause = this.cause();
+        if (cause != null)
+        {
+            PlatformDependent.throwException(cause);
+        }
+    }
+
+    public Promise<V> addListener(GenericFutureListener<? extends Future<? super V>> listener)
+    {
+        listenersUpdater.accumulateAndGet(this, listener, AsyncPromise::appendListener);
+        if (isDone())
+            notifyListeners();
+        return this;
+    }
+
+    public Promise<V> addListeners(GenericFutureListener<? extends Future<? super V>> ... listeners)
+    {
+        // this could be more efficient if we cared, but we do not
+        return addListener(future -> {
+            for (GenericFutureListener<? extends Future<? super V>> listener : listeners)
+                AsyncPromise.invokeListener((GenericFutureListener<Future<? super V>>)listener, future);
+        });
+    }
+
+    public Promise<V> removeListener(GenericFutureListener<? extends Future<? super V>> listener)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public Promise<V> removeListeners(GenericFutureListener<? extends Future<? super V>> ... listeners)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @SuppressWarnings("unchecked")
+    private void notifyListeners()
+    {
+        if (!executor.inEventLoop())
+        {
+            // submit this method, to guarantee we invoke in the submitted order
+            executor.execute(this::notifyListeners);
+            return;
+        }
+
+        if (listeners == null || listeners instanceof DeferredGenericFutureListener<?>)
+            return; // either no listeners, or we are already notifying listeners, so we'll get to the new one when ready
+
+        // first run our notifiers
+        while (true)
+        {
+            GenericFutureListener listeners = listenersUpdater.getAndSet(this, NOTIFYING);
+            if (listeners != null)
+                invokeListener(listeners, this);
+
+            if (listenersUpdater.compareAndSet(this, NOTIFYING, null))
+                return;
+        }
+    }
+
+    private static <F extends Future<?>> void invokeListener(GenericFutureListener<F> listener, F future)
+    {
+        try
+        {
+            listener.operationComplete(future);
+        }
+        catch (Throwable t)
+        {
+            logger.error("Failed to invoke listener {} to {}", listener, future, t);
+        }
+    }
+
+    private static <F extends Future<?>> GenericFutureListener<F> appendListener(GenericFutureListener<F> prevListener, GenericFutureListener<F> newListener)
+    {
+        GenericFutureListener<F> result = newListener;
+
+        if (prevListener != null && prevListener != NOTIFYING)
+        {
+            result = future -> {
+                invokeListener(prevListener, future);
+                // we will wrap the outer invocation with invokeListener, so no need to do it here too
+                newListener.operationComplete(future);
+            };
+        }
+
+        if (prevListener instanceof DeferredGenericFutureListener<?>)
+        {
+            GenericFutureListener<F> wrap = result;
+            result = (DeferredGenericFutureListener<F>) wrap::operationComplete;
+        }
+
+        return result;
+    }
+
+    public Promise<V> await() throws InterruptedException
+    {
+        await(0L, (signal, nanos) -> { signal.await(); return true; } );
+        return this;
+    }
+
+    public Promise<V> awaitUninterruptibly()
+    {
+        await(0L, (signal, nanos) -> { signal.awaitUninterruptibly(); return true; } );
+        return this;
+    }
+
+    public boolean await(long timeout, TimeUnit unit) throws InterruptedException
+    {
+        return await(unit.toNanos(timeout),
+                     (signal, nanos) -> signal.awaitUntil(nanos + System.nanoTime()));
+    }
+
+    public boolean await(long timeoutMillis) throws InterruptedException
+    {
+        return await(timeoutMillis, TimeUnit.MILLISECONDS);
+    }
+
+    public boolean awaitUninterruptibly(long timeout, TimeUnit unit)
+    {
+        return await(unit.toNanos(timeout),
+                     (signal, nanos) -> signal.awaitUntilUninterruptibly(nanos + System.nanoTime()));
+    }
+
+    public boolean awaitUninterruptibly(long timeoutMillis)
+    {
+        return awaitUninterruptibly(timeoutMillis, TimeUnit.MILLISECONDS);
+    }
+
+    interface Awaiter<T extends Throwable>
+    {
+        boolean await(WaitQueue.Signal value, long nanos) throws T;
+    }
+
+    /**
+     * A clean way to implement each variant of await using lambdas; we permit a nanos parameter
+     * so that we can implement this without any unnecessary lambda allocations, although not
+     * all implementations need the nanos parameter (i.e. those that wait indefinitely)
+     */
+    private <T extends Throwable> boolean await(long nanos, Awaiter<T> awaiter) throws T
+    {
+        if (isDone())
+            return true;
+
+        WaitQueue.Signal await = registerToWait();
+        if (null != await)
+            return awaiter.await(await, nanos);
+
+        return true;
+    }
+
+    /**
+     * Register a signal that will be notified when the promise is completed;
+     * if the promise becomes completed before this signal is registered, null is returned
+     */
+    private WaitQueue.Signal registerToWait()
+    {
+        WaitQueue waiting = this.waiting;
+        if (waiting == null && !waitingUpdater.compareAndSet(this, null, waiting = new WaitQueue()))
+            waiting = this.waiting;
+        assert waiting != null;
+
+        WaitQueue.Signal signal = waiting.register();
+        if (!isDone())
+            return signal;
+        signal.cancel();
+        return null;
+    }
+
+    private void notifyWaiters()
+    {
+        WaitQueue waiting = this.waiting;
+        if (waiting != null)
+            waiting.signalAll();
+    }
+
+    public String toString()
+    {
+        Object result = this.result;
+        if (isSuccess(result))
+            return "(success: " + result + ')';
+        if (result == UNCANCELLABLE)
+            return "(uncancellable)";
+        if (result == CANCELLED)
+            return "(cancelled)";
+        if (isDone(result))
+            return "(failure: " + ((FailureHolder) result).cause + ')';
+        return "(incomplete)";
+    }
+}
diff --git a/src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.java b/src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.java
new file mode 100644
index 0000000..84fb8ac
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.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.net;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.primitives.Ints;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.Channel;
+import org.apache.cassandra.io.util.RebufferingInputStream;
+
+// TODO: rewrite
+public class AsyncStreamingInputPlus extends RebufferingInputStream
+{
+    public static class InputTimeoutException extends IOException
+    {
+    }
+
+    private static final long DEFAULT_REBUFFER_BLOCK_IN_MILLIS = TimeUnit.MINUTES.toMillis(3);
+
+    private final Channel channel;
+
+    /**
+     * The parent, or owning, buffer of the current buffer being read from ({@link super#buffer}).
+     */
+    private ByteBuf currentBuf;
+
+    private final BlockingQueue<ByteBuf> queue;
+
+    private final long rebufferTimeoutNanos;
+
+    private volatile boolean isClosed;
+
+    public AsyncStreamingInputPlus(Channel channel)
+    {
+        this(channel, DEFAULT_REBUFFER_BLOCK_IN_MILLIS, TimeUnit.MILLISECONDS);
+    }
+
+    AsyncStreamingInputPlus(Channel channel, long rebufferTimeout, TimeUnit rebufferTimeoutUnit)
+    {
+        super(Unpooled.EMPTY_BUFFER.nioBuffer());
+        currentBuf = Unpooled.EMPTY_BUFFER;
+
+        queue = new LinkedBlockingQueue<>();
+        rebufferTimeoutNanos = rebufferTimeoutUnit.toNanos(rebufferTimeout);
+
+        this.channel = channel;
+        channel.config().setAutoRead(false);
+    }
+
+    /**
+     * Append a {@link ByteBuf} to the end of the einternal queue.
+     *
+     * Note: it's expected this method is invoked on the netty event loop.
+     */
+    public boolean append(ByteBuf buf) throws IllegalStateException
+    {
+        if (isClosed) return false;
+
+        queue.add(buf);
+
+        /*
+         * it's possible for append() to race with close(), so we need to ensure
+         * that the bytebuf gets released in that scenario
+         */
+        if (isClosed)
+            while ((buf = queue.poll()) != null)
+                buf.release();
+
+        return true;
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * Release open buffers and poll the {@link #queue} for more data.
+     * <p>
+     * This is best, and more or less expected, to be invoked on a consuming thread (not the event loop)
+     * becasue if we block on the queue we can't fill it on the event loop (as that's where the buffers are coming from).
+     *
+     * @throws EOFException when no further reading from this instance should occur. Implies this instance is closed.
+     * @throws InputTimeoutException when no new buffers arrive for reading before
+     * the {@link #rebufferTimeoutNanos} elapses while blocking. It's then not safe to reuse this instance again.
+     */
+    @Override
+    protected void reBuffer() throws EOFException, InputTimeoutException
+    {
+        if (queue.isEmpty())
+            channel.read();
+
+        currentBuf.release();
+        currentBuf = null;
+        buffer = null;
+
+        ByteBuf next = null;
+        try
+        {
+            next = queue.poll(rebufferTimeoutNanos, TimeUnit.NANOSECONDS);
+        }
+        catch (InterruptedException ie)
+        {
+            // nop
+        }
+
+        if (null == next)
+            throw new InputTimeoutException();
+
+        if (next == Unpooled.EMPTY_BUFFER) // Unpooled.EMPTY_BUFFER is the indicator that the input is closed
+            throw new EOFException();
+
+        currentBuf = next;
+        buffer = next.nioBuffer();
+    }
+
+    public interface Consumer
+    {
+        int accept(ByteBuffer buffer) throws IOException;
+    }
+
+    /**
+     * Consumes bytes in the stream until the given length
+     */
+    public void consume(Consumer consumer, long length) throws IOException
+    {
+        while (length > 0)
+        {
+            if (!buffer.hasRemaining())
+                reBuffer();
+
+            final int position = buffer.position();
+            final int limit = buffer.limit();
+
+            buffer.limit(position + (int) Math.min(length, limit - position));
+            try
+            {
+                int copied = consumer.accept(buffer);
+                buffer.position(position + copied);
+                length -= copied;
+            }
+            finally
+            {
+                buffer.limit(limit);
+            }
+        }
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * As long as this method is invoked on the consuming thread the returned value will be accurate.
+     */
+    @VisibleForTesting
+    public int unsafeAvailable()
+    {
+        long count = buffer != null ? buffer.remaining() : 0;
+        for (ByteBuf buf : queue)
+            count += buf.readableBytes();
+
+        return Ints.checkedCast(count);
+    }
+
+    // TODO:JEB add docs
+    // TL;DR if there's no Bufs open anywhere here, issue a channle read to try and grab data.
+    public void maybeIssueRead()
+    {
+        if (isEmpty())
+            channel.read();
+    }
+
+    public boolean isEmpty()
+    {
+        return queue.isEmpty() && (buffer == null || !buffer.hasRemaining());
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * Note: This should invoked on the consuming thread.
+     */
+    @Override
+    public void close()
+    {
+        if (isClosed)
+            return;
+
+        if (currentBuf != null)
+        {
+            currentBuf.release();
+            currentBuf = null;
+            buffer = null;
+        }
+
+        while (true)
+        {
+            try
+            {
+                ByteBuf buf = queue.poll(Long.MAX_VALUE, TimeUnit.NANOSECONDS);
+                if (buf == Unpooled.EMPTY_BUFFER)
+                    break;
+                else
+                    buf.release();
+            }
+            catch (InterruptedException e)
+            {
+                //
+            }
+        }
+
+        isClosed = true;
+    }
+
+    /**
+     * Mark this stream as closed, but do not release any of the resources.
+     *
+     * Note: this is best to be called from the producer thread.
+     */
+    public void requestClosure()
+    {
+        queue.add(Unpooled.EMPTY_BUFFER);
+    }
+
+    // TODO: let's remove this like we did for AsyncChannelOutputPlus
+    public ByteBufAllocator getAllocator()
+    {
+        return channel.alloc();
+    }
+}
diff --git a/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java b/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
new file mode 100644
index 0000000..a52070e
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.net;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.nio.channels.FileChannel;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.WriteBufferWaterMark;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.net.SharedDefaultFileRegion.SharedFileChannel;
+import org.apache.cassandra.streaming.StreamManager.StreamRateLimiter;
+import org.apache.cassandra.utils.memory.BufferPool;
+
+import static java.lang.Math.min;
+
+/**
+ * A {@link DataOutputStreamPlus} that writes ASYNCHRONOUSLY to a Netty Channel.
+ *
+ * The close() and flush() methods synchronously wait for pending writes, and will propagate any exceptions
+ * encountered in writing them to the wire.
+ *
+ * The correctness of this class depends on the ChannelPromise we create against a Channel always being completed,
+ * which appears to be a guarantee provided by Netty so long as the event loop is running.
+ */
+public class AsyncStreamingOutputPlus extends AsyncChannelOutputPlus
+{
+    private static final Logger logger = LoggerFactory.getLogger(AsyncStreamingOutputPlus.class);
+
+    final int defaultLowWaterMark;
+    final int defaultHighWaterMark;
+
+    public AsyncStreamingOutputPlus(Channel channel)
+    {
+        super(channel);
+        WriteBufferWaterMark waterMark = channel.config().getWriteBufferWaterMark();
+        this.defaultLowWaterMark = waterMark.low();
+        this.defaultHighWaterMark = waterMark.high();
+        allocateBuffer();
+    }
+
+    private void allocateBuffer()
+    {
+        // this buffer is only used for small quantities of data
+        buffer = BufferPool.getAtLeast(8 << 10, BufferType.OFF_HEAP);
+    }
+
+    @Override
+    protected void doFlush(int count) throws IOException
+    {
+        if (!channel.isOpen())
+            throw new ClosedChannelException();
+
+        // flush the current backing write buffer only if there's any pending data
+        ByteBuffer flush = buffer;
+        if (flush.position() == 0)
+            return;
+
+        flush.flip();
+        int byteCount = flush.limit();
+        ChannelPromise promise = beginFlush(byteCount, 0, Integer.MAX_VALUE);
+        channel.writeAndFlush(GlobalBufferPoolAllocator.wrap(flush), promise);
+        allocateBuffer();
+    }
+
+    public long position()
+    {
+        return flushed() + buffer.position();
+    }
+
+    public interface BufferSupplier
+    {
+        /**
+         * Request a buffer with at least the given capacity.
+         * This method may only be invoked once, and the lifetime of buffer it returns will be managed
+         * by the AsyncChannelOutputPlus it was created for.
+         */
+        ByteBuffer get(int capacity) throws IOException;
+    }
+
+    public interface Write
+    {
+        /**
+         * Write to a buffer, and flush its contents to the channel.
+         * <p>
+         * The lifetime of the buffer will be managed by the AsyncChannelOutputPlus you issue this Write to.
+         * If the method exits successfully, the contents of the buffer will be written to the channel, otherwise
+         * the buffer will be cleaned and the exception propagated to the caller.
+         */
+        void write(BufferSupplier supplier) throws IOException;
+    }
+
+    /**
+     * Provide a lambda that can request a buffer of suitable size, then fill the buffer and have
+     * that buffer written and flushed to the underlying channel, without having to handle buffer
+     * allocation, lifetime or cleanup, including in case of exceptions.
+     * <p>
+     * Any exception thrown by the Write will be propagated to the caller, after any buffer is cleaned up.
+     */
+    public int writeToChannel(Write write, StreamRateLimiter limiter) throws IOException
+    {
+        doFlush(0);
+        class Holder
+        {
+            ChannelPromise promise;
+            ByteBuffer buffer;
+        }
+        Holder holder = new Holder();
+
+        try
+        {
+            write.write(size -> {
+                if (holder.buffer != null)
+                    throw new IllegalStateException("Can only allocate one ByteBuffer");
+                limiter.acquire(size);
+                holder.promise = beginFlush(size, defaultLowWaterMark, defaultHighWaterMark);
+                holder.buffer = BufferPool.get(size);
+                return holder.buffer;
+            });
+        }
+        catch (Throwable t)
+        {
+            // we don't currently support cancelling the flush, but at this point we are recoverable if we want
+            if (holder.buffer != null)
+                BufferPool.put(holder.buffer);
+            if (holder.promise != null)
+                holder.promise.tryFailure(t);
+            throw t;
+        }
+
+        ByteBuffer buffer = holder.buffer;
+        BufferPool.putUnusedPortion(buffer);
+
+        int length = buffer.limit();
+        channel.writeAndFlush(GlobalBufferPoolAllocator.wrap(buffer), holder.promise);
+        return length;
+    }
+
+    /**
+     * <p>
+     * Writes all data in file channel to stream, 1MiB at a time, with at most 2MiB in flight at once.
+     * This method takes ownership of the provided {@code FileChannel}.
+     * <p>
+     * WARNING: this method blocks only for permission to write to the netty channel; it exits before
+     * the write is flushed to the network.
+     */
+    public long writeFileToChannel(FileChannel file, StreamRateLimiter limiter) throws IOException
+    {
+        // write files in 1MiB chunks, since there may be blocking work performed to fetch it from disk,
+        // the data is never brought in process and is gated by the wire anyway
+        return writeFileToChannel(file, limiter, 1 << 20, 1 << 20, 2 << 20);
+    }
+
+    public long writeFileToChannel(FileChannel file, StreamRateLimiter limiter, int batchSize, int lowWaterMark, int highWaterMark) throws IOException
+    {
+        final long length = file.size();
+        long bytesTransferred = 0;
+
+        final SharedFileChannel sharedFile = SharedDefaultFileRegion.share(file);
+        try
+        {
+            while (bytesTransferred < length)
+            {
+                int toWrite = (int) min(batchSize, length - bytesTransferred);
+
+                limiter.acquire(toWrite);
+                ChannelPromise promise = beginFlush(toWrite, lowWaterMark, highWaterMark);
+
+                SharedDefaultFileRegion fileRegion = new SharedDefaultFileRegion(sharedFile, bytesTransferred, toWrite);
+                channel.writeAndFlush(fileRegion, promise);
+
+                if (logger.isTraceEnabled())
+                    logger.trace("Writing {} bytes at position {} of {}", toWrite, bytesTransferred, length);
+                bytesTransferred += toWrite;
+            }
+
+            return bytesTransferred;
+        }
+        finally
+        {
+            sharedFile.release();
+        }
+    }
+
+    /**
+     * Discard any buffered data, and the buffers that contain it.
+     * May be invoked instead of {@link #close()} if we terminate exceptionally.
+     */
+    public void discard()
+    {
+        if (buffer != null)
+        {
+            BufferPool.put(buffer);
+            buffer = null;
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/net/BackPressureState.java b/src/java/org/apache/cassandra/net/BackPressureState.java
index 886c075..de19bf3 100644
--- a/src/java/org/apache/cassandra/net/BackPressureState.java
+++ b/src/java/org/apache/cassandra/net/BackPressureState.java
@@ -27,7 +27,7 @@ public interface BackPressureState
     /**
      * Called when a message is sent to a replica.
      */
-    void onMessageSent(MessageOut<?> message);
+    void onMessageSent(Message<?> message);
 
     /**
      * Called when a response is received from a replica.
diff --git a/src/java/org/apache/cassandra/net/BufferPoolAllocator.java b/src/java/org/apache/cassandra/net/BufferPoolAllocator.java
new file mode 100644
index 0000000..8782c03
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/BufferPoolAllocator.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.net;
+
+import java.nio.ByteBuffer;
+
+import io.netty.buffer.AbstractByteBufAllocator;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.buffer.UnpooledUnsafeDirectByteBuf;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.utils.memory.BufferPool;
+
+/**
+ * A trivial wrapper around BufferPool for integrating with Netty, but retaining ownership of pooling behaviour
+ * that is integrated into Cassandra's other pooling.
+ */
+abstract class BufferPoolAllocator extends AbstractByteBufAllocator
+{
+    BufferPoolAllocator()
+    {
+        super(true);
+    }
+
+    @Override
+    public boolean isDirectBufferPooled()
+    {
+        return true;
+    }
+
+    /** shouldn't be invoked */
+    @Override
+    protected ByteBuf newHeapBuffer(int minCapacity, int maxCapacity)
+    {
+        return Unpooled.buffer(minCapacity, maxCapacity);
+    }
+
+    @Override
+    protected ByteBuf newDirectBuffer(int minCapacity, int maxCapacity)
+    {
+        ByteBuf result = new Wrapped(this, getAtLeast(minCapacity));
+        result.clear();
+        return result;
+    }
+
+    ByteBuffer get(int size)
+    {
+        return BufferPool.get(size, BufferType.OFF_HEAP);
+    }
+
+    ByteBuffer getAtLeast(int size)
+    {
+        return BufferPool.getAtLeast(size, BufferType.OFF_HEAP);
+    }
+
+    void put(ByteBuffer buffer)
+    {
+        BufferPool.put(buffer);
+    }
+
+    void putUnusedPortion(ByteBuffer buffer)
+    {
+        BufferPool.putUnusedPortion(buffer);
+    }
+
+    void release()
+    {
+    }
+
+    /**
+     * A simple extension to UnpooledUnsafeDirectByteBuf that returns buffers to BufferPool on deallocate,
+     * and permits extracting the buffer from it to take ownership and use directly.
+     */
+    public static class Wrapped extends UnpooledUnsafeDirectByteBuf
+    {
+        private ByteBuffer wrapped;
+
+        Wrapped(BufferPoolAllocator allocator, ByteBuffer wrap)
+        {
+            super(allocator, wrap, wrap.capacity());
+            wrapped = wrap;
+        }
+
+        @Override
+        public void deallocate()
+        {
+            if (wrapped != null)
+                BufferPool.put(wrapped);
+        }
+
+        public ByteBuffer adopt()
+        {
+            if (refCnt() > 1)
+                throw new IllegalStateException();
+            ByteBuffer adopt = wrapped;
+            adopt.position(readerIndex()).limit(writerIndex());
+            wrapped = null;
+            return adopt;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/net/CallbackInfo.java b/src/java/org/apache/cassandra/net/CallbackInfo.java
deleted file mode 100644
index f2ed8a1..0000000
--- a/src/java/org/apache/cassandra/net/CallbackInfo.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.net;
-
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.locator.InetAddressAndPort;
-
-/**
- * Encapsulates the callback information.
- * The ability to set the message is useful in cases for when a hint needs 
- * to be written due to a timeout in the response from a replica.
- */
-public class CallbackInfo
-{
-    protected final InetAddressAndPort target;
-    protected final IAsyncCallback callback;
-    protected final IVersionedSerializer<?> serializer;
-    private final boolean failureCallback;
-
-    /**
-     * Create CallbackInfo without sent message
-     *
-     * @param target target to send message
-     * @param callback
-     * @param serializer serializer to deserialize response message
-     * @param failureCallback True when we have a callback to handle failures
-     */
-    public CallbackInfo(InetAddressAndPort target, IAsyncCallback callback, IVersionedSerializer<?> serializer, boolean failureCallback)
-    {
-        this.target = target;
-        this.callback = callback;
-        this.serializer = serializer;
-        this.failureCallback = failureCallback;
-    }
-
-    public boolean shouldHint()
-    {
-        return false;
-    }
-
-    public boolean isFailureCallback()
-    {
-        return failureCallback;
-    }
-
-    public String toString()
-    {
-        return "CallbackInfo(" +
-               "target=" + target +
-               ", callback=" + callback +
-               ", serializer=" + serializer +
-               ", failureCallback=" + failureCallback +
-               ')';
-    }
-}
diff --git a/src/java/org/apache/cassandra/net/ChunkedInputPlus.java b/src/java/org/apache/cassandra/net/ChunkedInputPlus.java
new file mode 100644
index 0000000..3aad8d9
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/ChunkedInputPlus.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.net;
+
+import java.io.EOFException;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
+
+import org.apache.cassandra.io.util.RebufferingInputStream;
+
+/**
+ * A specialised {@link org.apache.cassandra.io.util.DataInputPlus} implementation for deserializing large messages
+ * that are split over multiple {@link FrameDecoder.Frame}s.
+ *
+ * Ensures that every underlying {@link ShareableBytes} frame is released, and promptly so, as frames are consumed.
+ *
+ * {@link #close()} <em>MUST</em> be invoked in the end.
+ */
+class ChunkedInputPlus extends RebufferingInputStream
+{
+    private final PeekingIterator<ShareableBytes> iter;
+
+    private ChunkedInputPlus(PeekingIterator<ShareableBytes> iter)
+    {
+        super(iter.peek().get());
+        this.iter = iter;
+    }
+
+    /**
+     * Creates a {@link ChunkedInputPlus} from the provided {@link ShareableBytes} buffers.
+     *
+     * The provided iterable <em>must</em> contain at least one buffer.
+     */
+    static ChunkedInputPlus of(Iterable<ShareableBytes> buffers)
+    {
+        PeekingIterator<ShareableBytes> iter = Iterators.peekingIterator(buffers.iterator());
+        if (!iter.hasNext())
+            throw new IllegalArgumentException();
+        return new ChunkedInputPlus(iter);
+    }
+
+    @Override
+    protected void reBuffer() throws EOFException
+    {
+        buffer = null;
+        iter.peek().release();
+        iter.next();
+
+        if (!iter.hasNext())
+            throw new EOFException();
+
+        buffer = iter.peek().get();
+    }
+
+    @Override
+    public void close()
+    {
+        buffer = null;
+        iter.forEachRemaining(ShareableBytes::release);
+    }
+
+    /**
+     * Returns the number of unconsumed bytes. Will release any outstanding buffers and consume the underlying iterator.
+     *
+     * Should only be used for sanity checking, once the input is no longer needed, as it will implicitly close the input.
+     */
+    int remainder()
+    {
+        buffer = null;
+
+        int bytes = 0;
+        while (iter.hasNext())
+        {
+            ShareableBytes chunk = iter.peek();
+            bytes += chunk.remaining();
+            chunk.release();
+            iter.next();
+        }
+        return bytes;
+    }
+}
diff --git a/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java b/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java
deleted file mode 100644
index b58ca47..0000000
--- a/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.net;
-
-import java.io.*;
-import java.net.Inet4Address;
-import java.net.Inet6Address;
-import java.net.InetAddress;
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.util.DataInputBuffer;
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.streaming.messages.StreamMessage;
-
-/*
- * As of version 4.0 the endpoint description includes a port number as an unsigned short
- */
-public class CompactEndpointSerializationHelper implements IVersionedSerializer<InetAddressAndPort>
-{
-    public static final IVersionedSerializer<InetAddressAndPort> instance = new CompactEndpointSerializationHelper();
-
-    /**
-     * Streaming uses its own version numbering so we need to ignore it and always use currrent version.
-     * There is no cross version streaming so it will always use the latest address serialization.
-     **/
-    public static final IVersionedSerializer<InetAddressAndPort> streamingInstance = new IVersionedSerializer<InetAddressAndPort>()
-    {
-        public void serialize(InetAddressAndPort inetAddressAndPort, DataOutputPlus out, int version) throws IOException
-        {
-            instance.serialize(inetAddressAndPort, out, MessagingService.current_version);
-        }
-
-        public InetAddressAndPort deserialize(DataInputPlus in, int version) throws IOException
-        {
-            return instance.deserialize(in, MessagingService.current_version);
-        }
-
-        public long serializedSize(InetAddressAndPort inetAddressAndPort, int version)
-        {
-            return instance.serializedSize(inetAddressAndPort, MessagingService.current_version);
-        }
-    };
-
-    private CompactEndpointSerializationHelper() {}
-
-    public void serialize(InetAddressAndPort endpoint, DataOutputPlus out, int version) throws IOException
-    {
-        if (version >= MessagingService.VERSION_40)
-        {
-            byte[] buf = endpoint.addressBytes;
-            out.writeByte(buf.length + 2);
-            out.write(buf);
-            out.writeShort(endpoint.port);
-        }
-        else
-        {
-            byte[] buf = endpoint.addressBytes;
-            out.writeByte(buf.length);
-            out.write(buf);
-        }
-    }
-
-    public InetAddressAndPort deserialize(DataInputPlus in, int version) throws IOException
-    {
-        int size = in.readByte() & 0xFF;
-        switch(size)
-        {
-            //The original pre-4.0 serialiation of just an address
-            case 4:
-            case 16:
-            {
-                byte[] bytes = new byte[size];
-                in.readFully(bytes, 0, bytes.length);
-                return InetAddressAndPort.getByAddress(bytes);
-            }
-            //Address and one port
-            case 6:
-            case 18:
-            {
-                byte[] bytes = new byte[size - 2];
-                in.readFully(bytes);
-
-                int port = in.readShort() & 0xFFFF;
-                return InetAddressAndPort.getByAddressOverrideDefaults(InetAddress.getByAddress(bytes), bytes, port);
-            }
-            default:
-                throw new AssertionError("Unexpected size " + size);
-
-        }
-    }
-
-    public long serializedSize(InetAddressAndPort from, int version)
-    {
-        //4.0 includes a port number
-        if (version >= MessagingService.VERSION_40)
-        {
-            if (from.address instanceof Inet4Address)
-                return 1 + 4 + 2;
-            assert from.address instanceof Inet6Address;
-            return 1 + 16 + 2;
-        }
-        else
-        {
-            if (from.address instanceof Inet4Address)
-                return 1 + 4;
-            assert from.address instanceof Inet6Address;
-            return 1 + 16;
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/net/async/ExpiredException.java b/src/java/org/apache/cassandra/net/ConnectionCategory.java
similarity index 71%
rename from src/java/org/apache/cassandra/net/async/ExpiredException.java
rename to src/java/org/apache/cassandra/net/ConnectionCategory.java
index 191900c..d739e9d 100644
--- a/src/java/org/apache/cassandra/net/async/ExpiredException.java
+++ b/src/java/org/apache/cassandra/net/ConnectionCategory.java
@@ -16,13 +16,19 @@
  * limitations under the License.
  */
 
-package org.apache.cassandra.net.async;
+package org.apache.cassandra.net;
 
-/**
- * Thrown when a {@link QueuedMessage} has timed out (has sat in the netty outbound channel for too long).
- */
-class ExpiredException extends Exception
+public enum ConnectionCategory
 {
-    @SuppressWarnings("ThrowableInstanceNeverThrown")
-    static final ExpiredException INSTANCE = new ExpiredException();
+    MESSAGING, STREAMING;
+
+    public boolean isStreaming()
+    {
+        return this == STREAMING;
+    }
+
+    public boolean isMessaging()
+    {
+        return this == MESSAGING;
+    }
 }
diff --git a/src/java/org/apache/cassandra/net/ConnectionType.java b/src/java/org/apache/cassandra/net/ConnectionType.java
new file mode 100644
index 0000000..db83d06
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/ConnectionType.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.net;
+
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+
+public enum ConnectionType
+{
+    LEGACY_MESSAGES (0), // only used for inbound
+    URGENT_MESSAGES (1),
+    SMALL_MESSAGES  (2),
+    LARGE_MESSAGES  (3),
+    STREAMING       (4);
+
+    public static final List<ConnectionType> MESSAGING_TYPES = ImmutableList.of(URGENT_MESSAGES, SMALL_MESSAGES, LARGE_MESSAGES);
+
+    public final int id;
+
+    ConnectionType(int id)
+    {
+        this.id = id;
+    }
+
+    public int twoBitID()
+    {
+        if (id < 0 || id > 0b11)
+            throw new AssertionError();
+        return id;
+    }
+
+    public boolean isStreaming()
+    {
+        return this == STREAMING;
+    }
+
+    public boolean isMessaging()
+    {
+        return !isStreaming();
+    }
+
+    public ConnectionCategory category()
+    {
+        return this == STREAMING ? ConnectionCategory.STREAMING : ConnectionCategory.MESSAGING;
+    }
+
+    private static final ConnectionType[] values = values();
+
+    public static ConnectionType fromId(int id)
+    {
+        return values[id];
+    }
+}
diff --git a/src/java/org/apache/cassandra/net/Crc.java b/src/java/org/apache/cassandra/net/Crc.java
new file mode 100644
index 0000000..dbd2601
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/Crc.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.net;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
... 50148 lines suppressed ...


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


[cassandra] 02/02: Introduce a proxy test handler, extra unit tests for connection closure and message expirations

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

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

commit 0d48503c8c99d715164cdc59de646366408889ff
Author: Alex Petrov <ol...@gmail.com>
AuthorDate: Wed Jun 12 15:02:37 2019 +0100

    Introduce a proxy test handler, extra unit tests for connection closure and message expirations
    
    patch by Alex Petrov; reviewed by Aleksey Yeschenko and Benedict Elliott
    Smith for CASSANDRA-15066
---
 .../cassandra/net/ProxyHandlerConnectionsTest.java | 405 +++++++++++++++++++++
 .../cassandra/net/proxy/InboundProxyHandler.java   | 234 ++++++++++++
 .../cassandra/net/proxy/ProxyHandlerTest.java      | 222 +++++++++++
 3 files changed, 861 insertions(+)

diff --git a/test/unit/org/apache/cassandra/net/ProxyHandlerConnectionsTest.java b/test/unit/org/apache/cassandra/net/ProxyHandlerConnectionsTest.java
new file mode 100644
index 0000000..270a910
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/ProxyHandlerConnectionsTest.java
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.net;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+import java.util.function.ToLongFunction;
+
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.io.IVersionedAsymmetricSerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.proxy.InboundProxyHandler;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.net.ConnectionTest.SETTINGS;
+import static org.apache.cassandra.net.OutboundConnectionSettings.Framing.CRC;
+import static org.apache.cassandra.utils.MonotonicClock.approxTime;
+
+public class ProxyHandlerConnectionsTest
+{
+    private static final SocketFactory factory = new SocketFactory();
+
+    private final Map<Verb, Supplier<? extends IVersionedAsymmetricSerializer<?, ?>>> serializers = new HashMap<>();
+    private final Map<Verb, Supplier<? extends IVerbHandler<?>>> handlers = new HashMap<>();
+    private final Map<Verb, ToLongFunction<TimeUnit>> timeouts = new HashMap<>();
+
+    private void unsafeSetSerializer(Verb verb, Supplier<? extends IVersionedAsymmetricSerializer<?, ?>> supplier) throws Throwable
+    {
+        serializers.putIfAbsent(verb, verb.unsafeSetSerializer(supplier));
+    }
+
+    protected void unsafeSetHandler(Verb verb, Supplier<? extends IVerbHandler<?>> supplier) throws Throwable
+    {
+        handlers.putIfAbsent(verb, verb.unsafeSetHandler(supplier));
+    }
+
+    private void unsafeSetExpiration(Verb verb, ToLongFunction<TimeUnit> expiration) throws Throwable
+    {
+        timeouts.putIfAbsent(verb, verb.unsafeSetExpiration(expiration));
+    }
+
+    @BeforeClass
+    public static void startup()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    @After
+    public void cleanup() throws Throwable
+    {
+        for (Map.Entry<Verb, Supplier<? extends IVersionedAsymmetricSerializer<?, ?>>> e : serializers.entrySet())
+            e.getKey().unsafeSetSerializer(e.getValue());
+        serializers.clear();
+        for (Map.Entry<Verb, Supplier<? extends IVerbHandler<?>>> e : handlers.entrySet())
+            e.getKey().unsafeSetHandler(e.getValue());
+        handlers.clear();
+        for (Map.Entry<Verb, ToLongFunction<TimeUnit>> e : timeouts.entrySet())
+            e.getKey().unsafeSetExpiration(e.getValue());
+        timeouts.clear();
+    }
+
+    @Test
+    public void testExpireInbound() throws Throwable
+    {
+        DatabaseDescriptor.setCrossNodeTimeout(true);
+        testOneManual((settings, inbound, outbound, endpoint, handler) -> {
+            unsafeSetSerializer(Verb._TEST_1, FakePayloadSerializer::new);
+
+            CountDownLatch connectionLatch = new CountDownLatch(1);
+            unsafeSetHandler(Verb._TEST_1, () -> v -> {
+                connectionLatch.countDown();
+            });
+            outbound.enqueue(Message.out(Verb._TEST_1, 1L));
+            connectionLatch.await(10, SECONDS);
+            Assert.assertEquals(0, connectionLatch.getCount());
+
+            // Slow things down
+            unsafeSetExpiration(Verb._TEST_1, unit -> unit.convert(50, MILLISECONDS));
+            handler.withLatency(100, MILLISECONDS);
+
+            unsafeSetHandler(Verb._TEST_1, () -> v -> {
+                throw new RuntimeException("Should have not been triggered " + v);
+            });
+            int expireMessages = 10;
+            for (int i = 0; i < expireMessages; i++)
+                outbound.enqueue(Message.out(Verb._TEST_1, 1L));
+
+            InboundMessageHandlers handlers = MessagingService.instance().getInbound(endpoint);
+            waitForCondition(() -> handlers.expiredCount() == expireMessages);
+            Assert.assertEquals(expireMessages, handlers.expiredCount());
+        });
+    }
+
+    @Test
+    public void testExpireSome() throws Throwable
+    {
+        DatabaseDescriptor.setCrossNodeTimeout(true);
+        testOneManual((settings, inbound, outbound, endpoint, handler) -> {
+            unsafeSetSerializer(Verb._TEST_1, FakePayloadSerializer::new);
+            connect(outbound);
+
+            AtomicInteger counter = new AtomicInteger();
+            unsafeSetHandler(Verb._TEST_1, () -> v -> {
+                counter.incrementAndGet();
+            });
+
+            int expireMessages = 10;
+            for (int i = 0; i < expireMessages; i++)
+                outbound.enqueue(Message.out(Verb._TEST_1, 1L));
+            waitForCondition(() -> counter.get() == 10);
+
+            unsafeSetExpiration(Verb._TEST_1, unit -> unit.convert(50, MILLISECONDS));
+            handler.withLatency(100, MILLISECONDS);
+
+            InboundMessageHandlers handlers = MessagingService.instance().getInbound(endpoint);
+            for (int i = 0; i < expireMessages; i++)
+                outbound.enqueue(Message.out(Verb._TEST_1, 1L));
+            waitForCondition(() -> handlers.expiredCount() == 10);
+
+            handler.withLatency(2, MILLISECONDS);
+
+            for (int i = 0; i < expireMessages; i++)
+                outbound.enqueue(Message.out(Verb._TEST_1, 1L));
+            waitForCondition(() -> counter.get() == 20);
+        });
+    }
+
+    @Test
+    public void testExpireSomeFromBatch() throws Throwable
+    {
+        DatabaseDescriptor.setCrossNodeTimeout(true);
+        testManual((settings, inbound, outbound, endpoint, handler) -> {
+            unsafeSetSerializer(Verb._TEST_1, FakePayloadSerializer::new);
+            connect(outbound);
+
+            Message msg = Message.out(Verb._TEST_1, 1L);
+            int messageSize = msg.serializedSize(MessagingService.current_version);
+            DatabaseDescriptor.setInternodeMaxMessageSizeInBytes(messageSize * 40);
+
+            AtomicInteger counter = new AtomicInteger();
+            unsafeSetHandler(Verb._TEST_1, () -> v -> {
+                counter.incrementAndGet();
+            });
+
+            unsafeSetExpiration(Verb._TEST_1, unit -> unit.convert(200, MILLISECONDS));
+            handler.withLatency(100, MILLISECONDS);
+
+            int expireMessages = 20;
+            long nanoTime = approxTime.now();
+            CountDownLatch enqueueDone = new CountDownLatch(1);
+            outbound.unsafeRunOnDelivery(() -> Uninterruptibles.awaitUninterruptibly(enqueueDone, 10, SECONDS));
+            for (int i = 0; i < expireMessages; i++)
+            {
+                boolean expire = i % 2 == 0;
+                Message.Builder builder = Message.builder(Verb._TEST_1, 1L);
+
+                if (settings.right.acceptVersions == ConnectionTest.legacy)
+                {
+                    // backdate messages; leave 50 milliseconds to leave outbound path
+                    builder.withCreatedAt(nanoTime - (expire ? 0 : MILLISECONDS.toNanos(150)));
+                }
+                else
+                {
+                    // Give messages 50 milliseconds to leave outbound path
+                    builder.withCreatedAt(nanoTime)
+                           .withExpiresAt(nanoTime + (expire ? MILLISECONDS.toNanos(50) : MILLISECONDS.toNanos(1000)));
+                }
+                outbound.enqueue(builder.build());
+            }
+            enqueueDone.countDown();
+
+            InboundMessageHandlers handlers = MessagingService.instance().getInbound(endpoint);
+            waitForCondition(() -> handlers.expiredCount() == 10 && counter.get() == 10,
+                             () -> String.format("Expired: %d, Arrived: %d", handlers.expiredCount(), counter.get()));
+        });
+    }
+
+    @Test
+    public void suddenDisconnect() throws Throwable
+    {
+        testManual((settings, inbound, outbound, endpoint, handler) -> {
+            handler.onDisconnect(() -> handler.reset());
+
+            unsafeSetSerializer(Verb._TEST_1, FakePayloadSerializer::new);
+            connect(outbound);
+
+            CountDownLatch closeLatch = new CountDownLatch(1);
+            handler.withCloseAfterRead(closeLatch::countDown);
+            AtomicInteger counter = new AtomicInteger();
+            unsafeSetHandler(Verb._TEST_1, () -> v -> counter.incrementAndGet());
+
+            outbound.enqueue(Message.out(Verb._TEST_1, 1L));
+            waitForCondition(() -> !outbound.isConnected());
+
+            connect(outbound);
+            Assert.assertTrue(outbound.isConnected());
+            Assert.assertEquals(0, counter.get());
+        });
+    }
+
+    @Test
+    public void testCorruptionOnHandshake() throws Throwable
+    {
+        testManual((settings, inbound, outbound, endpoint, handler) -> {
+            unsafeSetSerializer(Verb._TEST_1, FakePayloadSerializer::new);
+            // Invalid CRC
+            handler.withPayloadTransform(msg -> {
+                ByteBuf bb = (ByteBuf) msg;
+                bb.setByte(bb.readableBytes() / 2, 0xffff);
+                return msg;
+            });
+            tryConnect(outbound, 1, SECONDS, false);
+            Assert.assertTrue(!outbound.isConnected());
+
+            // Invalid protocol magic
+            handler.withPayloadTransform(msg -> {
+                ByteBuf bb = (ByteBuf) msg;
+                bb.setByte(0, 0xffff);
+                return msg;
+            });
+            tryConnect(outbound, 1, SECONDS, false);
+            Assert.assertTrue(!outbound.isConnected());
+            if (settings.right.framing == CRC)
+            {
+                Assert.assertEquals(2, outbound.connectionAttempts());
+                Assert.assertEquals(0, outbound.successfulConnections());
+            }
+        });
+    }
+
+    private static void waitForCondition(Supplier<Boolean> cond) throws Throwable
+    {
+        CompletableFuture.runAsync(() -> {
+            while (!cond.get()) {}
+        }).get(10, SECONDS);
+    }
+
+    private static void waitForCondition(Supplier<Boolean> cond, Supplier<String> s) throws Throwable
+    {
+        try
+        {
+            CompletableFuture.runAsync(() -> {
+                while (!cond.get()) {}
+            }).get(10, SECONDS);
+        }
+        catch (TimeoutException e)
+        {
+            throw new AssertionError(s.get());
+        }
+    }
+
+    private static class FakePayloadSerializer implements IVersionedSerializer<Long>
+    {
+        private final int size;
+        private FakePayloadSerializer()
+        {
+            this(1);
+        }
+
+        // Takes long and repeats it size times
+        private FakePayloadSerializer(int size)
+        {
+            this.size = size;
+        }
+
+        public void serialize(Long i, DataOutputPlus out, int version) throws IOException
+        {
+            for (int j = 0; j < size; j++)
+            {
+                out.writeLong(i);
+            }
+        }
+
+        public Long deserialize(DataInputPlus in, int version) throws IOException
+        {
+            long l = in.readLong();
+            for (int i = 0; i < size - 1; i++)
+            {
+                if (in.readLong() != l)
+                    throw new AssertionError();
+            }
+
+            return l;
+        }
+
+        public long serializedSize(Long t, int version)
+        {
+            return Long.BYTES * size;
+        }
+    }
+    interface ManualSendTest
+    {
+        void accept(Pair<InboundConnectionSettings, OutboundConnectionSettings> settings, InboundSockets inbound, OutboundConnection outbound, InetAddressAndPort endpoint, InboundProxyHandler.Controller handler) throws Throwable;
+    }
+
+    private void testManual(ManualSendTest test) throws Throwable
+    {
+        for (ConnectionTest.Settings s: SETTINGS)
+        {
+            doTestManual(s, test);
+            cleanup();
+        }
+    }
+
+    private void testOneManual(ManualSendTest test) throws Throwable
+    {
+        testOneManual(test, 1);
+    }
+
+    private void testOneManual(ManualSendTest test, int i) throws Throwable
+    {
+        ConnectionTest.Settings s = SETTINGS.get(i);
+        doTestManual(s, test);
+        cleanup();
+    }
+
+    private void doTestManual(ConnectionTest.Settings settings, ManualSendTest test) throws Throwable
+    {
+        InetAddressAndPort endpoint = FBUtilities.getBroadcastAddressAndPort();
+
+        InboundConnectionSettings inboundSettings = settings.inbound.apply(new InboundConnectionSettings())
+                                                                    .withBindAddress(endpoint)
+                                                                    .withSocketFactory(factory);
+
+        InboundSockets inbound = new InboundSockets(Collections.singletonList(inboundSettings));
+
+        OutboundConnectionSettings outboundSettings = settings.outbound.apply(new OutboundConnectionSettings(endpoint))
+                                                                       .withConnectTo(endpoint)
+                                                                       .withDefaultReserveLimits()
+                                                                       .withSocketFactory(factory);
+
+        ResourceLimits.EndpointAndGlobal reserveCapacityInBytes = new ResourceLimits.EndpointAndGlobal(new ResourceLimits.Concurrent(outboundSettings.applicationSendQueueReserveEndpointCapacityInBytes), outboundSettings.applicationSendQueueReserveGlobalCapacityInBytes);
+        OutboundConnection outbound = new OutboundConnection(settings.type, outboundSettings, reserveCapacityInBytes);
+        try
+        {
+            InboundProxyHandler.Controller controller = new InboundProxyHandler.Controller();
+            inbound.open(pipeline -> {
+                InboundProxyHandler handler = new InboundProxyHandler(controller);
+                pipeline.addLast(handler);
+            }).sync();
+            test.accept(Pair.create(inboundSettings, outboundSettings), inbound, outbound, endpoint, controller);
+        }
+        finally
+        {
+            outbound.close(false);
+            inbound.close().get(30L, SECONDS);
+            outbound.close(false).get(30L, SECONDS);
+            MessagingService.instance().messageHandlers.clear();
+        }
+    }
+
+    private void connect(OutboundConnection outbound) throws Throwable
+    {
+        tryConnect(outbound, 10, SECONDS, true);
+    }
+
+    private void tryConnect(OutboundConnection outbound, long timeout, TimeUnit timeUnit, boolean throwOnFailure) throws Throwable
+    {
+        CountDownLatch connectionLatch = new CountDownLatch(1);
+        unsafeSetHandler(Verb._TEST_1, () -> v -> {
+            connectionLatch.countDown();
+        });
+        outbound.enqueue(Message.out(Verb._TEST_1, 1L));
+        connectionLatch.await(timeout, timeUnit);
+        if (throwOnFailure)
+            Assert.assertEquals(0, connectionLatch.getCount());
+    }
+}
diff --git a/test/unit/org/apache/cassandra/net/proxy/InboundProxyHandler.java b/test/unit/org/apache/cassandra/net/proxy/InboundProxyHandler.java
new file mode 100644
index 0000000..7e3b004
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/proxy/InboundProxyHandler.java
@@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.net.proxy;
+
+import java.util.ArrayDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.util.concurrent.EventExecutor;
+import io.netty.util.concurrent.ScheduledFuture;
+
+public class InboundProxyHandler extends ChannelInboundHandlerAdapter
+{
+    private final ArrayDeque<Forward> forwardQueue;
+    private ScheduledFuture scheduled = null;
+    private final Controller controller;
+    public InboundProxyHandler(Controller controller)
+    {
+        this.controller = controller;
+        this.forwardQueue = new ArrayDeque<>(1024);
+    }
+
+    @Override
+    public void channelActive(ChannelHandlerContext ctx) throws Exception
+    {
+        super.channelActive(ctx);
+        ctx.read();
+    }
+
+    @Override
+    public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+        controller.onDisconnect.run();
+
+        if (scheduled != null)
+        {
+            scheduled.cancel(true);
+            scheduled = null;
+        }
+
+        if (!forwardQueue.isEmpty())
+            forwardQueue.clear();
+
+        super.channelInactive(ctx);
+    }
+
+
+    @Override
+    public void channelRead(ChannelHandlerContext ctx, Object msg)
+    {
+        Forward forward = controller.forwardStrategy.forward(ctx, msg);
+        forwardQueue.offer(forward);
+        maybeScheduleNext(ctx.channel().eventLoop());
+        controller.onRead.run();
+        ctx.channel().read();
+    }
+
+    private void maybeScheduleNext(EventExecutor executor)
+    {
+        if (forwardQueue.isEmpty())
+        {
+            // Ran out of items to process
+            scheduled = null;
+        }
+        else if (scheduled == null)
+        {
+            // Schedule next available or let the last in line schedule it
+            Forward forward = forwardQueue.poll();
+            scheduled = forward.schedule(executor);
+            scheduled.addListener((e) -> {
+                scheduled = null;
+                maybeScheduleNext(executor);
+            });
+        }
+    }
+
+    private static class Forward
+    {
+        final long arrivedAt;
+        final long latency;
+        final Runnable handler;
+
+        private Forward(long arrivedAt, long latency, Runnable handler)
+        {
+            this.arrivedAt = arrivedAt;
+            this.latency = latency;
+            this.handler = handler;
+        }
+
+        ScheduledFuture schedule(EventExecutor executor)
+        {
+            long now = System.currentTimeMillis();
+            long elapsed = now - arrivedAt;
+            long runIn = latency - elapsed;
+
+            if (runIn > 0)
+                return executor.schedule(handler, runIn, TimeUnit.MILLISECONDS);
+            else
+                return executor.schedule(handler, 0, TimeUnit.MILLISECONDS);
+        }
+    }
+
+    private static class ForwardNormally implements ForwardStrategy
+    {
+        static ForwardNormally instance = new ForwardNormally();
+
+        public Forward forward(ChannelHandlerContext ctx, Object msg)
+        {
+            return new Forward(System.currentTimeMillis(),
+                               0,
+                               () -> ctx.fireChannelRead(msg));
+        }
+    }
+
+    public interface ForwardStrategy
+    {
+        public Forward forward(ChannelHandlerContext ctx, Object msg);
+    }
+
+    private static class ForwardWithLatency implements ForwardStrategy
+    {
+        private final long latency;
+        private final TimeUnit timeUnit;
+
+        ForwardWithLatency(long latency, TimeUnit timeUnit)
+        {
+            this.latency = latency;
+            this.timeUnit = timeUnit;
+        }
+
+        public Forward forward(ChannelHandlerContext ctx, Object msg)
+        {
+            return new Forward(System.currentTimeMillis(),
+                               timeUnit.toMillis(latency),
+                               () -> ctx.fireChannelRead(msg));
+        }
+    }
+
+    private static class CloseAfterRead implements ForwardStrategy
+    {
+        private final Runnable afterClose;
+
+        CloseAfterRead(Runnable afterClose)
+        {
+            this.afterClose = afterClose;
+        }
+
+        public Forward forward(ChannelHandlerContext ctx, Object msg)
+        {
+            return  new Forward(System.currentTimeMillis(),
+                                0,
+                                () -> {
+                                    ctx.channel().close().syncUninterruptibly();
+                                    afterClose.run();
+                                });
+        }
+    }
+
+    private static class TransformPayload<T> implements ForwardStrategy
+    {
+        private final Function<T, T> fn;
+
+        TransformPayload(Function<T, T> fn)
+        {
+            this.fn = fn;
+        }
+
+        public Forward forward(ChannelHandlerContext ctx, Object msg)
+        {
+            return new Forward(System.currentTimeMillis(),
+                               0,
+                               () -> ctx.fireChannelRead(fn.apply((T) msg)));
+        }
+    }
+
+    public static class Controller
+    {
+        private volatile InboundProxyHandler.ForwardStrategy forwardStrategy;
+        private volatile Runnable onRead = () -> {};
+        private volatile Runnable onDisconnect = () -> {};
+
+        public Controller()
+        {
+            this.forwardStrategy = ForwardNormally.instance;
+        }
+        public void onRead(Runnable onRead)
+        {
+            this.onRead = onRead;
+        }
+
+        public void onDisconnect(Runnable onDisconnect)
+        {
+            this.onDisconnect = onDisconnect;
+        }
+
+        public void reset()
+        {
+            this.forwardStrategy = ForwardNormally.instance;
+        }
+
+        public void withLatency(long latency, TimeUnit timeUnit)
+        {
+            this.forwardStrategy = new ForwardWithLatency(latency, timeUnit);
+        }
+
+        public void withCloseAfterRead(Runnable afterClose)
+        {
+            this.forwardStrategy = new CloseAfterRead(afterClose);
+        }
+
+        public <T> void withPayloadTransform(Function<T, T> fn)
+        {
+            this.forwardStrategy = new TransformPayload<>(fn);
+        }
+    }
+
+}
diff --git a/test/unit/org/apache/cassandra/net/proxy/ProxyHandlerTest.java b/test/unit/org/apache/cassandra/net/proxy/ProxyHandlerTest.java
new file mode 100644
index 0000000..d070f56
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/proxy/ProxyHandlerTest.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.net.proxy;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.local.LocalAddress;
+import io.netty.channel.local.LocalChannel;
+import io.netty.channel.local.LocalServerChannel;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.handler.logging.LogLevel;
+import io.netty.handler.logging.LoggingHandler;
+
+public class ProxyHandlerTest
+{
+    private final Object PAYLOAD = new Object();
+
+    @Test
+    public void testLatency() throws Throwable
+    {
+        test((proxyHandler, testHandler, channel) -> {
+            int count = 1;
+            long latency = 100;
+            CountDownLatch latch = new CountDownLatch(count);
+            long start = System.currentTimeMillis();
+            testHandler.onRead = new Consumer<Object>()
+            {
+                int last = -1;
+                public void accept(Object o)
+                {
+                    // Make sure that order is preserved
+                    Assert.assertEquals(last + 1, o);
+                    last = (int) o;
+
+                    long elapsed = System.currentTimeMillis() - start;
+                    Assert.assertTrue("Latency was:" + elapsed, elapsed > latency);
+                    latch.countDown();
+                }
+            };
+
+            proxyHandler.withLatency(latency, TimeUnit.MILLISECONDS);
+
+            for (int i = 0; i < count; i++)
+            {
+                ByteBuf bb = Unpooled.buffer(Integer.BYTES);
+                bb.writeInt(i);
+                channel.writeAndFlush(i);
+            }
+
+            Assert.assertTrue(latch.await(10, TimeUnit.SECONDS));
+        });
+    }
+
+    @Test
+    public void testNormalDelivery() throws Throwable
+    {
+        test((proxyHandler, testHandler, channelPipeline) -> {
+            int count = 10;
+            CountDownLatch latch = new CountDownLatch(count);
+            AtomicLong end = new AtomicLong();
+            testHandler.onRead = (o) -> {
+                end.set(System.currentTimeMillis());
+                latch.countDown();
+            };
+
+            for (int i = 0; i < count; i++)
+                channelPipeline.writeAndFlush(PAYLOAD);
+            Assert.assertTrue(latch.await(10, TimeUnit.SECONDS));
+
+        });
+    }
+
+    @Test
+    public void testLatencyForMany() throws Throwable
+    {
+        class Event {
+            private final long latency;
+            private final long start;
+            private final int idx;
+
+            Event(long latency, int idx)
+            {
+                this.latency = latency;
+                this.start = System.currentTimeMillis();
+                this.idx = idx;
+            }
+        }
+
+        test((proxyHandler, testHandler, channel) -> {
+            int count = 150;
+            CountDownLatch latch = new CountDownLatch(count);
+            AtomicInteger counter = new AtomicInteger();
+            testHandler.onRead = new Consumer<Object>()
+            {
+                int lastSeen = -1;
+                public void accept(Object o)
+                {
+                    Event e = (Event) o;
+                    Assert.assertEquals(lastSeen + 1, e.idx);
+                    lastSeen = e.idx;
+                    long elapsed = System.currentTimeMillis() - e.start;
+                    Assert.assertTrue(elapsed >= e.latency);
+                    counter.incrementAndGet();
+                    latch.countDown();
+                }
+            };
+
+            int idx = 0;
+            for (int i = 0; i < count / 3; i++)
+            {
+                for (long latency : new long[]{ 100, 200, 0 })
+                {
+                    proxyHandler.withLatency(latency, TimeUnit.MILLISECONDS);
+                    CountDownLatch read = new CountDownLatch(1);
+                    proxyHandler.onRead(read::countDown);
+                    channel.writeAndFlush(new Event(latency, idx++));
+                    Assert.assertTrue(read.await(10, TimeUnit.SECONDS));
+                }
+            }
+
+            Assert.assertTrue(latch.await(10, TimeUnit.SECONDS));
+            Assert.assertEquals(counter.get(), count);
+        });
+    }
+
+    private interface DoTest
+    {
+        public void doTest(InboundProxyHandler.Controller proxy, TestHandler testHandler, Channel channel) throws Throwable;
+    }
+
+
+    public void test(DoTest test) throws Throwable
+    {
+        EventLoopGroup serverGroup = new NioEventLoopGroup(1);
+        EventLoopGroup clientGroup = new NioEventLoopGroup(1);
+
+        InboundProxyHandler.Controller controller = new InboundProxyHandler.Controller();
+        InboundProxyHandler proxyHandler = new InboundProxyHandler(controller);
+        TestHandler testHandler = new TestHandler();
+
+        ServerBootstrap sb = new ServerBootstrap();
+        sb.group(serverGroup)
+          .channel(LocalServerChannel.class)
+          .childHandler(new ChannelInitializer<LocalChannel>() {
+              @Override
+              public void initChannel(LocalChannel ch)
+              {
+                  ch.pipeline()
+                    .addLast(proxyHandler)
+                    .addLast(testHandler);
+              }
+          })
+          .childOption(ChannelOption.AUTO_READ, false);
+
+        Bootstrap cb = new Bootstrap();
+        cb.group(clientGroup)
+          .channel(LocalChannel.class)
+          .handler(new ChannelInitializer<LocalChannel>() {
+              @Override
+              public void initChannel(LocalChannel ch) throws Exception {
+                  ch.pipeline()
+                    .addLast(new LoggingHandler(LogLevel.TRACE));
+              }
+          });
+
+        final LocalAddress addr = new LocalAddress("test");
+
+        Channel serverChannel = sb.bind(addr).sync().channel();
+
+        Channel clientChannel = cb.connect(addr).sync().channel();
+        test.doTest(controller, testHandler, clientChannel);
+
+        clientChannel.close();
+        serverChannel.close();
+        serverGroup.shutdownGracefully();
+        clientGroup.shutdownGracefully();
+    }
+
+
+    public static class TestHandler extends ChannelInboundHandlerAdapter
+    {
+        private Consumer<Object> onRead = (o) -> {};
+        @Override
+        public void channelRead(ChannelHandlerContext ctx, Object msg)
+        {
+            onRead.accept(msg);
+        }
+    }
+}


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