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 14:54:52 UTC
[cassandra] 01/02: Rewrite the internode messaging subsystem
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 32d7e01df2bbe8fcd21a6bf1d0691f912eecee75
Author: Benedict Elliott Smith <be...@apache.org>
AuthorDate: Sat Jul 28 07:09:01 2018 -0700
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