From 881f2172aee97b0adf5bd389be4153a6fa7c5591 Mon Sep 17 00:00:00 2001 From: Benedict Elliott Smith Date: Fri, 19 Dec 2025 19:10:30 +0000 Subject: [PATCH] Introduce: - Clean Shutdown/Restart - Rebootstrap to allow nodes to rejoin consensus if they are out of sync, or did not shutdown cleanly Improve: - Improve efficiency of BTreeReducingRangeMap - DurableBefore backed by BTreeReducingRangeMap - Soft reject new transactions when a replica has a backlog of work - system_accord_debug.command_store_ops supports starting journal replay Fix: - Ensure SequentialExecutor.owner is unset only by the owner - Detect and avoid taking two AccordExecutor locks simultaneously - MaxConflicts serializer - tryToExecuteListening after replay, to handle invalidated dependencies - TxnNamedRead does not close a RowIterator, leading to native memory leaks - GetLatestDepsNack serialization - journal.readLast() did not read last - DefaultRemoteListeners not correctly synchronised - RangeTxnScanner cancellation assumes was already started - Start cfk compaction before replay - txn_graph should avoid visiting parents twice (possible if two different dependency chains connecting them in the graph) - Topology.cloneEquivalentWithEpoch should also share nodeLookup and ranges, especially to accelerate computeWaitForEpoch/computeScope - Do not invoke slowReplicaDelay or slowCoordinatorDelay during restore - Do not fail if slowReplicaDelay or slowCoordinatorDelay are invoked without knowing the transaction patch by Benedict; reviewed by Alex Petrov and Ariel Weisberg for CASSANDRA-21355 --- .gitmodules | 4 +- modules/accord | 2 +- .../apache/cassandra/config/AccordSpec.java | 105 +- .../cassandra/config/DatabaseDescriptor.java | 3 +- .../cql3/conditions/ColumnCondition.java | 2 +- .../schema/AlterSchemaStatement.java | 2 +- .../statements/schema/KeyspaceAttributes.java | 2 +- .../statements/schema/TableAttributes.java | 2 +- .../cassandra/db/ColumnFamilyStore.java | 13 + .../db/compaction/CompactionIterator.java | 46 +- .../db/compaction/CursorCompactor.java | 3 + .../db/memtable/AbstractMemtable.java | 12 +- .../cassandra/db/memtable/Memtable.java | 4 +- .../db/streaming/CassandraStreamReceiver.java | 2 +- .../db/virtual/AccordDebugKeyspace.java | 88 +- .../index/accord/RouteIndexFormat.java | 12 +- .../index/accord/RouteJournalIndex.java | 8 +- .../index/sai/plan/QueryViewBuilder.java | 1 + .../io/util/CompressedFrameDataInputPlus.java | 120 ++ .../util/CompressedFrameDataOutputPlus.java | 141 ++ .../apache/cassandra/journal/Compactor.java | 6 +- .../apache/cassandra/journal/DumpUtil.java | 4 +- .../cassandra/journal/EntrySerializer.java | 24 +- .../org/apache/cassandra/journal/Flusher.java | 54 +- .../cassandra/journal/InMemoryIndex.java | 5 +- .../org/apache/cassandra/journal/Journal.java | 307 ++-- .../apache/cassandra/journal/Metadata.java | 9 +- .../apache/cassandra/journal/OnDiskIndex.java | 5 +- .../org/apache/cassandra/journal/Params.java | 13 +- .../org/apache/cassandra/journal/Segment.java | 3 +- .../cassandra/journal/StaticSegment.java | 87 +- .../cassandra/journal/ValueSerializer.java | 15 + src/java/org/apache/cassandra/net/Verb.java | 20 +- .../DistributedMetadataLogKeyspace.java | 2 +- .../cassandra/schema/KeyspaceParams.java | 2 +- .../cassandra/schema/SchemaKeyspace.java | 2 +- .../org/apache/cassandra/schema/TableId.java | 7 +- .../cassandra/schema/TableMetadata.java | 2 +- .../apache/cassandra/schema/TableParams.java | 2 +- .../cassandra/service/StorageService.java | 41 +- .../cassandra/service/accord/AccordCache.java | 12 +- .../service/accord/AccordCommandStore.java | 439 +++++- .../service/accord/AccordCommandStores.java | 82 +- .../service/accord/AccordDataStore.java | 217 ++- .../service/accord/AccordDurableOnFlush.java | 145 +- .../service/accord/AccordExecutor.java | 235 +++- .../AccordExecutorAbstractLockLoop.java | 12 +- .../AccordExecutorAbstractSemiSyncSubmit.java | 4 +- .../service/accord/AccordJournal.java | 1253 ----------------- .../service/accord/AccordJournalTable.java | 646 --------- .../accord/AccordJournalValueSerializers.java | 343 ----- .../service/accord/AccordKeyspace.java | 2 +- .../service/accord/AccordMessageSink.java | 1 + .../service/accord/AccordObjectSizes.java | 10 +- .../accord/AccordResponseVerbHandler.java | 1 + .../service/accord/AccordResult.java | 3 +- .../service/accord/AccordSafeCommand.java | 16 +- .../accord/AccordSafeCommandStore.java | 9 + .../accord/AccordSafeCommandsForKey.java | 4 +- .../service/accord/AccordSafeState.java | 6 +- .../service/accord/AccordService.java | 382 +++-- .../cassandra/service/accord/AccordTask.java | 45 +- .../service/accord/AccordVerbHandler.java | 1 + .../service/accord/IAccordService.java | 13 +- .../service/accord/InMemoryRangeIndex.java | 28 +- .../cassandra/service/accord/JournalKey.java | 31 +- .../cassandra/service/accord/RangeIndex.java | 23 +- .../cassandra/service/accord/TokenRange.java | 15 +- .../service/accord/api/AccordAgent.java | 54 +- .../service/accord/api/AccordRoutableKey.java | 2 +- .../accord/api/AccordTopologySorter.java | 4 +- .../service/accord/api/TokenKey.java | 6 +- .../service/accord/debug/DebugTxnGraph.java | 14 +- .../accord/interop/AccordInteropAdapter.java | 2 +- .../interop/AccordInteropExecution.java | 2 +- .../AbstractSegmentCompactor.java} | 44 +- .../service/accord/journal/AccordJournal.java | 698 +++++++++ .../accord/journal/CommandChangeWriter.java | 195 +++ .../accord/journal/CommandChanges.java | 363 +++++ .../journal/DurableBeforePersister.java | 54 + .../IndexedRange.java} | 10 +- .../{ => journal}/JournalRangeIndex.java | 27 +- .../accord/journal/MergeSerializer.java | 46 + .../accord/journal/MergeSerializers.java | 343 +++++ .../service/accord/journal/Merger.java | 99 ++ .../accord/journal/RangeSearchManager.java | 293 ++++ .../RangeSearcher.java} | 15 +- .../service/accord/journal/Replay.java | 283 ++++ .../service/accord/journal/ReplayMarkers.java | 100 ++ .../SegmentCompactor.java} | 6 +- .../SegmentRangeSearcher.java} | 38 +- .../journal/TableAndSegmentKeyIterator.java | 101 ++ .../accord/journal/TableKeyIterator.java | 100 ++ .../accord/journal/TableRecordIterator.java | 131 ++ ...opologyUpdate.java => TopologyRecord.java} | 332 ++--- .../service/accord/repair/AccordRepair.java | 4 +- .../{ => serializers}/AccordSerializers.java | 4 +- .../serializers/CommandStoreSerializers.java | 697 ++++++++- .../accord/serializers/EncodeAsVInt32.java | 6 + .../GetDurableBeforeSerializers.java | 2 +- .../accord/serializers/KeySerializers.java | 10 +- .../serializers/LatestDepsSerializers.java | 167 ++- .../serializers/RecoverySerializers.java | 80 +- .../SimpleReplySerializer.java} | 25 +- .../{ => topology}/AccordEndpointMapper.java | 2 +- .../accord/{ => topology}/AccordFastPath.java | 2 +- .../AccordFastPathCoordinator.java | 7 +- .../{ => topology}/AccordStaleReplicas.java | 2 +- .../{ => topology}/AccordSyncPropagator.java | 17 +- .../accord/{ => topology}/AccordTopology.java | 14 +- .../{ => topology}/AccordTopologyService.java | 28 +- .../{ => topology}/EndpointMapping.java | 10 +- .../FastPathStrategy.java | 2 +- .../{ => topology}/FetchTopologies.java | 3 +- .../InheritKeyspaceFastPathStrategy.java | 2 +- .../ParameterizedFastPathStrategy.java | 2 +- .../SimpleFastPathStrategy.java | 2 +- .../{ => topology}/WatermarkCollector.java | 6 +- .../service/accord/txn/TxnCondition.java | 2 +- .../service/accord/txn/TxnNamedRead.java | 9 +- .../cassandra/service/accord/txn/TxnRead.java | 2 +- .../service/accord/txn/TxnReference.java | 2 +- .../accord/txn/TxnReferenceOperation.java | 4 +- .../service/accord/txn/TxnUpdate.java | 2 +- .../apache/cassandra/tcm/ClusterMetadata.java | 6 +- .../cassandra/tcm/ClusterMetadataService.java | 4 +- .../tcm/StubClusterMetadataService.java | 4 +- .../tcm/compatibility/GossipHelper.java | 4 +- .../sequences/CancelCMSReconfiguration.java | 2 +- .../AccordMarkHardRemoved.java | 2 +- .../transformations/AccordMarkRejoining.java | 2 +- .../tcm/transformations/AccordMarkStale.java | 2 +- .../ReconfigureAccordFastPath.java | 2 +- .../cms/PrepareCMSReconfiguration.java | 2 +- .../tools/StandaloneJournalUtil.java | 9 +- src/java/org/apache/cassandra/utils/Crc.java | 6 +- .../apache/cassandra/utils/ExecutorUtils.java | 3 +- .../utils/JVMStabilityInspector.java | 3 +- test/conf/cassandra.yaml | 5 + .../distributed/impl/AbstractCluster.java | 2 +- .../cassandra/distributed/impl/Instance.java | 20 +- .../test/accord/AccordBootstrapTest.java | 2 +- ...rdCommandStoreTryExecuteListeningTest.java | 91 +- .../accord/AccordJournalIntegrationTest.java | 6 +- .../test/accord/AccordLoadTest.java | 116 +- .../test/accord/AccordMetricsTest.java | 2 +- .../test/accord/AccordProgressLogTest.java | 4 +- .../test/accord/AccordSimpleFastPathTest.java | 2 +- .../journal/AccordJournalReplayTest.java | 3 +- ...rnalAccessRouteIndexOnStartupRaceTest.java | 6 +- .../journal/StatefulJournalRestartTest.java | 2 +- .../test/log/ClusterMetadataTestHelper.java | 4 +- .../fuzz/topology/AccordBootstrapTest.java | 4 + .../fuzz/topology/AccordBounceTest.java | 13 +- .../fuzz/topology/AccordHardCatchupTest.java | 139 ++ .../fuzz/topology/AccordRebootstrapTest.java | 169 +++ .../fuzz/topology/JournalGCTest.java | 4 +- .../fuzz/topology/TopologyMixupTestBase.java | 9 +- .../accord/AccordJournalCompactionTest.java | 12 +- .../accord/BurnTestKeySerializers.java | 4 +- .../{ => journal}/AccordJournalBurnTest.java | 45 +- .../NemesisSegmentCompactor.java} | 6 +- .../test/AccordJournalSimulationTest.java | 4 +- .../config/DatabaseDescriptorRefTest.java | 7 +- .../cassandra/db/ColumnFamilyStoreTest.java | 4 +- .../cassandra/db/SchemaCQLHelperTest.java | 2 +- .../index/accord/RouteIndexTest.java | 19 +- .../org/apache/cassandra/io/Serializers.java | 31 +- .../apache/cassandra/journal/JournalTest.java | 8 +- .../apache/cassandra/journal/SegmentTest.java | 5 +- .../apache/cassandra/journal/TestParams.java | 8 +- .../cassandra/locator/MetaStrategyTest.java | 4 +- .../cassandra/schema/FastPathSchemaTest.java | 4 +- .../service/accord/AccordCacheTest.java | 4 +- .../service/accord/AccordCommandTest.java | 5 + .../accord/AccordJournalOrderTest.java | 4 +- .../service/accord/AccordMessageSinkTest.java | 1 + .../service/accord/AccordSerializersTest.java | 1 + .../accord/AccordStaleReplicasTest.java | 1 + .../service/accord/AccordTestUtils.java | 1 + .../service/accord/AccordTopologyTest.java | 2 + .../service/accord/CommandChangeTest.java | 6 +- .../accord/DurableBeforeIntegrationTest.java | 96 ++ .../service/accord/EndpointMappingTest.java | 1 + .../service/accord/EpochSyncTest.java | 4 + .../service/accord/FetchTopologiesTest.java | 1 + .../accord/MaxConflictsIntegrationTest.java | 86 ++ .../accord/RangeTreeRangeAccessor.java | 0 .../accord/RouteInMemoryIndexTest.java | 12 +- .../accord/SimpleAccordEndpointMapper.java | 1 + .../accord/SimulatedAccordCommandStore.java | 11 +- .../SimulatedAccordCommandStoreTestBase.java | 10 +- .../accord/SimulatedAccordTaskTest.java | 1 + .../accord/WatermarkCollectorTest.java | 1 + ...pdateTest.java => TopologyRecordTest.java} | 71 +- .../repair/RequiredResponseTrackerTest.java | 2 +- .../CommandStoreSerializersTest.java | 108 +- .../CommandsForKeySerializerTest.java | 3 +- .../accord/serializers/TokenKeyTest.java | 4 +- .../AccordFastPathCoordinatorTest.java | 4 +- .../AccordSyncPropagatorTest.java | 3 +- .../FastPathParsingTest.java | 2 +- .../ParameterizedFastPathStrategyTest.java | 6 +- .../SimpleFastPathStrategyTest.java | 2 +- .../tcm/ClusterMetadataSerializerTest.java | 4 +- .../cassandra/utils/AccordGenerators.java | 2 +- .../cassandra/utils/CassandraGenerators.java | 12 +- .../utils/SimpleBitSetSerializersTest.java | 2 +- 208 files changed, 7138 insertions(+), 3790 deletions(-) create mode 100644 src/java/org/apache/cassandra/io/util/CompressedFrameDataInputPlus.java create mode 100644 src/java/org/apache/cassandra/io/util/CompressedFrameDataOutputPlus.java delete mode 100644 src/java/org/apache/cassandra/service/accord/AccordJournal.java delete mode 100644 src/java/org/apache/cassandra/service/accord/AccordJournalTable.java delete mode 100644 src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java rename src/java/org/apache/cassandra/service/accord/{AbstractAccordSegmentCompactor.java => journal/AbstractSegmentCompactor.java} (85%) create mode 100644 src/java/org/apache/cassandra/service/accord/journal/AccordJournal.java create mode 100644 src/java/org/apache/cassandra/service/accord/journal/CommandChangeWriter.java create mode 100644 src/java/org/apache/cassandra/service/accord/journal/CommandChanges.java create mode 100644 src/java/org/apache/cassandra/service/accord/journal/DurableBeforePersister.java rename src/java/org/apache/cassandra/service/accord/{IndexRange.java => journal/IndexedRange.java} (87%) rename src/java/org/apache/cassandra/service/accord/{ => journal}/JournalRangeIndex.java (95%) create mode 100644 src/java/org/apache/cassandra/service/accord/journal/MergeSerializer.java create mode 100644 src/java/org/apache/cassandra/service/accord/journal/MergeSerializers.java create mode 100644 src/java/org/apache/cassandra/service/accord/journal/Merger.java create mode 100644 src/java/org/apache/cassandra/service/accord/journal/RangeSearchManager.java rename src/java/org/apache/cassandra/service/accord/{JournalRangeSearcher.java => journal/RangeSearcher.java} (90%) create mode 100644 src/java/org/apache/cassandra/service/accord/journal/Replay.java create mode 100644 src/java/org/apache/cassandra/service/accord/journal/ReplayMarkers.java rename src/java/org/apache/cassandra/service/accord/{AccordSegmentCompactor.java => journal/SegmentCompactor.java} (91%) rename src/java/org/apache/cassandra/service/accord/{JournalSegmentRangeSearcher.java => journal/SegmentRangeSearcher.java} (87%) create mode 100644 src/java/org/apache/cassandra/service/accord/journal/TableAndSegmentKeyIterator.java create mode 100644 src/java/org/apache/cassandra/service/accord/journal/TableKeyIterator.java create mode 100644 src/java/org/apache/cassandra/service/accord/journal/TableRecordIterator.java rename src/java/org/apache/cassandra/service/accord/journal/{AccordTopologyUpdate.java => TopologyRecord.java} (71%) rename src/java/org/apache/cassandra/service/accord/{ => serializers}/AccordSerializers.java (97%) rename src/java/org/apache/cassandra/service/accord/{ImmediateAsyncExecutor.java => serializers/SimpleReplySerializer.java} (58%) rename src/java/org/apache/cassandra/service/accord/{ => topology}/AccordEndpointMapper.java (97%) rename src/java/org/apache/cassandra/service/accord/{ => topology}/AccordFastPath.java (99%) rename src/java/org/apache/cassandra/service/accord/{ => topology}/AccordFastPathCoordinator.java (97%) rename src/java/org/apache/cassandra/service/accord/{ => topology}/AccordStaleReplicas.java (99%) rename src/java/org/apache/cassandra/service/accord/{ => topology}/AccordSyncPropagator.java (96%) rename src/java/org/apache/cassandra/service/accord/{ => topology}/AccordTopology.java (97%) rename src/java/org/apache/cassandra/service/accord/{ => topology}/AccordTopologyService.java (91%) rename src/java/org/apache/cassandra/service/accord/{ => topology}/EndpointMapping.java (96%) rename src/java/org/apache/cassandra/service/accord/{fastpath => topology}/FastPathStrategy.java (99%) rename src/java/org/apache/cassandra/service/accord/{ => topology}/FetchTopologies.java (98%) rename src/java/org/apache/cassandra/service/accord/{fastpath => topology}/InheritKeyspaceFastPathStrategy.java (97%) rename src/java/org/apache/cassandra/service/accord/{fastpath => topology}/ParameterizedFastPathStrategy.java (99%) rename src/java/org/apache/cassandra/service/accord/{fastpath => topology}/SimpleFastPathStrategy.java (98%) rename src/java/org/apache/cassandra/service/accord/{ => topology}/WatermarkCollector.java (97%) create mode 100644 test/distributed/org/apache/cassandra/fuzz/topology/AccordHardCatchupTest.java create mode 100644 test/distributed/org/apache/cassandra/fuzz/topology/AccordRebootstrapTest.java rename test/distributed/org/apache/cassandra/service/accord/{ => journal}/AccordJournalBurnTest.java (91%) rename test/distributed/org/apache/cassandra/service/accord/{NemesisAccordSegmentCompactor.java => journal/NemesisSegmentCompactor.java} (93%) create mode 100644 test/unit/org/apache/cassandra/service/accord/DurableBeforeIntegrationTest.java create mode 100644 test/unit/org/apache/cassandra/service/accord/MaxConflictsIntegrationTest.java rename {src/java => test/unit}/org/apache/cassandra/service/accord/RangeTreeRangeAccessor.java (100%) rename test/unit/org/apache/cassandra/service/accord/journal/{AccordTopologyUpdateTest.java => TopologyRecordTest.java} (58%) rename test/unit/org/apache/cassandra/service/accord/{ => topology}/AccordFastPathCoordinatorTest.java (98%) rename test/unit/org/apache/cassandra/service/accord/{ => topology}/AccordSyncPropagatorTest.java (99%) rename test/unit/org/apache/cassandra/service/accord/{fastpath => topology}/FastPathParsingTest.java (98%) rename test/unit/org/apache/cassandra/service/accord/{fastpath => topology}/ParameterizedFastPathStrategyTest.java (96%) rename test/unit/org/apache/cassandra/service/accord/{fastpath => topology}/SimpleFastPathStrategyTest.java (96%) diff --git a/.gitmodules b/.gitmodules index 616dacf610a7..fc098aecd182 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,4 +1,4 @@ [submodule "modules/accord"] path = modules/accord - url = https://github.com/apache/cassandra-accord.git - branch = trunk + url = https://github.com/belliottsmith/cassandra-accord.git + branch = fixes-260226 diff --git a/modules/accord b/modules/accord index 8229cbf269f1..3e61ac97cc73 160000 --- a/modules/accord +++ b/modules/accord @@ -1 +1 @@ -Subproject commit 8229cbf269f14553c338cf47efc9466bbc7ee81b +Subproject commit 3e61ac97cc73e904a971637f2736c7a48070091a diff --git a/src/java/org/apache/cassandra/config/AccordSpec.java b/src/java/org/apache/cassandra/config/AccordSpec.java index 49ec58b7803b..9965a52e594e 100644 --- a/src/java/org/apache/cassandra/config/AccordSpec.java +++ b/src/java/org/apache/cassandra/config/AccordSpec.java @@ -28,18 +28,19 @@ import org.apache.cassandra.service.accord.serializers.Version; import org.apache.cassandra.service.consensus.TransactionalMode; +import static org.apache.cassandra.config.AccordSpec.CatchupMode.NORMAL; import static org.apache.cassandra.config.AccordSpec.QueueShardModel.THREAD_POOL_PER_SHARD; import static org.apache.cassandra.config.AccordSpec.QueueSubmissionModel.SYNC; import static org.apache.cassandra.config.AccordSpec.RangeIndexMode.in_memory; +// TODO (expected): rename to AccordConf? public class AccordSpec { public volatile boolean enabled = false; + // TODO (expected): move to JournalSpec public volatile String journal_directory; - public volatile boolean enable_journal_compaction = true; - /** * Enables the virtual Accord debug-only keyspace with tables * that expose internal state to aid the developers working @@ -92,6 +93,8 @@ public enum QueueSubmissionModel /** * The queue workers only require ownership of the lock, submissions happens fully asynchronously. + * + * NOTE: EXPERIMENTAL */ ASYNC, @@ -136,16 +139,16 @@ public enum QueueSubmissionModel public DurationSpec.IntMillisecondsBound repair_timeout = new DurationSpec.IntMillisecondsBound("10m"); public String recover_txn = "5s*attempts <= 60s"; public StringRetryStrategy recover_syncpoint = new StringRetryStrategy("60s <= 30s*attempts...60s*attempts <= 600s"); - public String fetch_txn = "1s*attempts"; - public String fetch_syncpoint = "5s*attempts"; - public String expire_txn = "5s*attempts"; + public String fetch_txn = "2s*attempts <= 60s"; + public String fetch_syncpoint = "5s*attempts <= 60s"; + public String expire_txn = "5s*attempts <= 60s"; public String expire_syncpoint = "60s*attempts<=300s"; public String expire_epoch_wait = "10s"; // we don't want to wait ages for durability as it blocks other durability progress; even this might be too long, as we can always retry public String expire_durability = "10s*attempts <= 30s"; public String slow_syncpoint_preaccept = "10s"; - public String slow_txn_preaccept = "30ms <= p50*2 <= 100ms"; - public String slow_read = "30ms <= p50*2 <= 100ms"; + public String slow_txn_preaccept = "30ms <= p50*2 <= 1000ms"; + public String slow_read = "30ms <= p50*2 <= 1000ms"; public StringRetryStrategy retry_syncpoint = new StringRetryStrategy("10s*attempt <= 600s"); public StringRetryStrategy retry_durability = new StringRetryStrategy("10s*attempt <= 600s"); public StringRetryStrategy retry_bootstrap = new StringRetryStrategy("10s*attempt <= 600s"); @@ -156,8 +159,8 @@ public enum QueueSubmissionModel public volatile DurationSpec.IntSecondsBound fast_path_update_delay = null; - public volatile int shard_durability_target_splits = 16; - public volatile int shard_durability_max_splits = 128; + public volatile int shard_durability_target_splits = 8; + public volatile int shard_durability_max_splits = 64; public volatile DurationSpec.IntSecondsBound durability_txnid_lag = new DurationSpec.IntSecondsBound(5); public volatile DurationSpec.IntSecondsBound shard_durability_cycle = new DurationSpec.IntSecondsBound(5, TimeUnit.MINUTES); public volatile DurationSpec.IntSecondsBound global_durability_cycle = new DurationSpec.IntSecondsBound(5, TimeUnit.MINUTES); @@ -175,19 +178,39 @@ public enum TransactionalRangeMigration */ public volatile TransactionalRangeMigration range_migration = TransactionalRangeMigration.auto; + public enum RebootstrapMode + { + full_repair, truncate_and_stream + } + + public enum CatchupMode + { + DISABLED, + NORMAL, + FALLBACK_TO_HARD, + HARD + } + /** * default transactional mode for tables created by this node when no transactional mode has been specified in the DDL */ public TransactionalMode default_transactional_mode = TransactionalMode.off; public boolean ephemeralReadEnabled = true; - public boolean state_cache_listener_jfr_enabled = true; + public boolean state_cache_listener_jfr_enabled = false; + + public float hard_reject_ratio = 0.5f; + public int min_soft_reject_count = 10; + public int max_soft_reject_count = 100; + public DurationSpec.LongMicrosecondsBound soft_reject_age = new DurationSpec.LongMicrosecondsBound("10s"); + public DurationSpec.LongMicrosecondsBound soft_reject_cumulative_age = new DurationSpec.LongMicrosecondsBound("60s"); public DurationSpec.IntSecondsBound catchup_on_start_success_latency = new DurationSpec.IntSecondsBound(60); public DurationSpec.IntSecondsBound catchup_on_start_fail_latency = new DurationSpec.IntSecondsBound(900); public int catchup_on_start_max_attempts = 5; // TODO (required): roll this back to catchup_on_start_exit_on_failure: true public boolean catchup_on_start_exit_on_failure = false; - public boolean catchup_on_start = true; + public CatchupMode catchup_on_start = NORMAL; + public DurationSpec.IntSecondsBound shutdown_grace_period = new DurationSpec.IntSecondsBound(15 * 60); public enum RangeIndexMode { in_memory, journal_sai } public RangeIndexMode range_index_mode = in_memory; @@ -203,16 +226,68 @@ public enum MixedTimeSourceHandling public static class JournalSpec implements Params { + public enum ReplayMode + { + /** + * Replay all journal entries and erase local state such as CommandsForKey that can be recreated + */ + RESET, + + /** + * Replay all journal entries + */ + ALL, + + /** + * Replay journal entries for commands that intersect a non-durable range. + * Ordinarily it should be necessary to only replay commands that do not intersect any durable ranges. + */ + PART_NON_DURABLE, + + /** + * Replay journal entries for commands that are not durable to the data or command stores. + * THIS MODE IS NOT YET SAFE TO RUN + */ + NON_DURABLE + } + + public enum ReplaySavePoint + { + NO, + LATEST + } + + public enum StopMarkerFailurePolicy + { + /** + * If the start marker exceeds the stop marker then exit, since we cannot guarantee our consensus log is complete. + */ + EXIT, + + /** + * If the start marker exceeds the stop marker startup, assuming the consensus log has been determined complete externally. + * Note this is VERY UNSAFE if you care about isolation guarantees. + */ + UNSAFE_STARTUP, + + REBOOTSTRAP + } + public int segmentSize = 32 << 20; public int compactMaxSegments = 32; public FailurePolicy failurePolicy = FailurePolicy.STOP; - public ReplayMode replayMode = ReplayMode.ONLY_NON_DURABLE; + public ReplayMode replay = ReplayMode.PART_NON_DURABLE; + public ReplaySavePoint replaySavePoint = ReplaySavePoint.LATEST; + public int retainSavePoints = 2; + public StopMarkerFailurePolicy stopMarkerFailurePolicy = StopMarkerFailurePolicy.EXIT; + public RecoverableCrcFailurePolicy crcFailureOnRebuildPolicy = RecoverableCrcFailurePolicy.FAIL; public FlushMode flushMode = FlushMode.PERIODIC; public volatile DurationSpec flushPeriod; // pulls default from 'commitlog_sync_period' public DurationSpec periodicFlushLagBlock = new DurationSpec.IntMillisecondsBound("1500ms"); public DurationSpec.IntMillisecondsBound compactionPeriod = new DurationSpec.IntMillisecondsBound("60000ms"); private volatile long flushCombinedBlockPeriod = Long.MIN_VALUE; public Version version = Version.DOWNGRADE_SAFE_VERSION; + public boolean enable_compaction = true; public JournalSpec setFlushPeriod(DurationSpec newFlushPeriod) { @@ -246,9 +321,9 @@ public FailurePolicy failurePolicy() } @Override - public ReplayMode replayMode() + public RecoverableCrcFailurePolicy crcFailureOnRebuildPolicy() { - return replayMode; + return crcFailureOnRebuildPolicy; } @Override @@ -260,7 +335,7 @@ public FlushMode flushMode() @Override public boolean enableCompaction() { - return DatabaseDescriptor.getAccord().enable_journal_compaction; + return enable_compaction; } @Override diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 17666c9abf59..aec9c0267187 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -5689,8 +5689,7 @@ public static int getAccordQueueShardCount() case THREAD_PER_SHARD_SYNC_QUEUE: return conf.accord.queue_shard_count.or(DatabaseDescriptor::getAvailableProcessors); case THREAD_POOL_PER_SHARD: - int defaultMax = getAccordQueueSubmissionModel() == AccordSpec.QueueSubmissionModel.SYNC ? 8 : 4; - return conf.accord.queue_shard_count.or(Math.min(defaultMax, DatabaseDescriptor.getAvailableProcessors())); + return conf.accord.queue_shard_count.or(DatabaseDescriptor.getAvailableProcessors()/4); } } diff --git a/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java b/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java index 37e764afd56e..ae8de72ddf14 100644 --- a/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java +++ b/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java @@ -57,7 +57,7 @@ import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull; import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt; -import static org.apache.cassandra.service.accord.AccordSerializers.columnMetadataSerializer; +import static org.apache.cassandra.service.accord.serializers.AccordSerializers.columnMetadataSerializer; import static org.apache.cassandra.utils.ByteBufferUtil.nullableByteBufferSerializer; /** diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java index 062dad75410c..a75501ed120f 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java @@ -45,7 +45,7 @@ import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.ClientWarn; import org.apache.cassandra.service.QueryState; -import org.apache.cassandra.service.accord.AccordTopology; +import org.apache.cassandra.service.accord.topology.AccordTopology; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.transport.Dispatcher; import org.apache.cassandra.transport.Event.SchemaChange; diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/KeyspaceAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/KeyspaceAttributes.java index d4ca44cdcf64..6b73d709be27 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/KeyspaceAttributes.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/KeyspaceAttributes.java @@ -30,7 +30,7 @@ import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.KeyspaceParams.Option; import org.apache.cassandra.schema.ReplicationParams; -import org.apache.cassandra.service.accord.fastpath.FastPathStrategy; +import org.apache.cassandra.service.accord.topology.FastPathStrategy; public final class KeyspaceAttributes extends PropertyDefinitions { diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java index 9ec04f502bc6..9e84b2d3cc08 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java @@ -37,7 +37,7 @@ import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableParams; import org.apache.cassandra.schema.TableParams.Option; -import org.apache.cassandra.service.accord.fastpath.FastPathStrategy; +import org.apache.cassandra.service.accord.topology.FastPathStrategy; import org.apache.cassandra.service.consensus.TransactionalMode; import org.apache.cassandra.service.consensus.migration.TransactionalMigrationFromMode; import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index aa62443a48a6..df862d3dc73d 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -408,8 +408,16 @@ public static Runnable getBackgroundCompactionTaskSubmitter() { return () -> { for (Keyspace keyspace : Keyspace.all()) + { for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores()) + { + if (SchemaConstants.ACCORD_KEYSPACE_NAME.equals(cfs.keyspace.getName())) + continue; + CompactionManager.instance.submitBackground(cfs); + } + } + }; } @@ -1144,6 +1152,11 @@ private Future waitForFlushes() return postFlushExecutor.submit(current::getCommitLogLowerBound); } + public Future waitForPriorFlushes() + { + return postFlushExecutor.submit(() -> null); + } + public CommitLogPosition forceBlockingFlush(FlushReason reason) { return FBUtilities.waitOnFuture(forceFlush(reason)); diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java index 7f97dc0663cc..9c1ec42defcf 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java @@ -89,9 +89,6 @@ import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.ClientState; -import org.apache.cassandra.service.accord.AccordJournal; -import org.apache.cassandra.service.accord.AccordJournalValueSerializers.FlyweightImage; -import org.apache.cassandra.service.accord.AccordJournalValueSerializers.FlyweightSerializer; import org.apache.cassandra.service.accord.AccordKeyspace; import org.apache.cassandra.service.accord.AccordKeyspace.CommandsForKeyAccessor; import org.apache.cassandra.service.accord.AccordService; @@ -100,8 +97,14 @@ import org.apache.cassandra.service.accord.IAccordService.AccordCompactionInfos; import org.apache.cassandra.service.accord.JournalKey; import org.apache.cassandra.service.accord.api.TokenKey; -import org.apache.cassandra.service.accord.journal.AccordTopologyUpdate; -import org.apache.cassandra.service.accord.journal.AccordTopologyUpdate.TopologyImage; +import org.apache.cassandra.service.accord.journal.CommandChangeWriter; +import org.apache.cassandra.service.accord.journal.CommandChanges; +import org.apache.cassandra.service.accord.journal.MergeSerializer; +import org.apache.cassandra.service.accord.journal.MergeSerializers; +import org.apache.cassandra.service.accord.journal.MergeSerializers.CommandChangeSerializer; +import org.apache.cassandra.service.accord.journal.Merger; +import org.apache.cassandra.service.accord.journal.TopologyRecord; +import org.apache.cassandra.service.accord.journal.TopologyRecord.TopologyImage; import org.apache.cassandra.service.accord.serializers.Version; import org.apache.cassandra.service.paxos.PaxosRepairHistory; import org.apache.cassandra.service.paxos.uncommitted.PaxosRows; @@ -122,8 +125,9 @@ import static org.apache.cassandra.config.DatabaseDescriptor.paxosStatePurging; import static org.apache.cassandra.service.accord.AccordKeyspace.CFKAccessor; import static org.apache.cassandra.service.accord.AccordKeyspace.JournalColumns.getJournalKey; -import static org.apache.cassandra.service.accord.journal.AccordTopologyUpdate.Kind.Image; -import static org.apache.cassandra.service.accord.journal.AccordTopologyUpdate.Kind.Repeat; +import static org.apache.cassandra.service.accord.journal.MergeSerializers.TopologySerializer.INSTANCE; +import static org.apache.cassandra.service.accord.journal.TopologyRecord.Kind.Image; +import static org.apache.cassandra.service.accord.journal.TopologyRecord.Kind.Repeat; /** * Merge multiple iterators over the content of sstable into a "compacted" iterator. @@ -875,7 +879,7 @@ class AccordJournalPurger extends AbstractPurger final ColumnMetadata versionColumn; JournalKey key; - AccordRowCompactor compactor; + AccordRowCompactor compactor; final Version userVersion; public AccordJournalPurger(AccordCompactionInfos compactionInfos, Version version, ColumnFamilyStore cfs) @@ -899,7 +903,7 @@ protected void beginPartition(UnfilteredRowIterator partition) compactor = new AccordCommandRowCompactor(infos, userVersion, nowInSec); break; case TOPOLOGY_UPDATE: - compactor = new TopologyCompactor((FlyweightSerializer) key.type.serializer, userVersion, infos.minEpoch); + compactor = new TopologyCompactor(userVersion, infos.minEpoch); break; default: compactor = new AccordMergingCompactor(key.type.serializer, userVersion); @@ -939,11 +943,11 @@ protected void collect(Row row) throws IOException } } - static abstract class AccordRowCompactor + static abstract class AccordRowCompactor { - final FlyweightSerializer serializer; + final MergeSerializer serializer; - AccordRowCompactor(FlyweightSerializer serializer) + AccordRowCompactor(MergeSerializer serializer) { this.serializer = serializer; } @@ -953,15 +957,15 @@ static abstract class AccordRowCompactor abstract UnfilteredRowIterator result(JournalKey journalKey, DecoratedKey partitionKey) throws IOException; } - static class TopologyCompactor extends AccordMergingCompactor + static class TopologyCompactor extends AccordMergingCompactor { TopologyImage lastImage; boolean hasWritten; final long minEpoch; - TopologyCompactor(FlyweightSerializer serializer, Version userVersion, long minEpoch) + TopologyCompactor(Version userVersion, long minEpoch) { - super(serializer, userVersion); + super(INSTANCE, userVersion); this.minEpoch = minEpoch; } @@ -1004,7 +1008,7 @@ else if (hasWritten && read.kind() == Repeat && lastImage.getUpdate().isEquivale } } - static class AccordMergingCompactor extends AccordRowCompactor + static class AccordMergingCompactor extends AccordRowCompactor { final T builder; final Version userVersion; @@ -1012,7 +1016,7 @@ static class AccordMergingCompactor extends AccordRowC long lastDescriptor; int lastOffset; - AccordMergingCompactor(FlyweightSerializer serializer, Version userVersion) + AccordMergingCompactor(MergeSerializer serializer, Version userVersion) { super(serializer); this.builder = serializer.mergerFor(); @@ -1072,7 +1076,7 @@ UnfilteredRowIterator result(JournalKey journalKey, DecoratedKey partitionKey) t static class AccordCommandRowEntry { - final AccordJournal.Builder builder = new AccordJournal.Builder(); + final CommandChanges builder = new CommandChanges(); Row row; boolean modified; @@ -1094,7 +1098,7 @@ void clear() } } - static class AccordCommandRowCompactor extends AccordRowCompactor + static class AccordCommandRowCompactor extends AccordRowCompactor { static final Object[] rowTemplate = BTree.build(BulkIterator.of(new Object[2]), 2, UpdateFunction.noOp); final long timestamp = ClientState.getTimestamp(); @@ -1103,14 +1107,14 @@ static class AccordCommandRowCompactor extends AccordRowCompactor entries = new ArrayList<>(); final ArrayDeque reuseEntries = new ArrayDeque<>(); AccordCompactionInfo info; AccordCommandRowCompactor(AccordCompactionInfos infos, Version userVersion, long nowInSec) { - super((FlyweightSerializer) JournalKey.Type.COMMAND_DIFF.serializer); + super((CommandChangeSerializer) JournalKey.Type.COMMAND_DIFF.serializer); this.infos = infos; this.userVersion = userVersion; this.userVersionCell = BufferCell.live(AccordKeyspace.JournalColumns.user_version, timestamp, Int32Type.instance.decompose(userVersion.version)); diff --git a/src/java/org/apache/cassandra/db/compaction/CursorCompactor.java b/src/java/org/apache/cassandra/db/compaction/CursorCompactor.java index 7d528b5e2d9d..438c5a06dc9b 100644 --- a/src/java/org/apache/cassandra/db/compaction/CursorCompactor.java +++ b/src/java/org/apache/cassandra/db/compaction/CursorCompactor.java @@ -157,6 +157,9 @@ public static boolean isSupported(AbstractCompactionStrategy.ScannerList scanner public static boolean unsupportedMetadata(TableMetadata metadata) { + if (metadata.keyspace.equals(SchemaConstants.ACCORD_KEYSPACE_NAME)) + return true; + if (!metadata.partitioner.supportsReusableKeys()) { if (LOGGER.isDebugEnabled()) logDebugReason(metadata, "Incompatible partitioner, does not support reusable keys:" + metadata.partitioner.getClass().getSimpleName()); diff --git a/src/java/org/apache/cassandra/db/memtable/AbstractMemtable.java b/src/java/org/apache/cassandra/db/memtable/AbstractMemtable.java index 1b6f999daa33..7a99d213dbda 100644 --- a/src/java/org/apache/cassandra/db/memtable/AbstractMemtable.java +++ b/src/java/org/apache/cassandra/db/memtable/AbstractMemtable.java @@ -26,7 +26,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Consumer; +import java.util.function.BiConsumer; import java.util.function.Supplier; import com.google.common.annotations.VisibleForTesting; @@ -56,7 +56,7 @@ public abstract class AbstractMemtable implements Memtable // The smallest local deletion time for all partitions in this memtable protected AtomicLong minLocalDeletionTime = new AtomicLong(Long.MAX_VALUE); private final long id = nextId.incrementAndGet(); - private Map> onFlush = ImmutableMap.of(); + private Map> onFlush = ImmutableMap.of(); // Note: statsCollector has corresponding statistics to the two above, but starts with an epoch value which is not // correct for their usage. @@ -154,7 +154,7 @@ public LifecycleTransaction setFlushTransaction(LifecycleTransaction flushTransa } @Override - public synchronized > T ensureFlushListener(Object key, Supplier factory) + public synchronized > T ensureFlushListener(Object key, Supplier factory) { if (onFlush == null) return null; @@ -163,7 +163,7 @@ public synchronized > T ensureFlushListener(Ob if (null == listener) { listener = factory.get(); - onFlush = ImmutableMap.>builder() + onFlush = ImmutableMap.>builder() .putAll(onFlush) .put(key, listener) .build(); @@ -173,13 +173,13 @@ public synchronized > T ensureFlushListener(Ob public void notifyFlushed() { - Collection> run; + Collection> run; synchronized (this) { run = onFlush.values(); onFlush = null; } - run.forEach(c -> c.accept(metadata())); + run.forEach(c -> c.accept(id, metadata())); } protected static class ColumnsCollector diff --git a/src/java/org/apache/cassandra/db/memtable/Memtable.java b/src/java/org/apache/cassandra/db/memtable/Memtable.java index ce9992b91e1b..41b5351cba84 100644 --- a/src/java/org/apache/cassandra/db/memtable/Memtable.java +++ b/src/java/org/apache/cassandra/db/memtable/Memtable.java @@ -19,7 +19,7 @@ package org.apache.cassandra.db.memtable; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Consumer; +import java.util.function.BiConsumer; import java.util.function.Supplier; import javax.annotation.concurrent.NotThreadSafe; @@ -425,7 +425,7 @@ default boolean shouldSwitch(ColumnFamilyStore.FlushReason reason) } // returns null if already flushed - > T ensureFlushListener(Object key, Supplier factory); + > T ensureFlushListener(Object key, Supplier factory); void notifyFlushed(); /** diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java index 4d3e830080f5..1bbc3d3507a4 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java @@ -50,9 +50,9 @@ import org.apache.cassandra.io.sstable.SSTableTxnSingleStreamWriter; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.service.accord.AccordService; -import org.apache.cassandra.service.accord.AccordTopology; import org.apache.cassandra.service.accord.IAccordService; import org.apache.cassandra.service.accord.TimeOnlyRequestBookkeeping.LatencyRequestBookkeeping; +import org.apache.cassandra.service.accord.topology.AccordTopology; import org.apache.cassandra.streaming.IncomingStream; import org.apache.cassandra.streaming.StreamReceiver; import org.apache.cassandra.streaming.StreamSession; diff --git a/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java b/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java index c6915ec2268c..89b801d0e3b3 100644 --- a/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java +++ b/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java @@ -36,6 +36,7 @@ import java.util.function.BiConsumer; import java.util.function.Function; import java.util.function.IntFunction; +import java.util.function.Supplier; import java.util.function.UnaryOperator; import java.util.stream.Collectors; @@ -61,6 +62,7 @@ import accord.impl.progresslog.DefaultProgressLog; import accord.impl.progresslog.DefaultProgressLog.ModeFlag; import accord.impl.progresslog.TxnStateKind; +import accord.local.CatchupHard; import accord.local.Cleanup; import accord.local.Command; import accord.local.CommandStore; @@ -105,6 +107,7 @@ import accord.utils.async.AsyncResult; import accord.utils.async.AsyncResults; +import org.apache.cassandra.config.AccordSpec.JournalSpec.ReplayMode; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.statements.schema.CreateTableStatement; import org.apache.cassandra.db.ColumnFamilyStore; @@ -135,7 +138,6 @@ import org.apache.cassandra.service.accord.AccordCommandStore; import org.apache.cassandra.service.accord.AccordCommandStores; import org.apache.cassandra.service.accord.AccordExecutor; -import org.apache.cassandra.service.accord.AccordJournal; import org.apache.cassandra.service.accord.AccordKeyspace; import org.apache.cassandra.service.accord.AccordOperations; import org.apache.cassandra.service.accord.AccordService; @@ -153,6 +155,7 @@ import org.apache.cassandra.service.accord.debug.DebugTxnDepsOrdered; import org.apache.cassandra.service.accord.debug.DebugTxnGraph; import org.apache.cassandra.service.accord.debug.TxnKindsAndDomains; +import org.apache.cassandra.service.accord.journal.AccordJournal; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; import org.apache.cassandra.service.consensus.migration.TableMigrationState; import org.apache.cassandra.tcm.ClusterMetadata; @@ -650,17 +653,17 @@ private DurableBeforeTable() public void collect(PartitionsCollector collector) { DurableBefore durableBefore = AccordService.unsafeInstance().node().durableBefore(); - durableBefore.foldlWithBounds( - (entry, ignore, start, end) -> { - TableId tableId = (TableId) start.prefix(); - collector.row(tableId.toString(), printToken(start)) + durableBefore.foldl( + (entry, ignore) -> { + TableId tableId = (TableId) entry.prefix(); + collector.row(tableId.toString(), printToken(entry.start())) .lazyCollect(columns -> { - columns.add("token_end", end, AccordDebugKeyspace::printToken) - .add("quorum", entry.quorumBefore, TO_STRING) - .add("universal", entry.universalBefore, TO_STRING); + columns.add("token_end", entry.end(), AccordDebugKeyspace::printToken) + .add("quorum", entry.quorum, TO_STRING) + .add("universal", entry.universal, TO_STRING); }); return null; - }, null, ignore -> false); + }, null); } } @@ -834,16 +837,16 @@ public void collect(PartitionsCollector collector) String tableIdStr = tableId.toString(); collector.partition(commandStoreId).collect(rows -> { - maxConflicts.foldlWithBounds( - (timestamp, rs, start, end) -> { - rows.add(printToken(start)) + maxConflicts.foldl( + (entry, rs) -> { + rows.add(printToken(entry.start())) .lazyCollect(columns -> { - columns.add("token_end", end, AccordDebugKeyspace::printToken) + columns.add("token_end", entry.end(), AccordDebugKeyspace::printToken) .add("table_id", tableIdStr) - .add("timestamp", timestamp, TO_STRING); + .add("timestamp", entry, TO_STRING); }); return rows; - }, rows, ignore -> false + }, rows ); }); } @@ -1565,7 +1568,7 @@ public void collect(PartitionsCollector collector) } } - accord.journal().forEach(key -> collect(collector, accord, key), min, max, true); + accord.journal().forEach(key -> collect(collector, accord, key), min, max, true, 0); } abstract void collect(PartitionsCollector collector, AccordService accord, JournalKey key); @@ -2020,6 +2023,9 @@ enum CommandStoreOp SET_PROGRESS_LOG_MODE("Set the specified progress log mode."), UNSET_PROGRESS_LOG_MODE("Unset the specified progress log mode."), TRY_EXECUTE_LISTENING("Try to execute all of the transactions (and their dependencies) that have registered listeners on other transactions."), + REPLAY("Run journal replay for all transactions"), + REBOOTSTRAP("Rebootstrap the command store. This invalidates the local journal, synchronises its data via data repair and rejoins the distributed state machine."), + HARD_CATCHUP("Hard catchup the command store. This invalidates the local journal for any ranges not up to date with some quorum, synchronises its data via data repair and rejoins the distributed state machine."), ; final String description; @@ -2071,12 +2077,16 @@ protected void applyRowUpdate(Object[] partitionKeys, Object[] clusteringKeys, C if (op == null) throw new IllegalArgumentException("Must specify 'op'"); + final AccordService accord = (AccordService) AccordService.unsafeInstance(); + final Node node = accord.node(); final Function> function; + Supplier> allFunction = null; switch (op) { default: throw new UnhandledEnum(op); case SET_PROGRESS_LOG_MODE: case UNSET_PROGRESS_LOG_MODE: + { if (param == null) throw new IllegalArgumentException("Must specify 'param' for " + op); ModeFlag mode = tryParse(param, true, ModeFlag.class, ModeFlag::valueOf); @@ -2088,25 +2098,53 @@ protected void applyRowUpdate(Object[] partitionKeys, Object[] clusteringKeys, C return AsyncResults.success(null); }; break; + } case TRY_EXECUTE_LISTENING: - if (param != null) - throw new IllegalArgumentException("'param' is not supported for " + op); - function = CommandStore::operatorTryToExecuteListeningTxns; + { + boolean loop; + if (param == null) loop = false; + else if (param.equalsIgnoreCase("loop")) loop = true; + else throw new InvalidRequestException("Unknown param for " + CommandStoreOp.TRY_EXECUTE_LISTENING + ": '" + param + "'; expect only 'loop' or missing"); + + function = commandStore -> commandStore.tryToExecuteListeningTxns(loop); + break; + } + case REPLAY: + { + ReplayMode replayMode = tryParse(param, true, ReplayMode.class, ReplayMode::valueOf); + function = commandStore -> { + accord.journal().replay(commandStore, replayMode, 0L); + return AsyncResults.success(null); + }; + break; + } + case REBOOTSTRAP: + allFunction = () -> node.commandStores().rebootstrap(node); + function = commandStore -> commandStore.rebootstrap(node); + break; + case HARD_CATCHUP: + allFunction = () -> CatchupHard.catchup(node, Arrays.asList(node.commandStores().all())).beginAsResult(); + function = commandStore -> CatchupHard.catchup(node, Collections.singletonList(commandStore)).beginAsResult(); break; } AsyncResult result; if (commandStoreId < 0) { - List> results = new ArrayList<>(); - AccordService.unsafeInstance().node() - .commandStores() - .forAllUnsafe(commandStore -> results.add(function.apply(commandStore))); - result = AsyncResults.allOf(results); + if (allFunction == null) + { + allFunction = () -> { + List> results = new ArrayList<>(); + for (CommandStore commandStore : node.commandStores().all()) + results.add(function.apply(commandStore)); + return AsyncResults.allOf(results); + }; + } + result = allFunction.get(); } else { - result = function.apply(AccordService.unsafeInstance().node().commandStores().forId(commandStoreId)); + result = function.apply(node.commandStores().forId(commandStoreId)); } AccordService.getBlocking(result); diff --git a/src/java/org/apache/cassandra/index/accord/RouteIndexFormat.java b/src/java/org/apache/cassandra/index/accord/RouteIndexFormat.java index 537f55ac3ab7..4e48ddb3f0b1 100644 --- a/src/java/org/apache/cassandra/index/accord/RouteIndexFormat.java +++ b/src/java/org/apache/cassandra/index/accord/RouteIndexFormat.java @@ -57,10 +57,10 @@ import org.apache.cassandra.io.util.FileHandle; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.serializers.UUIDSerializer; -import org.apache.cassandra.service.accord.AccordJournal; -import org.apache.cassandra.service.accord.AccordJournalTable; import org.apache.cassandra.service.accord.AccordKeyspace; import org.apache.cassandra.service.accord.JournalKey; +import org.apache.cassandra.service.accord.journal.AccordJournal; +import org.apache.cassandra.service.accord.journal.CommandChanges; import org.apache.cassandra.service.accord.serializers.CommandSerializers; import org.apache.cassandra.service.accord.serializers.KeySerializers; import org.apache.cassandra.service.accord.serializers.Version; @@ -160,7 +160,7 @@ public static ByteBuffer extractParticipants(RouteJournalIndex index, TxnId txnI return null; ByteBuffer record = recordCell.buffer(); Version user_version = Version.fromVersion(Int32Type.instance.compose(user_versionCell.buffer())); - AccordJournal.Builder builder = extract(txnId, record, user_version); + CommandChanges builder = extract(txnId, record, user_version); StoreParticipants participants = builder.participants(); if (participants == null) return null; @@ -179,10 +179,10 @@ public static ByteBuffer extractParticipants(RouteJournalIndex index, TxnId txnI } } - public static AccordJournal.Builder extract(TxnId txnId, ByteBuffer record, Version userVersion) + public static CommandChanges extract(TxnId txnId, ByteBuffer record, Version userVersion) { - AccordJournal.Builder builder = new AccordJournal.Builder(txnId, AccordJournal.Load.ALL); - AccordJournalTable.readBuffer(record, builder::deserializeNext, userVersion); + CommandChanges builder = new CommandChanges(txnId, AccordJournal.Load.ALL); + builder.deserializeNext(record, userVersion); return builder; } diff --git a/src/java/org/apache/cassandra/index/accord/RouteJournalIndex.java b/src/java/org/apache/cassandra/index/accord/RouteJournalIndex.java index eb20373a7393..b00202ce0220 100644 --- a/src/java/org/apache/cassandra/index/accord/RouteJournalIndex.java +++ b/src/java/org/apache/cassandra/index/accord/RouteJournalIndex.java @@ -89,10 +89,10 @@ import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.ClientState; -import org.apache.cassandra.service.accord.AccordJournalTable; import org.apache.cassandra.service.accord.AccordKeyspace; import org.apache.cassandra.service.accord.JournalKey; import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.service.accord.journal.RangeSearchManager; import org.apache.cassandra.service.accord.serializers.CommandSerializers; import org.apache.cassandra.utils.AbstractIterator; import org.apache.cassandra.utils.concurrent.Future; @@ -394,7 +394,7 @@ public Searcher searcherFor(ReadCommand command) Timestamp maxTxnId = TxnId.MAX; for (RowFilter.Expression e : expressions) { - if (e.column() == AccordJournalTable.SyntheticColumn.participants.metadata) + if (e.column() == RangeSearchManager.SyntheticColumn.participants.metadata) { switch (e.operator()) { @@ -410,11 +410,11 @@ public Searcher searcherFor(ReadCommand command) return null; } } - else if (e.column() == AccordJournalTable.SyntheticColumn.store_id.metadata && e.operator() == Operator.EQ) + else if (e.column() == RangeSearchManager.SyntheticColumn.store_id.metadata && e.operator() == Operator.EQ) { storeId = Int32Type.instance.compose(e.getIndexValue()); } - else if (e.column() == AccordJournalTable.SyntheticColumn.txn_id.metadata) + else if (e.column() == RangeSearchManager.SyntheticColumn.txn_id.metadata) { switch (e.operator()) { diff --git a/src/java/org/apache/cassandra/index/sai/plan/QueryViewBuilder.java b/src/java/org/apache/cassandra/index/sai/plan/QueryViewBuilder.java index 0d2aeb3838fb..7987c59f9101 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/QueryViewBuilder.java +++ b/src/java/org/apache/cassandra/index/sai/plan/QueryViewBuilder.java @@ -92,6 +92,7 @@ public QueryView(Collection view, Set referen public void close() { referencedIndexes.forEach(SSTableIndex::releaseQuietly); + referencedIndexes.clear(); } } diff --git a/src/java/org/apache/cassandra/io/util/CompressedFrameDataInputPlus.java b/src/java/org/apache/cassandra/io/util/CompressedFrameDataInputPlus.java new file mode 100644 index 000000000000..7d6be50d4991 --- /dev/null +++ b/src/java/org/apache/cassandra/io/util/CompressedFrameDataInputPlus.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.io.util; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.util.List; +import java.util.zip.Checksum; + +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.UnversionedSerializer; +import org.apache.cassandra.io.compress.ICompressor; +import org.apache.cassandra.io.compress.ZstdCompressor; +import org.apache.cassandra.utils.CollectionSerializers; +import org.apache.cassandra.utils.Crc; + +import static org.apache.cassandra.io.compress.ZstdCompressor.DEFAULT_COMPRESSION_LEVEL; +import static org.apache.cassandra.io.util.CompressedFrameDataOutputPlus.DEFAULT_FRAME_SIZE; +import static org.apache.cassandra.io.util.CompressedFrameDataOutputPlus.SIZE_OF_HEADER; + +public class CompressedFrameDataInputPlus extends RebufferingInputStream +{ + final ICompressor compressor; + final Checksum checksum; + final ReadableByteChannel channel; + ByteBuffer compressed; + + protected CompressedFrameDataInputPlus(int frameSize, ReadableByteChannel channel, ICompressor compressor, Checksum checksum) + { + super(compressor.preferredBufferType().allocate(frameSize)); + this.compressor = compressor; + this.checksum = checksum; + this.channel = channel; + this.compressed = compressor.preferredBufferType().allocate(frameSize); + buffer.limit(0); + } + + @Override + protected void reBuffer() throws IOException + { + compressed.position(0); + compressed.limit(SIZE_OF_HEADER); + while (channel.read(compressed) >= 0 && compressed.hasRemaining()); + compressed.flip(); + long headerChecksum = compressed.getLong(); + int length = compressed.getShort(); + + boolean decompress = length >= 0; + if (!decompress) + length = -1 - length; + + compressed.clear(); + compressed.limit(length); + while (compressed.hasRemaining()) + { + if (channel.read(compressed) < 0) + throw new EOFException("Incomplete file: header stipulated " + length + " bytes but found only " + compressed.position()); + } + compressed.flip(); + this.checksum.update(compressed); + compressed.flip(); + long dataChecksum = checksum.getValue(); + if (headerChecksum != dataChecksum) + throw new IOException("Invalid checksum: " + headerChecksum + " != " + dataChecksum); + + buffer.clear(); + if (decompress) compressor.uncompress(compressed, buffer); + else buffer.put(compressed); + buffer.flip(); + } + + public static T read(File file, IVersionedSerializer serializer) throws IOException + { + try (CompressedFrameDataInputPlus in = new CompressedFrameDataInputPlus(DEFAULT_FRAME_SIZE, file.newReadChannel(), ZstdCompressor.getOrCreate(DEFAULT_COMPRESSION_LEVEL), Crc.crc32())) + { + int version = in.readUnsignedVInt32(); + return serializer.deserialize(in, version); + } + } + + public static T readOne(File file, UnversionedSerializer serializer) throws IOException + { + try (CompressedFrameDataInputPlus in = new CompressedFrameDataInputPlus(DEFAULT_FRAME_SIZE, file.newReadChannel(), ZstdCompressor.getOrCreate(DEFAULT_COMPRESSION_LEVEL), Crc.crc32())) + { + int version = in.readUnsignedVInt32(); + if (version != 0) + throw new IOException("Expected version 0 for unversioned serializer"); + return serializer.deserialize(in); + } + } + + public static List readList(File file, UnversionedSerializer serializer) throws IOException + { + try (CompressedFrameDataInputPlus in = new CompressedFrameDataInputPlus(DEFAULT_FRAME_SIZE, file.newReadChannel(), ZstdCompressor.getOrCreate(DEFAULT_COMPRESSION_LEVEL), Crc.crc32())) + { + int version = in.readUnsignedVInt32(); + if (version != 0) + throw new IOException("Expected version 0 for unversioned serializer"); + return CollectionSerializers.deserializeList(in, serializer); + } + } +} diff --git a/src/java/org/apache/cassandra/io/util/CompressedFrameDataOutputPlus.java b/src/java/org/apache/cassandra/io/util/CompressedFrameDataOutputPlus.java new file mode 100644 index 000000000000..eab8b7d3a626 --- /dev/null +++ b/src/java/org/apache/cassandra/io/util/CompressedFrameDataOutputPlus.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.io.util; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.List; +import java.util.Objects; +import java.util.zip.Checksum; + +import com.google.common.primitives.Shorts; + +import accord.utils.Invariants; + +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.UnversionedSerializer; +import org.apache.cassandra.io.compress.ICompressor; +import org.apache.cassandra.io.compress.ZstdCompressor; +import org.apache.cassandra.utils.CollectionSerializers; +import org.apache.cassandra.utils.Crc; +import org.apache.cassandra.utils.memory.MemoryUtil; + +import static org.apache.cassandra.io.compress.ZstdCompressor.DEFAULT_COMPRESSION_LEVEL; + +public class CompressedFrameDataOutputPlus extends BufferedDataOutputStreamPlus +{ + static final int SIZE_OF_HEADER = 10; + static final int DEFAULT_FRAME_SIZE = 16 << 10; + + private final ICompressor compressor; + private final Checksum checksum; + private ByteBuffer compressed; + protected CompressedFrameDataOutputPlus(int frameSize, WritableByteChannel out, ICompressor compressor, Checksum checksum) + { + super(out, compressor.preferredBufferType().allocate(frameSize)); + this.compressor = compressor; + this.compressed = compressor.preferredBufferType().allocate(frameSize + SIZE_OF_HEADER); + this.checksum = checksum; + if (frameSize > Short.MAX_VALUE) + throw new IllegalArgumentException("Frame size too large"); + } + + @Override + protected void doFlush(int count) throws IOException + { + buffer.flip(); + compressed.clear(); + compressed.position(SIZE_OF_HEADER); + compressor.compress(buffer, compressed); + compressed.flip(); + int limit = compressed.limit(); + int length = limit - SIZE_OF_HEADER; + if (length > buffer.limit()) + { + length = -(1 + buffer.limit()); + compressed.clear(); + compressed.position(SIZE_OF_HEADER); + buffer.position(0); + compressed.put(buffer); + compressed.flip(); + } + compressed.putShort(SIZE_OF_HEADER - 2, Shorts.checkedCast(length)); + compressed.position(SIZE_OF_HEADER); + checksum.update(compressed); + compressed.putLong(0, checksum.getValue()); + compressed.position(0); + while (compressed.hasRemaining()) + channel.write(compressed); + buffer.clear(); + } + + @Override + public void close() throws IOException + { + super.close(); + MemoryUtil.clean(compressed); + compressed = null; + } + + public static void write(File file, T value, IVersionedSerializer serializer, int version) throws IOException + { + try (CompressedFrameDataOutputPlus out = new CompressedFrameDataOutputPlus(DEFAULT_FRAME_SIZE, file.newReadWriteChannel(), ZstdCompressor.getOrCreate(DEFAULT_COMPRESSION_LEVEL), Crc.crc32())) + { + out.writeUnsignedVInt32(version); + serializer.serialize(value, out, version); + } + } + + public static void writeOne(File file, T value, UnversionedSerializer serializer) throws IOException + { + try (CompressedFrameDataOutputPlus out = new CompressedFrameDataOutputPlus(DEFAULT_FRAME_SIZE, file.newReadWriteChannel(), ZstdCompressor.getOrCreate(DEFAULT_COMPRESSION_LEVEL), Crc.crc32())) + { + out.writeUnsignedVInt32(0); + serializer.serialize(value, out); + } + if (Invariants.isParanoid()) + { + try + { + T reconcile = CompressedFrameDataInputPlus.readOne(file, serializer); + Invariants.require(Objects.equals(value, reconcile)); + } + catch (IOException e) {} + } + } + + public static void writeList(File file, List value, UnversionedSerializer serializer) throws IOException + { + try (CompressedFrameDataOutputPlus out = new CompressedFrameDataOutputPlus(DEFAULT_FRAME_SIZE, file.newReadWriteChannel(), ZstdCompressor.getOrCreate(DEFAULT_COMPRESSION_LEVEL), Crc.crc32())) + { + out.writeUnsignedVInt32(0); + CollectionSerializers.serializeList(value, out, serializer); + } + if (Invariants.isParanoid()) + { + try + { + List reconcile = CompressedFrameDataInputPlus.readList(file, serializer); + Invariants.require(Objects.equals(value, reconcile)); + } + catch (IOException e) {} + } + } +} diff --git a/src/java/org/apache/cassandra/journal/Compactor.java b/src/java/org/apache/cassandra/journal/Compactor.java index b6749bc39072..ffc48ff5f2e1 100644 --- a/src/java/org/apache/cassandra/journal/Compactor.java +++ b/src/java/org/apache/cassandra/journal/Compactor.java @@ -116,13 +116,17 @@ public boolean isTerminated() public void shutdown() { logger.debug("Shutting down " + executor); + if (scheduled != null) + scheduled.cancel(false); executor.shutdown(); } @Override public Object shutdownNow() { - return executor.shutdownNow(); + // if we call executor.shutdownNow() we can cause ClosedByInterruptException + shutdown(); + return null; } @Override diff --git a/src/java/org/apache/cassandra/journal/DumpUtil.java b/src/java/org/apache/cassandra/journal/DumpUtil.java index d98e758384ab..1a5491304358 100644 --- a/src/java/org/apache/cassandra/journal/DumpUtil.java +++ b/src/java/org/apache/cassandra/journal/DumpUtil.java @@ -20,6 +20,8 @@ import java.util.function.Consumer; +import static org.apache.cassandra.journal.Params.RecoverableCrcFailurePolicy.FAIL; + /** * Helper file to avoid exposing components outside their package-local visibility scope */ @@ -37,6 +39,6 @@ public static void dumpMetadata(Descriptor descriptor, Consumer out) public static StaticSegment open(Descriptor descriptor, KeySupport keySupport) { - return StaticSegment.open(descriptor, keySupport); + return StaticSegment.open(descriptor, keySupport, FAIL); } } diff --git a/src/java/org/apache/cassandra/journal/EntrySerializer.java b/src/java/org/apache/cassandra/journal/EntrySerializer.java index 70d1e934d49b..b2d138107e50 100644 --- a/src/java/org/apache/cassandra/journal/EntrySerializer.java +++ b/src/java/org/apache/cassandra/journal/EntrySerializer.java @@ -28,8 +28,6 @@ import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.Crc; -import static org.apache.cassandra.journal.Journal.validateCRC; - /** * Entry format: * @@ -73,10 +71,10 @@ static void write(K key, out.limit(headerCrcPosition); crc.update(out); out.limit(recordEnd); - out.putInt((int) crc.getValue()); + out.putInt(nonZeroCrc(crc)); crc.update(out); out.limit(recordEnd + 4); - out.putInt((int) crc.getValue()); + out.putInt(nonZeroCrc(crc)); } // we reuse record as the value we return @@ -173,11 +171,15 @@ static int tryRead(EntryHolder into, public static class MaybeRecoverableJournalError extends IOException { public final int knownLength; + public final int readCrc; + public final int computedCrc; - public MaybeRecoverableJournalError(int knownLength, Throwable cause) + public MaybeRecoverableJournalError(int knownLength, Crc.InvalidCrc cause) { super(cause); this.knownLength = knownLength; + this.readCrc = cause.read; + this.computedCrc = cause.computed; } @Override @@ -244,4 +246,16 @@ public void clear() value = null; } } + + static int nonZeroCrc(CRC32 crc) + { + int v = (int) crc.getValue(); + return v == 0 ? 1 : v; + } + + static void validateCRC(CRC32 crc, int readCRC) throws Crc.InvalidCrc + { + if (readCRC != nonZeroCrc(crc)) + throw new Crc.InvalidCrc(readCRC, nonZeroCrc(crc)); + } } diff --git a/src/java/org/apache/cassandra/journal/Flusher.java b/src/java/org/apache/cassandra/journal/Flusher.java index 2db7b0f6d09c..dddc29346fb4 100644 --- a/src/java/org/apache/cassandra/journal/Flusher.java +++ b/src/java/org/apache/cassandra/journal/Flusher.java @@ -17,6 +17,9 @@ */ package org.apache.cassandra.journal; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.LockSupport; @@ -102,19 +105,24 @@ void start() flushExecutor = executorFactory().infiniteLoop(flushExecutorName, new FlushRunnable(), SAFE, NON_DAEMON, SYNCHRONIZED); } - void shutdown() throws InterruptedException + void shutdown() { - logger.debug("Shutting down " + flushExecutor + " and awaiting termination"); + logger.debug("Shutting down " + flushExecutor); flushExecutor.shutdown(); - flushExecutor.awaitTermination(1, MINUTES); if (fsyncExecutor != null) { - logger.debug("Shutting down " + fsyncExecutor + " and awaiting termination"); - fsyncExecutor.shutdownNow(); // `now` to interrupt potentially parked runnable - fsyncExecutor.awaitTermination(1, MINUTES); + logger.debug("Shutting down " + fsyncExecutor); + fsyncExecutor.shutdown(); // `now` to interrupt potentially parked runnable } } + List executors() + { + if (fsyncExecutor != null) + return Arrays.asList(flushExecutor, fsyncExecutor); + return Collections.singletonList(flushExecutor); + } + @Simulate(with={MONITORS,GLOBAL_CLOCK,LOCK_SUPPORT}) // waits for writes to complete before triggering an fsync private class FlushRunnable implements Interruptible.Task @@ -140,6 +148,10 @@ public void run(Interruptible.State state) throws InterruptedException { doRun(state); } + catch (InterruptedException t) + { + throw t; + } catch (Throwable t) { if (!journal.handleError("Failed to flush segments to disk", t)) @@ -189,8 +201,8 @@ void notify(Thread notify) public void doRun(Interruptible.State state) throws InterruptedException { - if (state == NORMAL) awaitWork(); - else if (!hasWork()) return; + if (state == NORMAL) + awaitWork(); if (fsyncing == null) fsyncing = journal.oldestActiveSegment(); @@ -260,6 +272,10 @@ public void run(Interruptible.State state) throws InterruptedException { doRun(state); } + catch (InterruptedException t) + { + throw t; + } catch (Throwable t) { if (!journal.handleError("Failed to flush segments to disk", t)) @@ -534,8 +550,8 @@ private void awaitFsyncAt(long flushTime, Timer.Context context) { signal.awaitThrowUncheckedOnInterrupt(); - Journal.State state = journal.state.get(); - Invariants.require(state == Journal.State.NORMAL, + Journal.State state = journal.getState(); + Invariants.require(state.compareTo(Journal.State.STOPPED_READABLE) < 0, "Thread %s outlived journal, which is in %s state", Thread.currentThread(), state); } else @@ -544,6 +560,24 @@ private void awaitFsyncAt(long flushTime, Timer.Context context) while (fsyncFinishedFor < flushTime); } + void awaitFsync(ActiveSegment segment, int fsyncedTo) + { + while (true) + { + Journal.State state = journal.getState(); + Invariants.require(state.compareTo(Journal.State.STOPPED_READABLE) < 0, + "Thread %s outlived journal, which is in %s state", Thread.currentThread(), state); + + WaitQueue.Signal signal = fsyncComplete.register(); + if (segment.fsyncedTo() >= fsyncedTo) + { + signal.cancel(); + break; + } + signal.awaitThrowUncheckedOnInterrupt(); + } + } + private long flushPeriodNanos() { return params.flushPeriod(NANOSECONDS); diff --git a/src/java/org/apache/cassandra/journal/InMemoryIndex.java b/src/java/org/apache/cassandra/journal/InMemoryIndex.java index 974767590a37..ff8399b12a51 100644 --- a/src/java/org/apache/cassandra/journal/InMemoryIndex.java +++ b/src/java/org/apache/cassandra/journal/InMemoryIndex.java @@ -28,6 +28,7 @@ import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileOutputStreamPlus; +import org.apache.cassandra.journal.Params.RecoverableCrcFailurePolicy; import org.apache.cassandra.journal.StaticSegment.SequentialReader; /** @@ -154,11 +155,11 @@ public void persist(Descriptor descriptor) tmpFile.move(descriptor.fileFor(Component.INDEX)); } - static InMemoryIndex rebuild(Descriptor descriptor, KeySupport keySupport, int fsyncedLimit) + static InMemoryIndex rebuild(Descriptor descriptor, KeySupport keySupport, int fsyncedLimit, RecoverableCrcFailurePolicy crcFailurePolicy) { InMemoryIndex index = new InMemoryIndex<>(keySupport, new TreeMap<>(keySupport)); - try (SequentialReader reader = StaticSegment.sequentialReader(descriptor, keySupport, fsyncedLimit)) + try (SequentialReader reader = StaticSegment.sequentialReader(descriptor, keySupport, fsyncedLimit, crcFailurePolicy)) { while (reader.advance()) index.update(reader.key(), reader.offset, reader.buffer.position() - reader.offset); diff --git a/src/java/org/apache/cassandra/journal/Journal.java b/src/java/org/apache/cassandra/journal/Journal.java index 085848588c7a..06c060c365f1 100644 --- a/src/java/org/apache/cassandra/journal/Journal.java +++ b/src/java/org/apache/cassandra/journal/Journal.java @@ -27,10 +27,11 @@ import java.util.Iterator; import java.util.List; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.concurrent.locks.LockSupport; -import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Predicate; @@ -48,7 +49,6 @@ import org.apache.cassandra.concurrent.Interruptible; import org.apache.cassandra.concurrent.Interruptible.TerminateException; import org.apache.cassandra.concurrent.SequentialExecutorPlus; -import org.apache.cassandra.concurrent.Shutdownable; import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.io.util.DataOutputPlus; @@ -59,6 +59,7 @@ import org.apache.cassandra.utils.AbstractIterator; import org.apache.cassandra.utils.CloseableIterator; import org.apache.cassandra.utils.Crc; +import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.MergeIterator; import org.apache.cassandra.utils.Simulate; @@ -71,7 +72,7 @@ import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Interrupts.SYNCHRONIZED; import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.SimulatorSafe.SAFE; import static org.apache.cassandra.concurrent.Interruptible.State.NORMAL; -import static org.apache.cassandra.concurrent.Interruptible.State.SHUTTING_DOWN; +import static org.apache.cassandra.journal.Params.RecoverableCrcFailurePolicy.FAIL; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; import static org.apache.cassandra.utils.Simulate.With.MONITORS; import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue; @@ -89,7 +90,7 @@ must be fixed-size and byte-order comparable */ @Simulate(with=MONITORS) -public class Journal implements Shutdownable +public class Journal { private static final Logger logger = LoggerFactory.getLogger(Journal.class); @@ -104,10 +105,10 @@ public class Journal implements Shutdownable final Flusher flusher; final Compactor compactor; + final AllocateRunnable allocateRunnable = new AllocateRunnable(); Interruptible allocator; SequentialExecutorPlus closer, releaser; - volatile long replayLimit; final AtomicLong nextSegmentId = new AtomicLong(); private volatile ActiveSegment currentSegment = null; @@ -117,12 +118,11 @@ public class Journal implements Shutdownable private final AtomicReference> segments = new AtomicReference<>(); - final AtomicReference state = new AtomicReference<>(State.UNINITIALIZED); + private volatile State state = State.UNINITIALIZED; + private static final AtomicReferenceFieldUpdater stateUpdater = AtomicReferenceFieldUpdater.newUpdater(Journal.class, State.class, "state"); - // TODO (expected): we do not need wait queues here, we can just wait on a signal on a segment while its byte buffer is being allocated private final WaitQueue segmentPrepared = newWaitQueue(); - private final WaitQueue allocatorThreadWaitQueue = newWaitQueue(); - private final BooleanSupplier allocatorThreadWaitCondition = () -> (availableSegment == null); + private volatile Thread waitingAllocatorThread; private final FlusherCallbacks flusherCallbacks; @@ -199,37 +199,53 @@ public Journal(String name, this.compactor = new Compactor<>(this, segmentCompactor); } + public long peekSegmentId() + { + return nextSegmentId.get(); + } + public void onDurable(RecordPointer recordPointer, Runnable runnable) { flusherCallbacks.submit(recordPointer, runnable); } - public void start() + public void open() { - Invariants.require(state.compareAndSet(State.UNINITIALIZED, State.INITIALIZING), - "Unexpected journal state during initialization", state); - metrics.register(flusher); + Invariants.require(stateUpdater.compareAndSet(this, State.UNINITIALIZED, State.OPENING), + "Unexpected journal state before opening", state); deleteTmpFiles(); - List descriptors = Descriptor.list(directory); - // find the largest existing timestamp - descriptors.sort(null); - long maxTimestamp = descriptors.isEmpty() - ? Long.MIN_VALUE - : descriptors.get(descriptors.size() - 1).timestamp; - nextSegmentId.set(replayLimit = Math.max(currentTimeMillis(), maxTimestamp + 1)); - - segments.set(Segments.of(StaticSegment.open(descriptors, keySupport))); + segments.set(Segments.of(StaticSegment.open(descriptors, keySupport, params.crcFailureOnRebuildPolicy()))); + + Invariants.require(stateUpdater.compareAndSet(this, State.OPENING, State.OPEN_READABLE), + "Unexpected journal state once opened", state); + } + + public void start(long maxTableDescriptor) + { + if (state == State.UNINITIALIZED) + open(); + + Invariants.require(stateUpdater.compareAndSet(this, State.OPEN_READABLE, State.STARTING), + "Unexpected journal state before starting", state); + + nextSegmentId.set(Math.max(currentTimeMillis(), Math.max(maxDescriptor(), maxTableDescriptor) + 1)); + closer = executorFactory().sequential(name + "-closer"); releaser = executorFactory().sequential(name + "-releaser"); - allocator = executorFactory().infiniteLoop(name + "-allocator", new AllocateRunnable(), SAFE, NON_DAEMON, SYNCHRONIZED); + allocator = executorFactory().infiniteLoop(name + "-allocator", allocateRunnable, SAFE, NON_DAEMON, SYNCHRONIZED); + + // we use these metrics when advancing segments, so must register first + metrics.register(flusher); advanceSegment(null); - Invariants.require(state.compareAndSet(State.INITIALIZING, State.NORMAL), - "Unexpected journal state after initialization", state); + flusher.start(); compactor.start(); + Invariants.require(stateUpdater.compareAndSet(this, State.STARTING, State.WRITEABLE), + "Unexpected journal state once started", state); + final int maxSegments = 100; if (segments.get().count(Segment::isStatic) > maxSegments) { @@ -252,6 +268,29 @@ public void start() } } + public long maxDescriptor() + { + List> existingSegments = segments.get().allSorted(false); + return existingSegments.isEmpty() ? 0 : existingSegments.get(0).descriptor.timestamp; + } + + public State getState() + { + return state; + } + + public boolean isReadable() + { + State state = this.state; + return state.compareTo(State.OPEN_READABLE) >= 0 && state.compareTo(State.STOPPED_READABLE) <= 0; + } + + private boolean isNotStopped() + { + State state = this.state; + return state.compareTo(State.STARTING) >= 0 && state.compareTo(State.STOPPING) <= 0; + } + @VisibleForTesting public void runCompactorForTesting() { @@ -272,58 +311,72 @@ private void deleteTmpFiles() tmpFile.delete(); } - @Override + public boolean hasBeenOpened() + { + return state.compareTo(State.OPEN_READABLE) >= 0; + } + public boolean isTerminated() { - return state.get() == State.TERMINATED; + return state == State.STOPPED_READABLE; } - public void shutdown() + public void fsync() { - try - { - Invariants.require(state.compareAndSet(State.NORMAL, State.SHUTDOWN), - "Unexpected journal state while trying to shut down", state); - logger.debug("Shutting down " + allocator + " and awaiting termination"); - allocator.shutdown(); - wakeAllocator(); // Wake allocator to force it into shutdown - // TODO (expected): why are we awaitingTermination here when we have a separate method for it? - allocator.awaitTermination(1, TimeUnit.MINUTES); - segmentPrepared.signalAll(); // Wake up all threads waiting on the new segment - compactor.shutdown(); - compactor.awaitTermination(1, TimeUnit.MINUTES); - flusher.shutdown(); - closeAllSegments(); - logger.debug("Shutting down " + releaser + " and " + closer + " and awaiting termination"); - releaser.shutdown(); - closer.shutdown(); - closer.awaitTermination(1, TimeUnit.MINUTES); - releaser.awaitTermination(1, TimeUnit.MINUTES); - metrics.deregister(); - Invariants.require(state.compareAndSet(State.SHUTDOWN, State.TERMINATED), - "Unexpected journal state while trying to shut down", state); - } - catch (InterruptedException e) + ActiveSegment active = currentSegment; + int position = active.writtenToAtLeast(); + flusher.requestExtraFlush(); + flusher.awaitFsync(active, position); + } + + // return the last segment that was written to + public Descriptor stop() + { + logger.info("Stopping journal"); + logger.debug("Shutting down " + allocator); + boolean stop; + synchronized (allocateRunnable) { - logger.error("Could not shutdown journal", e); + // we synchronize on allocateRunnable to ensure it witnesses this change before the next attempt to allocate a segment + stop = stateUpdater.compareAndSet(this, State.WRITEABLE, State.STOPPING); } + Invariants.require(stop, "Unexpected journal state before stopping", state); + + // ensure prompt shutdown, though the above state change suffices semantically + allocator.shutdown(); + wakeAllocator(); + discardAvailableSegment(); + segmentPrepared.signalAll(); // Wake up all threads waiting on the new segment + + compactor.shutdown(); + + currentSegment.discardUnusedTail(); + flusher.requestExtraFlush(); + + Descriptor lastSegment = finaliseSegments(); // this flushes any pending writes + + flusher.shutdown(); + logger.debug("Shutting down " + releaser + " and " + closer); + releaser.shutdown(); + closer.shutdown(); + metrics.deregister(); + Invariants.require(stateUpdater.compareAndSet(this, State.STOPPING, State.STOPPED_READABLE), + "Unexpected journal state after stopping", state); + return lastSegment; } - @Override - public Object shutdownNow() + public void close() { - shutdown(); - return null; + logger.info("Closing journal"); + stateUpdater.compareAndSet(this, State.STOPPED_READABLE, State.CLOSING); + closeAllSegments(); + stateUpdater.compareAndSet(this, State.CLOSING, State.CLOSED); } - @Override - public boolean awaitTermination(long timeout, TimeUnit units) throws InterruptedException + public void awaitTerminationUntil(long deadlineNanos) throws InterruptedException, TimeoutException { - boolean r = true; - r &= allocator.awaitTermination(timeout, units); - r &= closer.awaitTermination(timeout, units); - r &= releaser.awaitTermination(timeout, units); - return r; + ExecutorUtils.awaitTerminationUntil(deadlineNanos, Arrays.asList(allocator, compactor, closer, releaser)); + ExecutorUtils.awaitTerminationUntil(deadlineNanos, flusher.executors()); } /** @@ -470,9 +523,6 @@ public static boolean readLast(K id, RecordConsumer consumer, OpOrder. { for (Segment segment : segments.allSorted(false)) { - if (!segment.index().mayContainId(id)) - continue; - if (segment.readLast(id, consumer)) return true; } @@ -542,7 +592,6 @@ public void unsafeConsumeBytesForTesting(int entrySize, Consumer cor private ActiveSegment.Allocation allocate(int entrySize) { - ActiveSegment segment = currentSegment; ActiveSegment.Allocation alloc; while (null == (alloc = segment.allocate(entrySize))) @@ -598,12 +647,12 @@ private void awaitAvailableSegment(ActiveSegment currentActiveSegment) WaitQueue.Signal prepared = segmentPrepared.register(metrics.waitingOnSegmentAllocation.time(), Context::stop); if (availableSegment == null && currentSegment == currentActiveSegment) { - prepared.awaitThrowUncheckedOnInterrupt(); - // In case we woke up due to shutdown signal or interrupt, check mode - State state = this.state.get(); - if (state.ordinal() > State.NORMAL.ordinal()) + State state = this.state; + if (state.ordinal() > State.WRITEABLE.ordinal()) throw new IllegalStateException("Can not obtain allocated segment due to shutdown " + state); + + prepared.awaitThrowUncheckedOnInterrupt(); } else prepared.cancel(); @@ -613,7 +662,9 @@ private void awaitAvailableSegment(ActiveSegment currentActiveSegment) private void wakeAllocator() { - allocatorThreadWaitQueue.signalAll(); + Thread wake = waitingAllocatorThread; + if (wake != null) + LockSupport.unpark(wake); } private void discardAvailableSegment() @@ -635,13 +686,10 @@ public void run(Interruptible.State state) throws InterruptedException { if (state == NORMAL) runNormal(); - else if (state == SHUTTING_DOWN) - shutDown(); } private void runNormal() throws InterruptedException { - boolean interrupted = false; try { if (availableSegment != null) @@ -649,14 +697,19 @@ private void runNormal() throws InterruptedException // synchronized to prevent thread interrupts while performing IO operations and also // clear interrupted status to prevent ClosedByInterruptException in createSegment() + boolean interrupted; synchronized (this) { + if (state.compareTo(State.STOPPING) >= 0) + throw new TerminateException(); + interrupted = Thread.interrupted(); availableSegment = createSegment(); - - segmentPrepared.signalAll(); - Thread.yield(); } + + segmentPrepared.signalAll(); + if (interrupted) throw new InterruptedException(); + else Thread.yield(); } catch (JournalWriteError e) { @@ -673,41 +726,18 @@ private void runNormal() throws InterruptedException TimeUnit.SECONDS.sleep(1L); // sleep for a second to avoid log spam } - interrupted = interrupted || Thread.interrupted(); - if (!interrupted) - { - try - { - // If we offered a segment, wait for it to be taken before reentering the loop. - // There could be a new segment in next not offered, but only on failure to discard it while - // shutting down-- nothing more can or needs to be done in that case. - WaitQueue.waitOnCondition(allocatorThreadWaitCondition, allocatorThreadWaitQueue); - } - catch (InterruptedException e) - { - interrupted = true; - } - } - - if (interrupted) + // If we offered a segment, wait for it to be taken before reentering the loop. + // There could be a new segment in next not offered, but only on failure to discard it while + // shutting down-- nothing more can or needs to be done in that case. + if (availableSegment != null) { - discardAvailableSegment(); - throw new InterruptedException(); - } - } - - private void shutDown() throws InterruptedException - { - try - { - // if shutdown() started and finished during segment creation, we'll be left with a - // segment that no one will consume; discard it - discardAvailableSegment(); - } - catch (Throwable t) - { - handleError("Failed shutting down segment allocator", t); - throw new TerminateException(); + waitingAllocatorThread = Thread.currentThread(); + boolean interrupted = false; + while (availableSegment != null && !(interrupted = Thread.interrupted())) + LockSupport.park(); + waitingAllocatorThread = null; + if (interrupted) + throw new InterruptedException(); } } } @@ -722,7 +752,8 @@ private void closeAllSegments() { Segments segments = swapSegments(ignore -> Segments.none()); - for (Segment segment : segments.all()) + List> all = segments.allSorted(false); + for (Segment segment : all) { if (segment.isActive()) ((ActiveSegment) segment).closeAndIfEmptyDiscard(this); @@ -731,6 +762,24 @@ private void closeAllSegments() } } + private Descriptor finaliseSegments() + { + while (true) + { + ActiveSegment oldestActive = oldestActiveSegment(); + oldestActive.discardUnusedTail(); + flusher.awaitFsync(oldestActive, oldestActive.writtenToAtLeast()); + if (oldestActive == currentSegment) + break; + } + + currentSegment.persistComponents(); + List> all = segments().allSorted(false); + if (all.isEmpty()) + return null; + return all.get(0).descriptor; + } + @SuppressWarnings("unused") ReferencedSegments selectAndReference(Predicate> selector) { @@ -761,29 +810,28 @@ private Segments swapSegments(Function, Segments> tra private void addNewActiveSegment(ActiveSegment activeSegment) { + Invariants.require(isNotStopped()); swapSegments(current -> current.withNewActiveSegment(activeSegment)); } private void removeEmptySegment(ActiveSegment activeSegment) { + Invariants.require(isNotStopped()); swapSegments(current -> current.withoutEmptySegment(activeSegment)); } private void replaceCompletedSegment(ActiveSegment activeSegment, StaticSegment staticSegment) { + Invariants.require(isNotStopped()); swapSegments(current -> current.withCompletedSegment(activeSegment, staticSegment)); } void replaceCompactedSegments(Collection> oldSegments, Collection> compactedSegments) { + Invariants.require(isNotStopped()); swapSegments(current -> current.withCompactedSegments(oldSegments, compactedSegments)); } - void selectSegmentToFlush(Collection> into) - { - segments().selectActive(currentSegment.descriptor.timestamp, into); - } - ActiveSegment oldestActiveSegment() { ActiveSegment current = currentSegment; @@ -856,7 +904,7 @@ public void run() activeSegment.updateWrittenTo(); activeSegment.fsync(); activeSegment.persistComponents(); - replaceCompletedSegment(activeSegment, StaticSegment.open(activeSegment.descriptor, keySupport)); + replaceCompletedSegment(activeSegment, StaticSegment.open(activeSegment.descriptor, keySupport, FAIL)); activeSegment.release(Journal.this); } } @@ -892,8 +940,8 @@ public void closeCurrentSegmentForTestingIfNonEmpty() static void validateCRC(CRC32 crc, int readCRC) throws Crc.InvalidCrc { - if (readCRC != (int) crc.getValue()) - throw new Crc.InvalidCrc(readCRC, (int) crc.getValue()); + if (readCRC != (int)crc.getValue()) + throw new Crc.InvalidCrc(readCRC, (int)crc.getValue()); } /* @@ -950,6 +998,7 @@ private String maybeAddDiskSpaceContext(String message) @VisibleForTesting public void truncateForTesting() { + Invariants.require(isNotStopped()); ActiveSegment discarding = currentSegment; if (!discarding.isEmpty()) // if there is no data in the segement then ignore it { @@ -1158,12 +1207,16 @@ class Head } } - enum State + public enum State { UNINITIALIZED, - INITIALIZING, - NORMAL, - SHUTDOWN, - TERMINATED + OPENING, + OPEN_READABLE, + STARTING, + WRITEABLE, + STOPPING, + STOPPED_READABLE, + CLOSING, + CLOSED } } diff --git a/src/java/org/apache/cassandra/journal/Metadata.java b/src/java/org/apache/cassandra/journal/Metadata.java index 8b1c55a3c354..30db3bbe84f7 100644 --- a/src/java/org/apache/cassandra/journal/Metadata.java +++ b/src/java/org/apache/cassandra/journal/Metadata.java @@ -27,6 +27,7 @@ import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileInputStreamPlus; import org.apache.cassandra.io.util.FileOutputStreamPlus; +import org.apache.cassandra.journal.Params.RecoverableCrcFailurePolicy; import org.apache.cassandra.utils.Crc; import static org.apache.cassandra.journal.Journal.validateCRC; @@ -132,11 +133,11 @@ static Metadata load(Descriptor descriptor) } } - static Metadata rebuild(Descriptor descriptor, KeySupport keySupport) + static Metadata rebuild(Descriptor descriptor, KeySupport keySupport, RecoverableCrcFailurePolicy crcFailurePolicy) { int recordsCount = 0; int fsyncLimit = 0; - try (StaticSegment.SequentialReader reader = StaticSegment.sequentialReader(descriptor, keySupport, Integer.MAX_VALUE)) + try (StaticSegment.SequentialReader reader = StaticSegment.sequentialReader(descriptor, keySupport, Integer.MAX_VALUE, crcFailurePolicy)) { while (reader.advance()) ++recordsCount; @@ -152,9 +153,9 @@ static Metadata rebuild(Descriptor descriptor, KeySupport keySupport) return new Metadata(recordsCount, fsyncLimit); } - static Metadata rebuildAndPersist(Descriptor descriptor, KeySupport keySupport) + static Metadata rebuildAndPersist(Descriptor descriptor, KeySupport keySupport, RecoverableCrcFailurePolicy crcFailurePolicy) { - Metadata metadata = rebuild(descriptor, keySupport); + Metadata metadata = rebuild(descriptor, keySupport, crcFailurePolicy); metadata.persist(descriptor); return metadata; } diff --git a/src/java/org/apache/cassandra/journal/OnDiskIndex.java b/src/java/org/apache/cassandra/journal/OnDiskIndex.java index 2cf74a2e1185..4c72aa4ad77c 100644 --- a/src/java/org/apache/cassandra/journal/OnDiskIndex.java +++ b/src/java/org/apache/cassandra/journal/OnDiskIndex.java @@ -33,6 +33,7 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.journal.Params.RecoverableCrcFailurePolicy; import org.apache.cassandra.utils.AbstractIterator; import org.apache.cassandra.utils.Crc; import org.apache.cassandra.utils.memory.MemoryUtil; @@ -370,9 +371,9 @@ private int compareWithKeyAt(K key, int keyIndex) return keySupport.compareWithKeyAt(key, buffer, offset, descriptor.userVersion); } - static OnDiskIndex rebuildAndPersist(Descriptor descriptor, KeySupport keySupport, int fsyncedLimit) + static OnDiskIndex rebuildAndPersist(Descriptor descriptor, KeySupport keySupport, int fsyncedLimit, RecoverableCrcFailurePolicy crcFailurePolicy) { - try (InMemoryIndex index = InMemoryIndex.rebuild(descriptor, keySupport, fsyncedLimit)) + try (InMemoryIndex index = InMemoryIndex.rebuild(descriptor, keySupport, fsyncedLimit, crcFailurePolicy)) { index.persist(descriptor); } diff --git a/src/java/org/apache/cassandra/journal/Params.java b/src/java/org/apache/cassandra/journal/Params.java index 161165177d21..99cd3bf5f714 100644 --- a/src/java/org/apache/cassandra/journal/Params.java +++ b/src/java/org/apache/cassandra/journal/Params.java @@ -24,7 +24,14 @@ public interface Params enum FlushMode { BATCH, GROUP, PERIODIC } enum FailurePolicy { STOP, STOP_JOURNAL, IGNORE, ALLOW_UNSAFE_STARTUP, DIE } - enum ReplayMode { RESET, ALL, ONLY_NON_DURABLE } + + enum RecoverableCrcFailurePolicy + { + FAIL, + IGNORE_ALL_ZERO_RECORDS, + IGNORE_CRC_ZERO_RECORDS, + IGNORE + } /** * @return maximum segment size @@ -41,13 +48,13 @@ enum ReplayMode { RESET, ALL, ONLY_NON_DURABLE } */ FailurePolicy failurePolicy(); + RecoverableCrcFailurePolicy crcFailureOnRebuildPolicy(); + /** * @return journal flush (sync) mode */ FlushMode flushMode(); - ReplayMode replayMode(); - boolean enableCompaction(); long compactionPeriod(TimeUnit units); diff --git a/src/java/org/apache/cassandra/journal/Segment.java b/src/java/org/apache/cassandra/journal/Segment.java index 7335bb2ca28a..5d6d65f82f52 100644 --- a/src/java/org/apache/cassandra/journal/Segment.java +++ b/src/java/org/apache/cassandra/journal/Segment.java @@ -44,7 +44,8 @@ public final void run() public final void tidy() { - executor.execute(this); + if (executor != null) executor.execute(this); + else onUnreferenced(); } } diff --git a/src/java/org/apache/cassandra/journal/StaticSegment.java b/src/java/org/apache/cassandra/journal/StaticSegment.java index bc327b952e4a..14a87d19af7c 100644 --- a/src/java/org/apache/cassandra/journal/StaticSegment.java +++ b/src/java/org/apache/cassandra/journal/StaticSegment.java @@ -32,11 +32,14 @@ import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.journal.Params.RecoverableCrcFailurePolicy; import org.apache.cassandra.utils.Closeable; import org.apache.cassandra.utils.Throwables; import org.apache.cassandra.utils.concurrent.Ref; import org.apache.cassandra.utils.memory.MemoryUtil; +import static org.apache.cassandra.journal.Params.RecoverableCrcFailurePolicy.FAIL; + /** * An immutable data segment that is no longer written to. *

@@ -76,12 +79,12 @@ private StaticSegment(Descriptor descriptor, * @param descriptors descriptors of the segments to load * @return list of the loaded segments */ - static List> open(Collection descriptors, KeySupport keySupport) + static List> open(Collection descriptors, KeySupport keySupport, RecoverableCrcFailurePolicy crcFailurePolicy) { List> segments = new ArrayList<>(descriptors.size()); for (Descriptor descriptor : descriptors) { - StaticSegment segment = open(descriptor, keySupport); + StaticSegment segment = open(descriptor, keySupport, crcFailurePolicy); segments.add(segment); } @@ -95,7 +98,7 @@ static List> open(Collection descriptors, KeySu * @return the loaded segment */ @SuppressWarnings({ "resource", "RedundantSuppression" }) - static StaticSegment open(Descriptor descriptor, KeySupport keySupport) + static StaticSegment open(Descriptor descriptor, KeySupport keySupport, RecoverableCrcFailurePolicy crcFailurePolicy) { if (!Component.DATA.existsFor(descriptor)) throw new IllegalArgumentException("Data file for segment " + descriptor + " doesn't exist"); @@ -115,7 +118,7 @@ static StaticSegment open(Descriptor descriptor, KeySupport keyS } if (metadata == null) - metadata = Metadata.rebuildAndPersist(descriptor, keySupport); + metadata = Metadata.rebuildAndPersist(descriptor, keySupport, crcFailurePolicy); OnDiskIndex index = null; @@ -133,7 +136,7 @@ static StaticSegment open(Descriptor descriptor, KeySupport keyS } if (index == null) - index = OnDiskIndex.rebuildAndPersist(descriptor, keySupport, metadata.fsyncLimit()); + index = OnDiskIndex.rebuildAndPersist(descriptor, keySupport, metadata.fsyncLimit(), crcFailurePolicy); try { @@ -300,7 +303,7 @@ boolean read(int offset, int size, EntrySerializer.EntryHolder into) */ public void forEachRecord(RecordConsumer consumer) { - try (SequentialReader reader = sequentialReader(descriptor, keySupport, fsyncLimit)) + try (SequentialReader reader = sequentialReader(descriptor, keySupport, fsyncLimit, FAIL)) { while (reader.advance()) { @@ -389,9 +392,9 @@ protected boolean eof() } } - static SequentialReader sequentialReader(Descriptor descriptor, KeySupport keySupport, int fsyncedLimit) + static SequentialReader sequentialReader(Descriptor descriptor, KeySupport keySupport, int fsyncedLimit, RecoverableCrcFailurePolicy crcFailurePolicy) { - return new SequentialReader<>(descriptor, keySupport, fsyncedLimit); + return new SequentialReader<>(descriptor, keySupport, fsyncedLimit, crcFailurePolicy); } /** @@ -405,11 +408,13 @@ static SequentialReader sequentialReader(Descriptor descriptor, KeySuppor static final class SequentialReader extends Reader { private final int fsyncedLimit; // exclusive + private final RecoverableCrcFailurePolicy crcFailurePolicy; - SequentialReader(Descriptor descriptor, KeySupport keySupport, int fsyncedLimit) + SequentialReader(Descriptor descriptor, KeySupport keySupport, int fsyncedLimit, RecoverableCrcFailurePolicy crcFailurePolicy) { super(descriptor, keySupport); this.fsyncedLimit = fsyncedLimit; + this.crcFailurePolicy = crcFailurePolicy; if (fsyncedLimit < buffer.limit()) buffer.limit(fsyncedLimit); } @@ -426,35 +431,45 @@ public boolean advance() private boolean doAdvance() { - offset = buffer.position(); - try - { - int length = EntrySerializer.tryRead(holder, keySupport, buffer.duplicate(), fsyncedLimit, descriptor.userVersion); - if (length < 0) - return eof(); - buffer.position(offset + length); - } - catch (EntrySerializer.MaybeRecoverableJournalError e) - { - logger.warn("Caught a recoverable journal error, skipping bytes", e); - int sizeMarker = buffer.getInt(offset); - if (e.knownLength <= Integer.BYTES || sizeMarker != offset + e.knownLength) - throw new JournalReadError(descriptor, file, e.getCause()); - - if (!areAllBytesZero(buffer, offset + Integer.BYTES, e.knownLength - Integer.BYTES)) - throw new JournalReadError(descriptor, file, e.getCause()); - - buffer.position(offset + e.knownLength); - // Recur here, as we anticipate a corrupt or incompletely written entry to be a very rare case. - return doAdvance(); - } - catch (IOException e) + while (true) { - throw new JournalReadError(descriptor, file, e); + offset = buffer.position(); + try + { + int length = EntrySerializer.tryRead(holder, keySupport, buffer.duplicate(), fsyncedLimit, descriptor.userVersion); + if (length < 0) + return eof(); + buffer.position(offset + length); + state = State.ADVANCED; + return true; + } + catch (EntrySerializer.MaybeRecoverableJournalError e) + { + int sizeMarker = buffer.getInt(offset); + if (e.knownLength <= Integer.BYTES || sizeMarker != offset + e.knownLength) + throw new JournalReadError(descriptor, file, e.getCause()); + + switch (crcFailurePolicy) + { + case IGNORE: break; + case IGNORE_ALL_ZERO_RECORDS: + if (areAllBytesZero(buffer, offset + Integer.BYTES, e.knownLength - Integer.BYTES)) + break; + case IGNORE_CRC_ZERO_RECORDS: + if (e.readCrc == 0) + break; + case FAIL: + throw new JournalReadError(descriptor, file, e.getCause()); + } + + logger.warn("Caught a recoverable journal error, skipping bytes", e); + buffer.position(offset + e.knownLength); + } + catch (IOException e) + { + throw new JournalReadError(descriptor, file, e); + } } - - state = State.ADVANCED; - return true; } private void reset() diff --git a/src/java/org/apache/cassandra/journal/ValueSerializer.java b/src/java/org/apache/cassandra/journal/ValueSerializer.java index 69690d39b28a..6cbaa7eddf2e 100644 --- a/src/java/org/apache/cassandra/journal/ValueSerializer.java +++ b/src/java/org/apache/cassandra/journal/ValueSerializer.java @@ -31,4 +31,19 @@ public interface ValueSerializer * redundant information in values, if it can be derived from keys. */ V deserialize(K key, DataInputPlus in, int userVersion) throws IOException; + + class Unsupported implements ValueSerializer + { + @Override + public void serialize(K key, V value, DataOutputPlus out, int userVersion) + { + throw new UnsupportedOperationException(); + } + + @Override + public V deserialize(K key, DataInputPlus in, int userVersion) + { + throw new UnsupportedOperationException(); + } + } } diff --git a/src/java/org/apache/cassandra/net/Verb.java b/src/java/org/apache/cassandra/net/Verb.java index ad2317e5abc0..97eea759aa29 100644 --- a/src/java/org/apache/cassandra/net/Verb.java +++ b/src/java/org/apache/cassandra/net/Verb.java @@ -83,23 +83,18 @@ import org.apache.cassandra.schema.SchemaVersionVerbHandler; import org.apache.cassandra.service.EchoVerbHandler; import org.apache.cassandra.service.SnapshotVerbHandler; -import org.apache.cassandra.service.accord.AccordSerializers; import org.apache.cassandra.service.accord.AccordService; -import org.apache.cassandra.service.accord.AccordSyncPropagator; -import org.apache.cassandra.service.accord.AccordSyncPropagator.Notification; -import org.apache.cassandra.service.accord.FetchTopologies; -import org.apache.cassandra.service.accord.WatermarkCollector; import org.apache.cassandra.service.accord.interop.AccordInteropApply; import org.apache.cassandra.service.accord.interop.AccordInteropRead; import org.apache.cassandra.service.accord.interop.AccordInteropReadRepair; import org.apache.cassandra.service.accord.interop.AccordInteropStableThenRead; import org.apache.cassandra.service.accord.serializers.AcceptSerializers; +import org.apache.cassandra.service.accord.serializers.AccordSerializers; import org.apache.cassandra.service.accord.serializers.ApplySerializers; import org.apache.cassandra.service.accord.serializers.AwaitSerializers; import org.apache.cassandra.service.accord.serializers.BeginInvalidationSerializers; import org.apache.cassandra.service.accord.serializers.CheckStatusSerializers; import org.apache.cassandra.service.accord.serializers.CommitSerializers; -import org.apache.cassandra.service.accord.serializers.EnumSerializer; import org.apache.cassandra.service.accord.serializers.FetchSerializers; import org.apache.cassandra.service.accord.serializers.GetDurableBeforeSerializers; import org.apache.cassandra.service.accord.serializers.GetEphmrlReadDepsSerializers; @@ -110,7 +105,12 @@ import org.apache.cassandra.service.accord.serializers.ReadDataSerializer; import org.apache.cassandra.service.accord.serializers.RecoverySerializers; import org.apache.cassandra.service.accord.serializers.SetDurableSerializers; +import org.apache.cassandra.service.accord.serializers.SimpleReplySerializer; import org.apache.cassandra.service.accord.serializers.Version; +import org.apache.cassandra.service.accord.topology.AccordSyncPropagator; +import org.apache.cassandra.service.accord.topology.AccordSyncPropagator.Notification; +import org.apache.cassandra.service.accord.topology.FetchTopologies; +import org.apache.cassandra.service.accord.topology.WatermarkCollector; import org.apache.cassandra.service.consensus.migration.ConsensusKeyMigrationState; import org.apache.cassandra.service.consensus.migration.ConsensusKeyMigrationState.ConsensusKeyMigrationFinished; import org.apache.cassandra.service.paxos.Commit; @@ -320,7 +320,7 @@ public enum Verb DATA_MOVEMENT_EXECUTED_REQ (817, P1, rpcTimeout, MISC, () -> DataMovement.Status.serializer, () -> DataMovements.instance, DATA_MOVEMENT_EXECUTED_RSP ), // accord - ACCORD_SIMPLE_RSP (119, P2, writeTimeout, IMMEDIATE, () -> accordEmbedded(EnumSerializer.simpleReply), AccordService::responseHandlerOrNoop ), + ACCORD_SIMPLE_RSP (119, P2, writeTimeout, IMMEDIATE, () -> accordEmbedded(SimpleReplySerializer.reply), AccordService::responseHandlerOrNoop ), ACCORD_PRE_ACCEPT_RSP (120, P2, writeTimeout, IMMEDIATE, () -> accordEmbedded(PreacceptSerializers.reply), AccordService::responseHandlerOrNoop ), ACCORD_PRE_ACCEPT_REQ (121, P2, writeTimeout, IMMEDIATE, () -> accordEmbedded(PreacceptSerializers.request), AccordService::requestHandlerOrNoop, ACCORD_PRE_ACCEPT_RSP ), ACCORD_ACCEPT_RSP (122, P2, writeTimeout, IMMEDIATE, () -> accordEmbedded(AcceptSerializers.reply), AccordService::responseHandlerOrNoop ), @@ -357,10 +357,10 @@ public enum Verb ACCORD_GET_MAX_CONFLICT_REQ (153, P2, readTimeout, IMMEDIATE, () -> accordEmbedded(GetMaxConflictSerializers.request), AccordService::requestHandlerOrNoop, ACCORD_GET_MAX_CONFLICT_RSP), ACCORD_GET_DURABLE_BEFORE_RSP (154, P2, rpcTimeout, IMMEDIATE, () -> accordEmbedded(GetDurableBeforeSerializers.reply), AccordService::responseHandlerOrNoop ), ACCORD_GET_DURABLE_BEFORE_REQ (155, P2, rpcTimeout, IMMEDIATE, () -> accordEmbedded(GetDurableBeforeSerializers.request), AccordService::requestHandlerOrNoop, ACCORD_GET_DURABLE_BEFORE_RSP ), - ACCORD_SET_SHARD_DURABLE_REQ (156, P2, rpcTimeout, IMMEDIATE, () -> accordEmbedded(SetDurableSerializers.shardDurable), AccordService::requestHandlerOrNoop, ACCORD_SIMPLE_RSP ), - ACCORD_SET_GLOBALLY_DURABLE_REQ (157, P2, rpcTimeout, IMMEDIATE, () -> accordEmbedded(SetDurableSerializers.globallyDurable),AccordService::requestHandlerOrNoop, ACCORD_SIMPLE_RSP ), + ACCORD_SET_SHARD_DURABLE_REQ (156, P2, rpcTimeout, MISC, () -> accordEmbedded(SetDurableSerializers.shardDurable), AccordService::requestHandlerOrNoop, ACCORD_SIMPLE_RSP ), + ACCORD_SET_GLOBALLY_DURABLE_REQ (157, P2, rpcTimeout, MISC, () -> accordEmbedded(SetDurableSerializers.globallyDurable),AccordService::requestHandlerOrNoop, ACCORD_SIMPLE_RSP ), - ACCORD_SYNC_NOTIFY_RSP (158, P2, writeTimeout, MISC, () -> accordEmbedded(EnumSerializer.simpleReply), RESPONSE_HANDLER), + ACCORD_SYNC_NOTIFY_RSP (158, P2, writeTimeout, MISC, () -> accordEmbedded(SimpleReplySerializer.reply), RESPONSE_HANDLER), ACCORD_SYNC_NOTIFY_REQ (159, P2, writeTimeout, MISC, () -> accordEmbedded(Notification.serializer), () -> AccordSyncPropagator.verbHandler, ACCORD_SYNC_NOTIFY_RSP ), CONSENSUS_KEY_MIGRATION (160, P1, writeTimeout, MISC, () -> accordEmbedded(ConsensusKeyMigrationFinished.serializer),() -> ConsensusKeyMigrationState.consensusKeyMigrationFinishedHandler), diff --git a/src/java/org/apache/cassandra/schema/DistributedMetadataLogKeyspace.java b/src/java/org/apache/cassandra/schema/DistributedMetadataLogKeyspace.java index 82adbb6d80d7..02c710ae49fc 100644 --- a/src/java/org/apache/cassandra/schema/DistributedMetadataLogKeyspace.java +++ b/src/java/org/apache/cassandra/schema/DistributedMetadataLogKeyspace.java @@ -35,7 +35,7 @@ import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.exceptions.CasWriteTimeoutException; import org.apache.cassandra.locator.MetaStrategy; -import org.apache.cassandra.service.accord.fastpath.FastPathStrategy; +import org.apache.cassandra.service.accord.topology.FastPathStrategy; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.MetadataSnapshots; diff --git a/src/java/org/apache/cassandra/schema/KeyspaceParams.java b/src/java/org/apache/cassandra/schema/KeyspaceParams.java index 4f4123536792..5649fc444084 100644 --- a/src/java/org/apache/cassandra/schema/KeyspaceParams.java +++ b/src/java/org/apache/cassandra/schema/KeyspaceParams.java @@ -28,7 +28,7 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.service.ClientState; -import org.apache.cassandra.service.accord.fastpath.FastPathStrategy; +import org.apache.cassandra.service.accord.topology.FastPathStrategy; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.serialization.MetadataSerializer; import org.apache.cassandra.tcm.serialization.Version; diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index f8057ed8b764..d466f09ed546 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -84,7 +84,7 @@ import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.schema.ColumnMetadata.ClusteringOrder; import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; -import org.apache.cassandra.service.accord.fastpath.FastPathStrategy; +import org.apache.cassandra.service.accord.topology.FastPathStrategy; import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; import org.apache.cassandra.utils.ByteBufferUtil; diff --git a/src/java/org/apache/cassandra/schema/TableId.java b/src/java/org/apache/cassandra/schema/TableId.java index fa1cdd5a71c6..6be485e8b48d 100644 --- a/src/java/org/apache/cassandra/schema/TableId.java +++ b/src/java/org/apache/cassandra/schema/TableId.java @@ -188,9 +188,14 @@ public final boolean equals(Object o) @Override public String toString() + { + return toShortString("tid:"); + } + + public String toShortString(String prefix) { if (msb == MAGIC) - return "tid:" + Long.toHexString(lsb); + return prefix + Long.toHexString(lsb); return asUUID().toString(); } diff --git a/src/java/org/apache/cassandra/schema/TableMetadata.java b/src/java/org/apache/cassandra/schema/TableMetadata.java index 05c92115b958..4f9fe1320afb 100644 --- a/src/java/org/apache/cassandra/schema/TableMetadata.java +++ b/src/java/org/apache/cassandra/schema/TableMetadata.java @@ -76,7 +76,7 @@ import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.service.accord.fastpath.FastPathStrategy; +import org.apache.cassandra.service.accord.topology.FastPathStrategy; import org.apache.cassandra.service.consensus.TransactionalMode; import org.apache.cassandra.service.consensus.migration.TransactionalMigrationFromMode; import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java index abc8714c6711..c9fb20d3c50c 100644 --- a/src/java/org/apache/cassandra/schema/TableParams.java +++ b/src/java/org/apache/cassandra/schema/TableParams.java @@ -34,7 +34,7 @@ import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.service.accord.fastpath.FastPathStrategy; +import org.apache.cassandra.service.accord.topology.FastPathStrategy; import org.apache.cassandra.service.consensus.TransactionalMode; import org.apache.cassandra.service.consensus.migration.TransactionalMigrationFromMode; import org.apache.cassandra.service.reads.PercentileSpeculativeRetryPolicy; diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 09527b5d23ad..2446dd7fa866 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -163,7 +163,6 @@ import org.apache.cassandra.metrics.StorageMetrics; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.RepairCoordinator; -import org.apache.cassandra.repair.RepairParallelism; import org.apache.cassandra.repair.SharedContext; import org.apache.cassandra.repair.autorepair.AutoRepair; import org.apache.cassandra.repair.messages.RepairOption; @@ -178,7 +177,6 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.schema.ViewMetadata; -import org.apache.cassandra.service.accord.AccordKeyspace.AccordColumnFamilyStores; import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationTarget; @@ -190,7 +188,6 @@ import org.apache.cassandra.service.paxos.cleanup.PaxosCleanupLocalCoordinator; import org.apache.cassandra.service.paxos.cleanup.PaxosRepairState; import org.apache.cassandra.service.snapshot.SnapshotManager; -import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.streaming.StreamManager; import org.apache.cassandra.streaming.StreamResultFuture; import org.apache.cassandra.streaming.StreamState; @@ -249,6 +246,7 @@ import static java.util.Arrays.stream; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.NANOSECONDS; import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.stream.Collectors.toList; import static java.util.stream.Collectors.toMap; @@ -260,7 +258,6 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.PAXOS_REPAIR_ON_TOPOLOGY_CHANGE_RETRY_DELAY_SECONDS; import static org.apache.cassandra.config.CassandraRelevantProperties.REPLACE_ADDRESS_FIRST_BOOT; import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_WRITE_SURVEY; -import static org.apache.cassandra.db.ColumnFamilyStore.FlushReason.INTERNALLY_FORCED; import static org.apache.cassandra.index.SecondaryIndexManager.getIndexName; import static org.apache.cassandra.index.SecondaryIndexManager.isIndexColumnFamily; import static org.apache.cassandra.io.util.FileUtils.ONE_MIB; @@ -3189,27 +3186,9 @@ private FutureTask createRepairTask(final int cmd, final String keyspace return new FutureTask<>(task); } - public RepairCoordinator repairAccordKeyspace(String keyspace, Collection> ranges) + public RepairCoordinator newRepairCoordinator(String keyspace, RepairOption options) { int cmd = nextRepairCommand.incrementAndGet(); - RepairOption options = new RepairOption(RepairParallelism.PARALLEL, // parallelism - false, // primaryRange - false, // incremental - false, // trace - 5, // jobThreads - ranges, // ranges - true, // pullRepair - true, // forceRepair - PreviewKind.NONE, // previewKind - false, // optimiseStreams - true, // ignoreUnreplicatedKeyspaces - true, // repairData - false, // repairPaxos - true, // dontPurgeTombstones - false, // repairAccord - false // permit no quorum - ); - return new RepairCoordinator(this, cmd, options, keyspace); } @@ -3889,7 +3868,7 @@ protected synchronized void drain(boolean isFinalShutdown) throws IOException, I } if (AccordService.isSetupOrStarting()) - AccordService.unsafeInstance().markShuttingDown(); + AccordService.unsafeInstance().stop(); // In-progress writes originating here could generate hints to be written, // which is currently scheduled on the mutation stage. So shut down MessagingService @@ -3907,12 +3886,14 @@ protected synchronized void drain(boolean isFinalShutdown) throws IOException, I if (AccordService.isSetupOrStarting()) { - logger.info("Flushing Accord caches"); - if (!AccordService.unsafeInstance().flushCaches().awaitUninterruptibly(1, MINUTES)) - logger.error("Could not flush Accord caches promptly"); - if (AccordColumnFamilyStores.commandsForKey != null) - AccordColumnFamilyStores.commandsForKey.forceBlockingFlush(INTERNALLY_FORCED); - AccordService.unsafeInstance().shutdownAndWait(1, MINUTES); + try + { + AccordService.unsafeInstance().shutdownAndWait(DatabaseDescriptor.getAccord().shutdown_grace_period.toDuration().toNanos(), NANOSECONDS); + } + catch (Throwable t) + { + logger.error("AccordService exception shutting down", t); + } } // ScheduledExecutors shuts down after MessagingService, as MessagingService may issue tasks to it. diff --git a/src/java/org/apache/cassandra/service/accord/AccordCache.java b/src/java/org/apache/cassandra/service/accord/AccordCache.java index e4623edfecf5..0ff30acde892 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordCache.java +++ b/src/java/org/apache/cassandra/service/accord/AccordCache.java @@ -68,6 +68,8 @@ import org.apache.cassandra.service.accord.AccordCacheEntry.LoadExecutor; import org.apache.cassandra.service.accord.AccordCacheEntry.Status; import org.apache.cassandra.service.accord.events.CacheEvents; +import org.apache.cassandra.service.accord.journal.CommandChangeWriter; +import org.apache.cassandra.service.accord.journal.CommandChanges; import org.apache.cassandra.service.accord.serializers.CommandSerializers; import org.apache.cassandra.service.accord.serializers.Version; import org.apache.cassandra.utils.NoSpamLogger; @@ -532,7 +534,7 @@ public void release(AccordSafeState safeRef, AccordTask owner) AccordCacheEntry node = cache.get(key); - require(!safeRef.invalidated()); + require(!safeRef.isUnsafe()); require(safeRef.global() != null, "safeRef node is null for %s", key); require(safeRef.global() == node, "safeRef node not in map: %s != %s", safeRef.global(), node); require(node.references() > 0, "references (%d) are zero for %s (%s)", node.references(), key, node); @@ -561,7 +563,7 @@ else if (node.isLoadingOrWaiting()) { evict = node.is(LOADED) && node.isNull(); } - safeRef.invalidate(); + safeRef.markUnsafe(); if (node.decrement() == 0) { @@ -1298,7 +1300,7 @@ public Object fullShrink(TxnId txnId, Command value) try { - return AccordJournal.asSerializedChange(null, value, Version.LATEST); + return CommandChangeWriter.asSerializedChange(null, value, Version.LATEST); } catch (IOException e) { @@ -1314,7 +1316,7 @@ public Shrink decideFullShrink(TxnId txnId, Command value) Invariants.expect(value.saveStatus().compareTo(SaveStatus.ReadyToExecute) < 0); // TODO (expected): improve heuristics and consider transaction size - if (SHRINK_WITHOUT_LOCK < 0 || value.partialDeps() == null || value.partialDeps().txnIds().size() < SHRINK_WITHOUT_LOCK) + if (SHRINK_WITHOUT_LOCK <= 0 || value.partialDeps() == null || value.partialDeps().txnIds().size() <= SHRINK_WITHOUT_LOCK) return Shrink.DONE; return Shrink.PERFORM_WITHOUT_LOCK; @@ -1323,7 +1325,7 @@ public Shrink decideFullShrink(TxnId txnId, Command value) @Override public @Nullable Command inflate(AccordCommandStore commandStore, TxnId key, Object serialized) { - AccordJournal.Builder builder = new AccordJournal.Builder(key); + CommandChanges builder = new CommandChanges(key); ByteBuffer buffer = (ByteBuffer) serialized; buffer.mark(); try (DataInputBuffer buf = new DataInputBuffer(buffer, false)) diff --git a/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java b/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java index 6bac1bd23e95..1904709b8c66 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java +++ b/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java @@ -18,17 +18,20 @@ package org.apache.cassandra.service.accord; +import java.util.Arrays; +import java.util.Comparator; import java.util.List; +import java.util.Map; import java.util.NavigableMap; import java.util.Objects; import java.util.concurrent.Callable; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.concurrent.locks.Lock; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.IntFunction; +import java.util.function.Predicate; import javax.annotation.Nullable; @@ -42,18 +45,29 @@ import accord.api.DataStore; import accord.api.Journal; import accord.api.LocalListeners; +import accord.api.LocalListeners.TxnListener; import accord.api.ProgressLog; import accord.api.RoutingKey; import accord.impl.AbstractReplayer; +import accord.impl.AbstractReplayer.Mode; import accord.impl.AbstractSafeCommandStore.CommandStoreCaches; +import accord.impl.DefaultLocalListeners; import accord.impl.progresslog.DefaultProgressLog; +import accord.impl.progresslog.TxnState; import accord.local.Command; import accord.local.CommandStore; import accord.local.CommandStores; import accord.local.CommandSummaries; +import accord.local.MaxConflicts; +import accord.local.MaxDecidedRX; import accord.local.NodeCommandStoreService; import accord.local.PreLoadContext; +import accord.local.PreLoadContext.Empty; import accord.local.RedundantBefore; +import accord.local.RedundantBefore.Bounds; +import accord.local.RedundantStatus.Property; +import accord.local.RedundantStatus.SomeStatus; +import accord.local.RejectBefore; import accord.local.SafeCommandStore; import accord.local.cfk.CommandsForKey; import accord.primitives.PartialTxn; @@ -61,32 +75,60 @@ import accord.primitives.Ranges; import accord.primitives.RoutableKey; import accord.primitives.Route; +import accord.primitives.SaveStatus; +import accord.primitives.Status; import accord.primitives.Timestamp; import accord.primitives.TxnId; import accord.utils.Invariants; +import accord.utils.ReducingRangeMap; import accord.utils.UnhandledEnum; import accord.utils.async.AsyncChain; import accord.utils.async.AsyncChains; +import accord.utils.async.AsyncResult; import accord.utils.async.AsyncResults.CountingResult; import org.apache.cassandra.config.AccordSpec; +import org.apache.cassandra.config.AccordSpec.JournalSpec.ReplayMode; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.journal.Descriptor; import org.apache.cassandra.metrics.LogLinearDecayingHistograms; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.TableMetadataRef; +import org.apache.cassandra.service.accord.AccordDurableOnFlush.ReportDurable; import org.apache.cassandra.service.accord.AccordKeyspace.CommandsForKeyAccessor; import org.apache.cassandra.service.accord.IAccordService.AccordCompactionInfo; import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.service.accord.journal.AccordJournal; +import org.apache.cassandra.service.accord.journal.JournalRangeIndex; import org.apache.cassandra.service.accord.txn.TxnRead; -import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.concurrent.Condition; import static accord.api.Journal.CommandUpdate; import static accord.api.Journal.FieldUpdates; +import static accord.impl.progresslog.DefaultProgressLog.ModeFlag.CATCH_UP; +import static accord.local.RedundantStatus.Property.LOCALLY_APPLIED; +import static accord.local.RedundantStatus.Property.LOCALLY_DURABLE_TO_COMMAND_STORE; +import static accord.local.RedundantStatus.Property.LOCALLY_DURABLE_TO_DATA_STORE; +import static accord.local.RedundantStatus.SomeStatus.LOCALLY_DURABLE_TO_COMMAND_STORE_ONLY; +import static accord.local.RedundantStatus.SomeStatus.LOCALLY_DURABLE_TO_DATA_STORE_ONLY; +import static accord.primitives.Status.Durability.HasOutcome.Universal; import static accord.utils.Invariants.require; -import static org.apache.cassandra.journal.Params.ReplayMode.ONLY_NON_DURABLE; +import static org.apache.cassandra.config.DatabaseDescriptor.getAccord; +import static org.apache.cassandra.io.util.CompressedFrameDataInputPlus.readList; +import static org.apache.cassandra.io.util.CompressedFrameDataInputPlus.readOne; +import static org.apache.cassandra.io.util.CompressedFrameDataOutputPlus.writeList; +import static org.apache.cassandra.io.util.CompressedFrameDataOutputPlus.writeOne; +import static org.apache.cassandra.service.accord.journal.ReplayMarkers.saveDirectory; +import static org.apache.cassandra.service.accord.serializers.CommandStoreSerializers.maxConflicts; +import static org.apache.cassandra.service.accord.serializers.CommandStoreSerializers.maxDecidedRX; +import static org.apache.cassandra.service.accord.serializers.CommandStoreSerializers.progressLogState; +import static org.apache.cassandra.service.accord.serializers.CommandStoreSerializers.redundantBefore; +import static org.apache.cassandra.service.accord.serializers.CommandStoreSerializers.rejectBefore; +import static org.apache.cassandra.service.accord.serializers.CommandStoreSerializers.txnListener; public class AccordCommandStore extends CommandStore { @@ -152,20 +194,35 @@ public void close() } } + static class Termination extends Condition.Sync + { + private boolean commandStoreFlushed; + private boolean dataStoreFlushed; + private boolean isReadyToTerminate() + { + return commandStoreFlushed && dataStoreFlushed; + } + } + static final AtomicReferenceFieldUpdater safeRedundantBeforeUpdater = AtomicReferenceFieldUpdater.newUpdater(AccordCommandStore.class, SafeRedundantBefore.class, "safeRedundantBefore"); + static final AtomicReferenceFieldUpdater terminatedUpdater + = AtomicReferenceFieldUpdater.newUpdater(AccordCommandStore.class, Termination.class, "terminated"); static final AtomicLong nextSafeRedundantBeforeTicket = new AtomicLong(); + private static final AtomicLong lastSystemTimestampMicros = new AtomicLong(); + public final String loggingId; public final Journal journal; private final AccordExecutor sharedExecutor; final AccordExecutor.SequentialExecutor exclusiveExecutor; private final ExclusiveCaches caches; - private long lastSystemTimestampMicros = Long.MIN_VALUE; private final RangeIndex rangeIndex; private final TableId tableId; private TableMetadataRef metadata; + volatile SafeRedundantBefore safeRedundantBefore; + volatile Termination terminated; private AccordSafeCommandStore current; LogLinearDecayingHistograms.Buffer metricsBuffer; @@ -193,12 +250,12 @@ public AccordCommandStore(int id, { commands = exclusive.commands.newInstance(this); commandsForKey = exclusive.commandsForKey.newInstance(this); - this.caches = new ExclusiveCaches(sharedExecutor.lock, exclusive.global, commands, commandsForKey); + this.caches = new ExclusiveCaches(sharedExecutor.unsafeLock(), exclusive.global, commands, commandsForKey); } this.exclusiveExecutor = sharedExecutor.executor(id); { - AccordSpec.RangeIndexMode mode = DatabaseDescriptor.getAccord().range_index_mode; + AccordSpec.RangeIndexMode mode = getAccord().range_index_mode; switch (mode) { default: throw new UnhandledEnum(mode); @@ -329,8 +386,8 @@ CommandsForKey loadCommandsForKey(RoutableKey key) if (cfk == null) return null; RedundantBefore.QuickBounds bounds = safeGetRedundantBefore().get(key); - if (bounds == null) - return cfk; // TODO (required): I don't think this should be possible? but we hit it on some test + if (!Invariants.expect(bounds != null, "No RedundantBefore information found when loading key %s", key)) + return cfk; return cfk.withGcBeforeAtLeast(bounds.gcBefore, false); } @@ -351,8 +408,7 @@ Runnable saveCommandsForKey(RoutingKey key, CommandsForKey after, Object seriali public long nextSystemTimestampMicros() { - lastSystemTimestampMicros = Math.max(TimeUnit.MILLISECONDS.toMicros(Clock.Global.currentTimeMillis()), lastSystemTimestampMicros + 1); - return lastSystemTimestampMicros; + return lastSystemTimestampMicros.accumulateAndGet(node.now(), (a, b) -> Math.max(a + 1, b)); } @Override public AsyncChain chain(PreLoadContext loadCtx, Function function) @@ -416,6 +472,70 @@ public void abort(AccordSafeCommandStore store) @Override public void shutdown() { + shutdownAsync(); + } + + public AsyncResult shutdownAsync() + { + terminatedUpdater.compareAndSet(this, null, new Termination()); + progressLog.stop(); + return execute((Empty)() -> "Shutdown", safeStore -> { + exclusiveExecutor.stop(); + logger.info("{} stopping. Durably applied: {}, waiting: {}", this, + DurablyAppliedTo.summarise(safeStore.redundantBefore(), DurablyAppliedTo::isDone), + DurablyAppliedTo.summarise(safeStore.redundantBefore(), DurablyAppliedTo::isNotDone)); + this.ensureDurable(null, ReportDurable.commandStoreFlush()); + dataStore.ensureDurable(this, RedundantBefore.EMPTY, ReportDurable.DATA_STORE_FLUSH); + }); + } + + @Override + public void markShardDurable(SafeCommandStore safeStore, TxnId globalSyncId, Ranges durableRanges, Status.Durability.HasOutcome durability) + { + super.markShardDurable(safeStore, globalSyncId, durableRanges, durability); + if (durability == Universal) + rangeIndex.prune(globalSyncId, durableRanges, safeStore.redundantBefore()); + } + + @Override + protected void markExclusiveSyncPointLocallyApplied(SafeCommandStore safeStore, TxnId syncId, Ranges ranges, SaveStatus prevStatus) + { + super.markExclusiveSyncPointLocallyApplied(safeStore, syncId, ranges, prevStatus); + rangeIndex.prune(syncId, ranges, safeStore.redundantBefore()); + } + + void maybeTerminated(boolean setCommandStoreDurable, boolean setDataStoreDurable) + { + if (terminated != null) + { + if (setCommandStoreDurable) terminated.commandStoreFlushed = true; + if (setDataStoreDurable) terminated.dataStoreFlushed = true; + if (terminated.isReadyToTerminate()) + { + Invariants.require(exclusiveExecutor.stopped()); + boolean syncPointsDurable = unsafeGetRedundantBefore().foldl((b, v, p2, p3) -> { + return v && (b == null || b.maxBound(LOCALLY_APPLIED).compareTo(b.maxBoundBoth(LOCALLY_DURABLE_TO_DATA_STORE, LOCALLY_DURABLE_TO_COMMAND_STORE)) <= 0); + }, true, null, null, ignore -> false); + + if (!syncPointsDurable) + logger.error("{} has flushed command and data stores, but sync points recorded in RedundantBefore are not durable: {}", this, DurablyAppliedTo.summarise(unsafeGetRedundantBefore())); + + exclusiveExecutor.terminate(); + terminated.signalAll(); + } + } + } + + public boolean awaitTerminationUntil(long deadlineNanos) + { + if (terminated == null) + throw new IllegalStateException("Not shutdown"); + return terminated.awaitUntilThrowUncheckedOnInterrupt(deadlineNanos); + } + + public boolean isTerminated() + { + return terminated != null && terminated.isSignalled(); } public void appendCommands(List diffs, Runnable onFlush) @@ -481,44 +601,81 @@ public final RedundantBefore safeGetRedundantBefore() return safeRedundantBefore.redundantBefore; } - public AccordCommandStoreReplayer replayer() + @Override + public AccordCommandStoreReplayer replayer(Mode mode) { - boolean replayOnlyNonDurable = true; - if (journal instanceof AccordJournal) - replayOnlyNonDurable = ((AccordJournal)journal).configuration().replayMode() == ONLY_NON_DURABLE; - return new AccordCommandStoreReplayer(this, replayOnlyNonDurable); + ReplayMode replayMode = getAccord().journal.replay; + return new AccordCommandStoreReplayer(this, mode); } - + static final AtomicLong nextDurabilityLoggingId = new AtomicLong(); + @Override protected void ensureDurable(Ranges ranges, RedundantBefore onCommandStoreDurable) { - if (node().isReplaying()) + ensureDurable(ranges, ReportDurable.of(onCommandStoreDurable)); + } + + protected void ensureDurable() + { + RedundantBefore forCommandStore = nonDurable(unsafeGetRedundantBefore(), LOCALLY_DURABLE_TO_COMMAND_STORE, LOCALLY_DURABLE_TO_COMMAND_STORE_ONLY); + RedundantBefore forDataStore = nonDurable(unsafeGetRedundantBefore(), LOCALLY_DURABLE_TO_DATA_STORE, LOCALLY_DURABLE_TO_DATA_STORE_ONLY); + this.ensureDurable(forCommandStore.ranges(Objects::nonNull), forCommandStore); + dataStore.ensureDurable(this, forDataStore, 0); + } + + private RedundantBefore nonDurable(RedundantBefore redundantBefore, Property durableProperty, SomeStatus durableStatus) + { + return redundantBefore.map(b -> { + if (b == null) + return null; + + TxnId applied = b.maxBound(LOCALLY_APPLIED); + TxnId durable = b.maxBound(durableProperty); + if (applied.compareTo(durable) <= 0) + return null; + return Bounds.create(b.range, b.maxBound(LOCALLY_APPLIED), durableStatus, null); + }); + } + + protected void ensureDurable(@Nullable Ranges ranges, ReportDurable onCommandStoreDurable) + { + if (node().isReplaying() && onCommandStoreDurable.flags == 0 && unsafeGetRedundantBefore().isAtLeast(onCommandStoreDurable.redundantBefore)) return; long reportId = nextDurabilityLoggingId.incrementAndGet(); - logger.debug("{} awaiting local metadata durability for {} ({})", this, ranges, reportId); + logger.debug("{} durability: ensuring for {} ({})", this, onCommandStoreDurable, reportId); executor().afterSubmittedAndConsequences(() -> { - logger.debug("{}: saving intersecting keys ({})", this, reportId); + logger.debug("{} durability: saving intersecting keys ({})", this, reportId); class Ready extends CountingResult implements Runnable { public Ready() { super(1); } @Override public void run() { decrement(); } + + void maybeFlush(ExclusiveCaches caches, AccordCacheEntry e) + { + if (e.isModified()) + { + increment(); + caches.global().saveWhenReadyExclusive(e, this); + } + } } Ready ready = new Ready(); try (ExclusiveCaches caches = lockCaches()) { - for (Range range : ranges) + if (ranges == null) { - for (RoutingKey k : caches.commandsForKeys().keysBetween(range.start(), range.startInclusive(), range.end(), range.endInclusive())) + for (AccordCacheEntry e : caches.commandsForKeys()) + ready.maybeFlush(caches, e); + } + else + { + for (Range range : ranges) { - AccordCacheEntry e = caches.commandsForKeys().getUnsafe(k); - if (e.isModified()) - { - ready.increment(); - caches.global().saveWhenReadyExclusive(e, ready); - } + for (RoutingKey k : caches.commandsForKeys().keysBetween(range.start(), range.startInclusive(), range.end(), range.endInclusive())) + ready.maybeFlush(caches, caches.commandsForKeys().getUnsafe(k)); } } } @@ -526,19 +683,14 @@ class Ready extends CountingResult implements Runnable ready.invoke((success, fail) -> { if (fail != null) { - logger.error("{}: failed to ensure durability of {} ({})", this, ranges, reportId, fail); + logger.error("{} failed to ensure durability of {} ({})", this, ranges, reportId, fail); } else { - logger.debug("{}: waiting for CommandsForKey to flush ({})", this, reportId); + logger.debug("{} waiting for CommandsForKey to flush ({})", this, reportId); ColumnFamilyStore cfs = AccordKeyspace.AccordColumnFamilyStores.commandsForKey; - AccordDurableOnFlush onFlush = null; - while (onFlush == null) - onFlush = cfs.getCurrentMemtable().ensureFlushListener(AccordDataStore.FlushListenerKey.KEY, AccordDurableOnFlush::new); - - if (!onFlush.add(id, onCommandStoreDurable)) - AccordDurableOnFlush.notify(cfs.metadata(), this, onCommandStoreDurable); + AccordDurableOnFlush.notifyOnDurable(cfs, this, onCommandStoreDurable); } }); ready.decrement(); @@ -561,26 +713,25 @@ public void unsafeUpdateRangesForEpoch() public static class AccordCommandStoreReplayer extends AbstractReplayer { private final AccordCommandStore commandStore; - private final boolean onlyNonDurable; - private AccordCommandStoreReplayer(AccordCommandStore commandStore, boolean onlyNonDurable) + private AccordCommandStoreReplayer(AccordCommandStore commandStore, Mode mode) { - super(commandStore, null); + super(commandStore, mode, null); this.commandStore = commandStore; - this.onlyNonDurable = onlyNonDurable; } @Override public AsyncChain replay(TxnId txnId) { - if (onlyNonDurable && !maybeShouldReplay(txnId)) + if (!maybeShouldReplay(txnId)) return AsyncChains.success(null); return commandStore.chain(PreLoadContext.contextFor(txnId, "Replay"), safeStore -> { - if (onlyNonDurable && !shouldReplay(txnId, safeStore.unsafeGet(txnId).current().participants())) + Replay replay = shouldReplay(txnId, safeStore.unsafeGet(txnId).current().participants()); + if (replay == Replay.NONE) return null; - initialiseState(safeStore, txnId); + replay(safeStore, txnId, replay); return safeStore.unsafeGet(txnId).current().route(); }); } @@ -622,6 +773,141 @@ void maybeLoadRangesForEpoch(CommandStores.RangesForEpoch rangesForEpoch) loadRangesForEpoch(rangesForEpoch); } + AsyncChain saveState(Descriptor descriptor) + { + return chain((AccordExecutor.Unterminatable)() -> "Save State", safeStore -> { + File storeDir = storeSaveDir(); + { + File[] tmpDirs = listTmpSaveDirs(storeDir); + if (tmpDirs != null) + { + logger.info("Cleaning up incomplete save points: {}", Arrays.toString(tmpDirs)); + for (File dir : tmpDirs) + dir.tryDeleteRecursive(); + } + } + + RedundantBefore validateRedundantBefore = journal.loadRedundantBefore(id); + Invariants.expect(validateRedundantBefore.equals(unsafeGetRedundantBefore()), "Journal RedundantBefore does not match in memory: %s != %s", validateRedundantBefore, unsafeGetRedundantBefore()); + + File[] sortedSaveDirs = listSortedSaveDirs(storeDir); + File tmpSaveDir = new File(storeDir, "tmp" + descriptor.timestamp); + File saveDir = new File(storeDir, "" + descriptor.timestamp); + if (sortedSaveDirs != null && Long.parseLong(sortedSaveDirs[sortedSaveDirs.length - 1].name()) >= descriptor.timestamp) + { + logger.error("There already exists a save point {} >= {}; aborting.", sortedSaveDirs[sortedSaveDirs.length - 1].name(), descriptor.timestamp); + return false; + } + + try + { + logger.info("{} saving state to {}", this, saveDir); + tmpSaveDir.createDirectoriesIfNotExists(); + writeOne(new File(tmpSaveDir, "max_decidedrx"), unsafeGetMaxDecidedRX(), maxDecidedRX); + writeOne(new File(tmpSaveDir, "max_conflicts"), unsafeGetMaxConflicts(), maxConflicts); + writeOne(new File(tmpSaveDir, "reject_before"), unsafeGetRejectBefore(), rejectBefore); + writeList(new File(tmpSaveDir, "listeners"), ((DefaultLocalListeners)listeners).snapshot(), txnListener); + writeList(new File(tmpSaveDir, "progress_log"), ((DefaultProgressLog)progressLog).snapshot(), progressLogState); + rangeIndex.save(new File(tmpSaveDir, "range_index")); + writeOne(new File(tmpSaveDir, "redundant_before"), unsafeGetRedundantBefore(), redundantBefore); + tmpSaveDir.move(saveDir); + } + catch (Throwable t) + { + logger.error("{} failed to save replay state {}", this, saveDir, t); + tmpSaveDir.tryDeleteRecursive(); + saveDir.tryDeleteRecursive(); + return false; + } + + if (sortedSaveDirs != null) + { + int delete = (sortedSaveDirs.length + 1) - getAccord().journal.retainSavePoints; + if (delete > 0) + { + sortedSaveDirs = Arrays.copyOf(sortedSaveDirs, delete); + logger.debug("Deleting old save points: {}", Arrays.toString(sortedSaveDirs)); + for (File dir : sortedSaveDirs) + dir.tryDeleteRecursive(); + } + } + + return true; + }); + } + + private File storeSaveDir() + { + return new File(saveDirectory(), String.format("%s_%d", tableId().toShortString(""), id())); + } + + private static File[] listSortedSaveDirs(File storeDir) + { + File[] savePoints = storeDir.tryList(f -> f.isDirectory() && f.name().matches("[0-9]+")); + if (savePoints == null || savePoints.length == 0) + return null; + + Arrays.sort(savePoints, Comparator.comparingLong(f -> Long.parseLong(f.name()))); + return savePoints; + } + + private static File[] listTmpSaveDirs(File storeDir) + { + File[] tmpDirs = storeDir.tryList(f -> f.isDirectory() && f.name().matches("tmp[0-9]+")); + if (tmpDirs == null || tmpDirs.length == 0) + return null; + return tmpDirs; + } + + AsyncChain> restoreState() + { + return chain((Empty)() -> "Restore State", safeStore -> { + File storeDir = storeSaveDir(); + File[] savePoints = listSortedSaveDirs(storeDir); + if (savePoints == null) + { + logger.info("{} no save points found at {}", this, storeDir); + return null; + } + + File savePoint = savePoints[savePoints.length - 1]; + long segment = Long.parseLong(savePoint.name()); + MaxDecidedRX mxd; MaxConflicts mxc; RejectBefore rjb; + List dll; List dpl; Object rgi; + RedundantBefore rdb; + try + { + logger.info("{} loading state from {}", this, savePoint); + mxd = readOne(new File(savePoint, "max_decidedrx"), maxDecidedRX); + mxc = readOne(new File(savePoint, "max_conflicts"), maxConflicts); + rjb = readOne(new File(savePoint, "reject_before"), rejectBefore); + dll = readList(new File(savePoint, "listeners"), txnListener); + dpl = readList(new File(savePoint, "progress_log"), progressLogState); + rgi = rangeIndex.load(new File(savePoint, "range_index")); + rdb = readOne(new File(savePoint, "redundant_before"), redundantBefore); + } + catch (Throwable t) + { + logger.warn("{} could not replay save point {}", this, savePoint, t); + return null; + } + + if (journal instanceof AccordJournal && ((AccordJournal)journal).maxDescriptor() <= segment) + Invariants.expect(rdb.equals(unsafeGetRedundantBefore())); + + rangeIndex.restore(rgi); + unsafeSetMaxDecidedRX(mxd); + unsafeSetMaxConflicts(mxc); + unsafeSetRejectBefore(rjb); + ((DefaultLocalListeners) listeners).restore(dll); + boolean unsetCatchup = ((DefaultProgressLog) progressLog).setModeExclusive(safeStore, CATCH_UP); + ((DefaultProgressLog) progressLog).restore(safeStore, dpl); + if (unsetCatchup) + ((DefaultProgressLog) progressLog).unsetModeExclusive(CATCH_UP); + return Map.entry(id, segment + 1); + }); + } + // TODO (expected): handle journal failures, and consider how we handle partial failures. // Very likely we will not be able to safely or cleanly handle partial failures of this logic, but decide and document. // TODO (desired): consider merging with PersistentField? This version is cheaper to manage which may be preferable at the CommandStore level. @@ -663,7 +949,74 @@ public String toString() if (metadata != null) sb.append(metadata).append('|'); sb.append(tableId); - sb.append('|').append(id).append(',').append(node.id().id).append(']'); + sb.append('|') + .append(id).append(',') + .append(executor().executorId).append(',') + .append(node.id().id) + .append(']'); return sb.toString(); } + + public static class DurablyAppliedTo + { + final TxnId journal, commandStore, dataStore; + + public DurablyAppliedTo(Bounds bounds) + { + this(bounds.maxBound(LOCALLY_APPLIED), bounds.maxBound(LOCALLY_DURABLE_TO_COMMAND_STORE), bounds.maxBound(LOCALLY_DURABLE_TO_DATA_STORE)); + } + + public DurablyAppliedTo(TxnId journal, TxnId commandStore, TxnId dataStore) + { + this.journal = journal; + this.commandStore = commandStore; + this.dataStore = dataStore; + } + + @Override + public String toString() + { + return "journal:" + journal + ", commandStore:" + commandStore + ", dataStore:" + dataStore; + } + + public boolean isDone() + { + return journal.compareTo(TxnId.min(commandStore, dataStore)) <= 0; + } + + public boolean isNotDone() + { + return !isDone(); + } + + @Override + public boolean equals(Object that) + { + return that instanceof DurablyAppliedTo && equals((DurablyAppliedTo) that); + } + + public boolean equals(DurablyAppliedTo that) + { + return this.journal.equals(that.journal) + && this.commandStore.equals(that.commandStore) + && this.dataStore.equals(that.dataStore); + } + + public static ReducingRangeMap summarise(RedundantBefore redundantBefore) + { + return redundantBefore.map(b -> b == null ? null : new DurablyAppliedTo(b), DurablyAppliedTo[]::new); + } + + public static ReducingRangeMap summarise(RedundantBefore redundantBefore, Predicate include) + { + return redundantBefore.map(b -> { + if (b == null) + return null; + DurablyAppliedTo result = new DurablyAppliedTo(b); + if (!include.test(result)) + return null; + return result; + }, DurablyAppliedTo[]::new); + } + } } diff --git a/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java b/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java index b885ae8424f0..df2ef42fb98c 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java +++ b/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java @@ -17,11 +17,18 @@ */ package org.apache.cassandra.service.accord; +import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Stream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import accord.api.Agent; import accord.api.DataStore; import accord.api.Journal; @@ -32,12 +39,20 @@ import accord.local.SequentialAsyncExecutor; import accord.local.ShardDistributor; import accord.utils.RandomSource; +import accord.utils.Reduce; +import accord.utils.async.AsyncChain; +import accord.utils.async.AsyncChains; +import accord.utils.async.AsyncResult; +import accord.utils.async.AsyncResults; import org.apache.cassandra.cache.CacheSize; import org.apache.cassandra.concurrent.ScheduledExecutors; import org.apache.cassandra.concurrent.Shutdownable; import org.apache.cassandra.config.AccordSpec.QueueShardModel; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.journal.Descriptor; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.service.accord.AccordCommandStore.DurablyAppliedTo; import org.apache.cassandra.service.accord.AccordExecutor.AccordExecutorFactory; import static org.apache.cassandra.config.AccordSpec.QueueShardModel.THREAD_PER_SHARD; @@ -46,10 +61,13 @@ import static org.apache.cassandra.service.accord.AccordExecutor.Mode.RUN_WITHOUT_LOCK; import static org.apache.cassandra.service.accord.AccordExecutor.Mode.RUN_WITH_LOCK; import static org.apache.cassandra.service.accord.AccordExecutor.constant; +import static org.apache.cassandra.service.accord.journal.ReplayMarkers.saveDirectory; import static org.apache.cassandra.utils.Clock.Global.nanoTime; public class AccordCommandStores extends CommandStores implements CacheSize, Shutdownable { + private static final Logger logger = LoggerFactory.getLogger(AccordCommandStores.class); + private final AccordExecutor[] executors; private final int mask; @@ -216,14 +234,49 @@ public boolean isTerminated() return Stream.of(executors).allMatch(AccordExecutor::isTerminated); } - @Override - public synchronized void shutdown() + public Set shutdownStores() + { + markShuttingDown(); + Set tableIds = new HashSet<>(); + List> async = new ArrayList<>(); + for (ShardHolder shard : current()) + { + AccordCommandStore commandStore = (AccordCommandStore) shard.store; + tableIds.add(commandStore.tableId()); + async.add(commandStore.shutdownAsync()); + } + if (!async.isEmpty()) + AccordService.getBlocking(AsyncResults.reduce(async, Reduce.toNull())); + return tableIds; + } + + public boolean awaitStoreTermination(long deadlineNanos) + { + for (ShardHolder shard : current()) + { + AccordCommandStore commandStore = (AccordCommandStore) shard.store; + if (!commandStore.awaitTerminationUntil(deadlineNanos)) + { + logger.warn("{} timeout awaiting durability: {}", commandStore, DurablyAppliedTo.summarise(commandStore.unsafeGetRedundantBefore())); + return false; + } + } + return true; + } + + public void shutdownExecutors() { - super.shutdown(); for (AccordExecutor executor : executors) executor.shutdown(); } + @Override + public void shutdown() + { + shutdownStores(); + shutdownExecutors(); + } + @Override public Object shutdownNow() { @@ -243,4 +296,27 @@ public boolean awaitTermination(long timeout, TimeUnit units) throws Interrupted } return true; } + + public AsyncChain saveState(Descriptor descriptor) + { + saveDirectory().createDirectoriesIfNotExists(); + List> chains = new ArrayList<>(); + for (ShardHolder shard : current()) + { + AccordCommandStore commandStore = (AccordCommandStore)shard.store; + chains.add(commandStore.saveState(descriptor)); + } + return AsyncChains.reduce(chains, Boolean::logicalAnd, true); + } + + public AsyncChain>> restoreState() + { + List>> chains = new ArrayList<>(); + for (ShardHolder shard : current()) + { + AccordCommandStore commandStore = (AccordCommandStore)shard.store; + chains.add(commandStore.restoreState()); + } + return AsyncChains.allOf(chains); + } } diff --git a/src/java/org/apache/cassandra/service/accord/AccordDataStore.java b/src/java/org/apache/cassandra/service/accord/AccordDataStore.java index a8f05f8f3cb5..7eba765c5b63 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordDataStore.java +++ b/src/java/org/apache/cassandra/service/accord/AccordDataStore.java @@ -18,6 +18,11 @@ package org.apache.cassandra.service.accord; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -26,15 +31,36 @@ import accord.local.Node; import accord.local.RedundantBefore; import accord.local.SafeCommandStore; -import accord.primitives.Range; import accord.primitives.Ranges; import accord.primitives.SyncPoint; +import accord.primitives.TxnId; +import accord.utils.Invariants; +import accord.utils.Reduce; import accord.utils.UnhandledEnum; +import accord.utils.async.AsyncResult; +import accord.utils.async.AsyncResults; +import org.apache.cassandra.concurrent.ScheduledExecutors; import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.memtable.Memtable; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.exceptions.UnknownTableException; +import org.apache.cassandra.locator.RangesAtEndpoint; +import org.apache.cassandra.repair.RepairCoordinator; +import org.apache.cassandra.repair.RepairParallelism; +import org.apache.cassandra.repair.messages.RepairOption; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.accord.AccordDurableOnFlush.ReportDurable; +import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.tcm.ClusterMetadata; + +import static accord.local.durability.DurabilityService.SyncLocal.NoLocal; +import static accord.local.durability.DurabilityService.SyncRemote.Quorum; +import static accord.primitives.Txn.Kind.ExclusiveSyncPoint; +import static accord.utils.Invariants.illegalArgument; public class AccordDataStore implements DataStore { @@ -45,75 +71,154 @@ enum FlushListenerKey { KEY } * Ensures data for the intersecting ranges is flushed to sstable before calling back with reportOnSuccess. * This is used to gate journal cleanup, since we skip the CommitLog for applying to the data table. */ - public void ensureDurable(CommandStore commandStore, Ranges ranges, RedundantBefore reportOnSuccess) + @Override + public void ensureDurable(CommandStore commandStore, Ranges ranges, RedundantBefore reportOnSuccess, int flags) { - if (commandStore.node().isReplaying()) + if (commandStore.node().isReplaying() || ranges.isEmpty()) return; - logger.debug("{} awaiting local data durability of {}", commandStore, ranges); - ColumnFamilyStore prev = null; - for (Range range : ranges) + logger.debug("{} awaiting local data durability for {}", commandStore, ranges); + ensureDurableInternal(commandStore, reportOnSuccess, flags); + } + + @Override + public void ensureDurable(CommandStore commandStore, RedundantBefore reportOnSuccess, int flags) + { + logger.debug("{} awaiting full local data durability", commandStore); + ensureDurableInternal(commandStore, reportOnSuccess, flags); + } + + private void ensureDurableInternal(CommandStore commandStore, RedundantBefore redundantBefore, int flags) + { + ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(((AccordCommandStore)commandStore).tableId()); + AccordDurableOnFlush.notifyOnDurable(cfs, commandStore, ReportDurable.of(redundantBefore, flags)); + } + + public FetchResult image(Node node, SafeCommandStore safeStore, Ranges ranges, SyncPoint syncPoint, FetchRanges callback) + { + AccordFetchCoordinator coordinator; + try + { + coordinator = new AccordFetchCoordinator(node, ranges, syncPoint, callback, safeStore.commandStore()); + } + catch (Throwable t) + { + return new FetchResult.Failure(t); + } + + coordinator.start(); + return coordinator.result(); + } + + public FetchResult sync(Node node, SafeCommandStore safeStore, Map rangesById, FetchRanges callback) + { + TableId tableId = ((AccordCommandStore)safeStore.commandStore()).tableId(); + Invariants.require(rangesById.values().stream().flatMap(Ranges::stream).allMatch(r -> tableId.equals(r.prefix()))); + Ranges ranges = rangesById.values().stream().reduce(Ranges::with).get(); + + ClusterMetadata cm = ClusterMetadata.current(); + TableMetadata tableMetadata = cm.schema.getTableMetadata(tableId); + if (tableMetadata == null) { - ColumnFamilyStore cfs; - if (prev != null && prev.metadata().id.equals(range.prefix())) cfs = prev; - else cfs = Schema.instance.getColumnFamilyStoreInstance((TableId) range.prefix()); - if (cfs == null) + Throwable fail = new UnknownTableException("Could not find tableId " + tableId + " in ClusterMetadata", tableId); + callback.fail(ranges, fail); + return new FetchResult.Failure(fail); + } + + class SyncResult extends AsyncResults.SettableResult implements FetchResult + { + @Override + public void abort(Ranges ranges) { - // TODO (expected): should we record this as durable? - continue; + throw new UnsupportedOperationException("Can not abort sync task"); } + } - while (true) - { - Memtable memtable = cfs.getCurrentMemtable(); - // If RX came when after a quiet period or if it raced with a previous memtable flush - if (memtable.isClean()) - { - AccordDurableOnFlush.notify(cfs.metadata(), commandStore, reportOnSuccess); - break; - } + // TODO (expected): add some automatic slicing of ranges and retry/back-off logic; but for now, + // since this is done at the command store level, and this is already a slice of a node, this should be fine + SyncResult syncResult = new SyncResult(); + + logger.info("Requesting quorum durability before initiating repair"); + List> syncs = new ArrayList<>(); + for (Map.Entry e : rangesById.entrySet()) + syncs.add(node.durability().sync("Sync Data Store", ExclusiveSyncPoint, e.getKey(), e.getValue(), NoLocal, Quorum, 1L, TimeUnit.HOURS)); - AccordDurableOnFlush onFlush = memtable.ensureFlushListener(FlushListenerKey.KEY, AccordDurableOnFlush::new); - if (onFlush != null && onFlush.add(commandStore.id(), reportOnSuccess)) - break; + AsyncResults.reduce(syncs, Reduce.toNull()).invoke((success, fail) -> { + if (fail != null) + { + logger.error("{} failed to achieve quorum durability before repair for rebootstrap of {}", safeStore.commandStore(), ranges); + syncResult.tryFailure(fail); + return; + } - if (cfs == prev) + RepairCoordinator coord = StorageService.instance.newRepairCoordinator(tableMetadata.keyspace, options(tableMetadata, ranges)); + coord.addProgressListener((tag, event) -> { + switch (event.getType()) { - // we must already have a successful notify, so just propagate - AccordDurableOnFlush.notify(cfs.metadata(), commandStore, reportOnSuccess); - break; + default: throw new UnhandledEnum(event.getType()); + case START: + callback.starting(ranges); + break; + case PROGRESS: + case COMPLETE: + case NOTIFICATION: + break; + case ABORT: + case ERROR: + IllegalStateException ex = new IllegalStateException(String.format("Repair failed: %s", event)); + callback.fail(ranges, ex); + syncResult.tryFailure(ex); + break; + case SUCCESS: + callback.fetched(ranges); + syncResult.trySuccess(null); + break; } - } + }); - prev = cfs; - } + ScheduledExecutors.optionalTasks.submit(coord).addCallback((s, f) -> { + if (f != null) + syncResult.tryFailure(f); + }); + }); + + return syncResult; } - @Override - public FetchResult fetch(Node node, SafeCommandStore safeStore, Ranges ranges, SyncPoint syncPoint, FetchRanges callback, FetchKind kind) + private static RepairOption options(TableMetadata tableMetadata, Ranges accordRanges) { - switch (kind) - { - default: throw new UnhandledEnum(kind); - case Image: - { - AccordFetchCoordinator coordinator; - try - { - coordinator = new AccordFetchCoordinator(node, ranges, syncPoint, callback, safeStore.commandStore()); - } - catch (Throwable t) - { - return new FetchResult.Failure(t); - } + List> ranges = new ArrayList<>(); + RangesAtEndpoint localRanges = StorageService.instance.getLocalReplicas(tableMetadata.keyspace); + // repair validation requires that we separate by local range, even if they are adjacent; + // unsure if this is important, so just splitting into ranges that are wholly contained by local ranges + accordRanges.forEach(accordRange -> { + Range range = ((TokenRange)accordRange).toKeyspaceRange(); + localRanges.ranges().forEach(localRange -> { + if (localRange.contains(range)) ranges.add(range); + else if (localRange.intersects(range)) ranges.addAll(range.intersectionWith(localRange)); + }); + }); - coordinator.start(); - return coordinator.result(); - } - case Sync: - { - throw new UnsupportedOperationException(); - } - } + Ranges matchedWithLocal = Ranges.of(ranges.stream().map(r -> TokenRange.fromKeyspaceRange(tableMetadata.id, r)).toArray(TokenRange[]::new)).mergeTouching(); + if (!matchedWithLocal.containsAll(accordRanges)) + throw illegalArgument("Local ranges %s do not fully cover requested accord ranges %s (overlap: %s)", localRanges, accordRanges, matchedWithLocal); + + return new RepairOption(RepairParallelism.PARALLEL, // parallelism + false, // primaryRange + false, // incremental + false, // trace + 5, // jobThreads + ranges, // ranges + true, // pullRepair + true, // forceRepair + PreviewKind.NONE, // previewKind + false, // optimiseStreams + true, // ignoreUnreplicatedKeyspaces + true, // repairData + false, // repairPaxos + true, // dontPurgeTombstones + false, // repairAccord + false + ); } } diff --git a/src/java/org/apache/cassandra/service/accord/AccordDurableOnFlush.java b/src/java/org/apache/cassandra/service/accord/AccordDurableOnFlush.java index 68c39284d345..a7dfd2a15ff6 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordDurableOnFlush.java +++ b/src/java/org/apache/cassandra/service/accord/AccordDurableOnFlush.java @@ -19,7 +19,7 @@ package org.apache.cassandra.service.accord; import java.util.Map; -import java.util.function.Consumer; +import java.util.function.BiConsumer; import org.agrona.collections.Int2ObjectHashMap; import org.slf4j.Logger; @@ -27,51 +27,164 @@ import accord.local.CommandStore; import accord.local.CommandStores; -import accord.local.PreLoadContext; import accord.local.RedundantBefore; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.lifecycle.View; +import org.apache.cassandra.db.memtable.Memtable; +import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableMetadata; -class AccordDurableOnFlush implements Consumer +class AccordDurableOnFlush implements BiConsumer { private static final Logger logger = LoggerFactory.getLogger(AccordDurableOnFlush.class); - private Int2ObjectHashMap commandStores = new Int2ObjectHashMap<>(); + public static class ReportDurable + { + public static final int COMMAND_STORE_FLUSH = 1; + public static final int DATA_STORE_FLUSH = 2; + + public final RedundantBefore redundantBefore; + final int flags; + + private ReportDurable(RedundantBefore redundantBefore, int flags) + { + this.redundantBefore = redundantBefore; + this.flags = flags; + } + + public boolean isDataStoreFlush() + { + return isDataStoreFlush(flags); + } + + public static boolean isDataStoreFlush(int flags) + { + return 0 != (flags & DATA_STORE_FLUSH); + } + + public boolean isCommandStoreFlush() + { + return isCommandStoreFlush(flags); + } + + public static boolean isCommandStoreFlush(int flags) + { + return 0 != (flags & COMMAND_STORE_FLUSH); + } + + public static ReportDurable of(RedundantBefore redundantBefore) + { + return of(redundantBefore, 0); + } + + public static ReportDurable of(RedundantBefore redundantBefore, int flags) + { + return new ReportDurable(redundantBefore, flags); + } + + public static ReportDurable commandStoreFlush() + { + return new ReportDurable(RedundantBefore.EMPTY, COMMAND_STORE_FLUSH); + } + + static ReportDurable merge(ReportDurable a, ReportDurable b) + { + return new ReportDurable(RedundantBefore.merge(a.redundantBefore, b.redundantBefore), a.flags | b.flags); + } + + @Override + public String toString() + { + return redundantBefore.toString(); + } + } + + private Int2ObjectHashMap commandStores = new Int2ObjectHashMap<>(); AccordDurableOnFlush() { } - synchronized boolean add(int commandStoreId, RedundantBefore reportOnFlush) + synchronized boolean add(int commandStoreId, ReportDurable reportOnFlush) { if (commandStores == null) return false; - commandStores.merge(commandStoreId, reportOnFlush, RedundantBefore::merge); + commandStores.merge(commandStoreId, reportOnFlush, ReportDurable::merge); return true; } @Override - public void accept(TableMetadata metadata) + public void accept(Long memtableId, TableMetadata metadata) { - Int2ObjectHashMap notify; + Int2ObjectHashMap notify; synchronized (this) { notify = commandStores; commandStores = null; } CommandStores commandStores = AccordService.unsafeInstance().node().commandStores(); - for (Map.Entry e : notify.entrySet()) + for (Map.Entry e : notify.entrySet()) + { + ReportDurable durable = e.getValue(); + notifyInOrder(memtableId, metadata, commandStores.forId(e.getKey()), durable); + } + } + + public static void notifyOnDurable(ColumnFamilyStore cfs, CommandStore commandStore, ReportDurable onDurable) + { + if (cfs == null) + { + // TODO (required): is this correct? Revisit when we improve DROP TABLE + notifyNow(commandStore, onDurable); + return; + } + View view = cfs.getTracker().getView(); + for (int i = view.liveMemtables.size() - 1; i >= 0 ; --i) + { + Memtable candidate = view.liveMemtables.get(i); + if (candidate.isClean()) + continue; + + AccordDurableOnFlush onFlush = candidate.ensureFlushListener(AccordDataStore.FlushListenerKey.KEY, AccordDurableOnFlush::new); + if (onFlush != null && onFlush.add(commandStore.id(), onDurable)) + return; + } + + for (int i = view.flushingMemtables.size() - 1; i >= 0 ; --i) + { + Memtable candidate = view.flushingMemtables.get(i); + AccordDurableOnFlush onFlush = candidate.ensureFlushListener(AccordDataStore.FlushListenerKey.KEY, AccordDurableOnFlush::new); + if (onFlush != null && onFlush.add(commandStore.id(), onDurable)) + return; + } + + notifyNow(commandStore, onDurable); + } + + static void notifyInOrder(long memtableId, TableMetadata metadata, CommandStore commandStore, ReportDurable report) + { + ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata.id); + if (cfs == null) { - RedundantBefore durable = e.getValue(); - notify(metadata, commandStores.forId(e.getKey()), durable); + notifyNow(commandStore, report); + return; } + View view = cfs.getTracker().getView(); + boolean notifyNow = true; + for (Memtable memtable : view.liveMemtables) + notifyNow &= memtable.getMemtableId() > memtableId; + for (Memtable memtable : view.flushingMemtables) + notifyNow &= memtable.getMemtableId() > memtableId; + if (notifyNow) notifyNow(commandStore, report); + else cfs.waitForPriorFlushes().addListener(() -> notifyNow(commandStore, report)); } - static void notify(TableMetadata metadata, CommandStore commandStore, RedundantBefore report) + static void notifyNow(CommandStore commandStore, ReportDurable report) { - logger.debug("Reporting flush of {}/{}; reporting {} to {}", metadata.id, metadata, report, commandStore); - commandStore.execute((PreLoadContext.Empty) () -> "Report Durable", safeStore -> { - safeStore.upsertRedundantBefore(report); - }); + logger.debug("{} reporting flush with {}", commandStore, report); + commandStore.execute((AccordExecutor.Unstoppable) () -> "Report Durable", safeStore -> { + safeStore.reportDurable(report.redundantBefore, report.flags); + }, commandStore.agent()); } } diff --git a/src/java/org/apache/cassandra/service/accord/AccordExecutor.java b/src/java/org/apache/cassandra/service/accord/AccordExecutor.java index 3bc5d4107d33..c53d04cd6183 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordExecutor.java +++ b/src/java/org/apache/cassandra/service/accord/AccordExecutor.java @@ -24,6 +24,7 @@ import java.util.Queue; import java.util.concurrent.Callable; import java.util.concurrent.CancellationException; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.LockSupport; @@ -86,6 +87,8 @@ import org.apache.cassandra.utils.concurrent.Condition; import org.apache.cassandra.utils.concurrent.Future; +import io.netty.util.concurrent.FastThreadLocal; + import static accord.utils.Invariants.createIllegalState; import static org.apache.cassandra.service.accord.AccordCache.CommandAdapter.COMMAND_ADAPTER; import static org.apache.cassandra.service.accord.AccordCache.CommandsForKeyAdapter.CFK_ADAPTER; @@ -116,13 +119,11 @@ public enum Mode { RUN_WITH_LOCK, RUN_WITHOUT_LOCK } // WARNING: this is a shared object, so close is NOT idempotent public static final class ExclusiveGlobalCaches extends GlobalCaches implements AutoCloseable { - final Lock lock; final AccordExecutor executor; public ExclusiveGlobalCaches(AccordExecutor executor, AccordCache global, AccordCache.Type commands, AccordCache.Type commandsForKey) { super(global, commands, commandsForKey); - this.lock = executor.lock; this.executor = executor; } @@ -130,8 +131,8 @@ public ExclusiveGlobalCaches(AccordExecutor executor, AccordCache global, Accord public void close() { executor.beforeUnlock(); - global.tryShrinkOrEvict(lock); - lock.unlock(); + global.tryShrinkOrEvict(executor.lock); + executor.unlock(); } } @@ -149,7 +150,7 @@ public GlobalCaches(AccordCache global, AccordCache.Type paranoidPriorityInversionCheck = new FastThreadLocal<>(); + + final Lock unsafeLock() + { + return lock; + } + + final void lock() + { + if (Invariants.isParanoid()) + { + Lock locked = paranoidPriorityInversionCheck.getAndSet(lock); + Invariants.require(locked == null || locked == lock, "Tried to take multiple AccordExecutor locks on same thread - this is dangerous for progress"); + } //noinspection LockAcquiredButNotSafelyReleased lock.lock(); - return caches; + } + + final void unlock() + { + lock.unlock(); + paranoidPriorityInversionCheck.set(null); + } + + final boolean tryLock() + { + boolean result = lock.tryLock(); + if (Invariants.isParanoid()) + { + if (result) + { + Lock locked = paranoidPriorityInversionCheck.getAndSet(lock); + if (locked != null && locked != lock) + { + lock.unlock(); + paranoidPriorityInversionCheck.set(locked); + Invariants.require(false, "Tried to take multiple AccordExecutor locks on same thread - this is dangerous for progress"); + return false; + } + } + else + { + Lock locked = paranoidPriorityInversionCheck.get(); + Invariants.require(locked == null || locked == lock, "Tried to take multiple AccordExecutor locks on same thread - this is dangerous for progress"); + } + } + return result; } public AccordCache cacheExclusive() @@ -289,7 +338,7 @@ public Stream active() public void waitForQuiescence() { Condition condition; - lock.lock(); + lock(); try { if (tasks == 0 && runningThreads == 0) @@ -302,7 +351,7 @@ public void waitForQuiescence() } finally { - lock.unlock(); + unlock(); } condition.awaitThrowUncheckedOnInterrupt(); } @@ -324,7 +373,7 @@ protected void notifyQuiescentExclusive() public void afterSubmittedAndConsequences(Runnable run) { - lock.lock(); + lock(); try { if (tasks == 0 && runningThreads == 0) @@ -343,7 +392,7 @@ public void afterSubmittedAndConsequences(Runnable run) } finally { - lock.unlock(); + unlock(); } } @@ -599,7 +648,7 @@ void submitExclusive(AccordTask task) enqueueLoadsExclusive(); } - void submitExclusive(Runnable runnable) + public void submitExclusive(Runnable runnable) { submitPlainExclusive(new PlainRunnable(null, runnable)); } @@ -710,7 +759,6 @@ void cancelExclusive(AccordTask task) finally { completeTaskExclusive(task); } break; - case FAILING: case RUNNING: case PERSISTING: case FINISHED: @@ -814,7 +862,7 @@ private Cancellable submit(Plain task) public void executeDirectlyWithLock(Runnable command) { - lock.lock(); + lock(); try { command.run(); @@ -822,7 +870,7 @@ public void executeDirectlyWithLock(Runnable command) finally { beforeUnlock(); - lock.unlock(); + unlock(); } } @@ -900,6 +948,7 @@ void clearRunning() public static abstract class Task extends IntrusivePriorityHeap.Node { int queuePosition; + Thread assigned; protected Task() { @@ -941,6 +990,16 @@ static abstract class SubmittableTask extends Task implements Submittable abstract void submitExclusive(AccordExecutor owner); } + // run the task even on a stopped commandStore + public interface Unstoppable extends PreLoadContext.Empty + { + } + + // run the task even on a terminated commandStore + public interface Unterminatable extends Unstoppable + { + } + static class SequentialQueueTask extends Task { private final SequentialExecutor queue; @@ -989,8 +1048,11 @@ public class SequentialExecutor extends TaskQueue implements SequentialAsy final int commandStoreId; final SequentialQueueTask selfTask; private Task task; + private Thread assigned; private volatile Thread owner, waiting; - private boolean running; + private boolean stopped; + private volatile boolean visibleStopped; + private boolean terminated; SequentialExecutor() { @@ -1007,21 +1069,46 @@ public class SequentialExecutor extends TaskQueue implements SequentialAsy void preRunTask() { Invariants.require(task != null); + assigned = Thread.currentThread(); task.preRunExclusive(); - running = true; } void runTask() { - Thread self = Thread.currentThread(); - while (!ownerUpdater.compareAndSet(this, null, self)) + outer: while (!ownerUpdater.compareAndSet(this, null, assigned)) { - waiting = self; - while (owner != null) + waiting = assigned; + while (true) + { + Thread owner = this.owner; + if (owner == assigned) break outer; + if (owner == null) continue outer; LockSupport.park(); - waiting = null; + } + } + waiting = null; + + try + { + if (stopped && reject(task)) + task.fail(new RejectedExecutionException(commandStoreId + " is terminated. Cannot execute " + ((AccordTask) task).preLoadContext())); + else + task.runInternal(); } - task.runInternal(); + finally + { + owner = null; + } + } + + private boolean reject(Task task) + { + if (!(task instanceof AccordTask)) + return true; + + PreLoadContext context = ((AccordTask) task).preLoadContext(); + + return !(terminated ? (context instanceof Unterminatable) : (context instanceof Unstoppable)); } void failTask(Throwable t) @@ -1034,9 +1121,10 @@ void cleanupTask() try { task.cleanupExclusive(); } finally { - owner = null; - running = false; + assigned = null; task = super.poll(); + + // it should only be possible for this method to be invoked once we're on the running queue AccordExecutor.this.running.remove(selfTask); if (task != null) { @@ -1052,12 +1140,12 @@ protected void append(Task newTask) { if (task != null) { - Invariants.require(running || waitingToRun.contains(selfTask)); + Invariants.require(assigned != null || waitingToRun.contains(selfTask)); super.append(newTask); } else { - Invariants.require(!running && isEmpty()); + Invariants.require(assigned == null && isEmpty()); task = newTask; selfTask.queuePosition = newTask.queuePosition; waitingToRun.append(selfTask); @@ -1067,37 +1155,47 @@ protected void append(Task newTask) @Override protected void remove(Task remove) { - Invariants.require(remove != null); - if (remove != task) - { - super.remove(remove); - } - else if (!running) - { - // cannot overwrite task while it is being executed - this cannot happen for AccordTask - // but can for other tasks that don't track their own state + if (remove == task) removeCurrentTask(remove); + else super.remove(remove); + } - task = super.poll(); - if (waitingToRun.contains(selfTask)) + @Override + protected boolean removeIfContains(Task remove) + { + if (remove == task) return removeCurrentTask(remove); + else return super.removeIfContains(remove); + } + + private boolean removeCurrentTask(Node remove) + { + if (assigned != null) + return false; + + Invariants.require(remove == task); + // cannot overwrite task while it is being executed - this cannot happen for AccordTask + // but can for other tasks that don't track their own state + + task = super.poll(); + if (waitingToRun.contains(selfTask)) + { + if (task == null) waitingToRun.remove(selfTask); + else { - if (task == null) waitingToRun.remove(selfTask); - else - { - selfTask.queuePosition = task.queuePosition; - waitingToRun.update(selfTask); - } + selfTask.queuePosition = task.queuePosition; + waitingToRun.update(selfTask); } - else + } + else + { + Invariants.expect(false, "%s should have been queued to run as it had the task %s pending, that has now been cancelled", this, remove); + if (task != null) { - Invariants.expect(false, "%s should have been queued to run as it had the task %s pending, that has now been cancelled", this, remove); - if (task != null) - { - selfTask.queuePosition = task.queuePosition; - waitingToRun.append(selfTask); - } + selfTask.queuePosition = task.queuePosition; + waitingToRun.append(selfTask); } } - Invariants.require(task == null || running || waitingToRun.contains(selfTask)); + Invariants.require(task == null || waitingToRun.contains(selfTask)); + return true; } public boolean inExecutor() @@ -1105,6 +1203,24 @@ public boolean inExecutor() return owner == Thread.currentThread(); } + public boolean stopped() + { + return visibleStopped; + } + + void stop() + { + Invariants.require(inExecutor()); + this.stopped = true; + this.visibleStopped = true; + } + + void terminate() + { + Invariants.require(inExecutor()); + this.visibleStopped = this.terminated = this.stopped = true; + } + @Override protected Task poll() { @@ -1120,7 +1236,7 @@ protected Task peek() @Override protected boolean contains(Task contains) { - return super.contains(contains) || (task == contains && !running); + return super.contains(contains) || (task == contains && assigned == null); } @Override @@ -1205,7 +1321,10 @@ public boolean tryExecuteImmediately(Runnable run) { this.owner = null; if (waiting != null) + { LockSupport.unpark(waiting); + ownerUpdater.compareAndSet(this, null, waiting); + } } } return true; @@ -1259,6 +1378,12 @@ protected void remove(T remove) super.remove(remove); } + @Override + protected boolean removeIfContains(T node) + { + return super.removeIfContains(node); + } + protected boolean contains(T contains) { return super.contains(contains); @@ -1607,8 +1732,8 @@ protected void runInternal() } catch (Throwable t) { + // shouldn't throw exceptions agent.onException(t); - return; } } @@ -1741,7 +1866,7 @@ public int compareTo(TaskInfo that) public List taskSnapshot() { List result = new ArrayList<>(); - lock.lock(); + lock(); try { addToSnapshot(result, waitingToLoad, TaskInfo.Status.WAITING_TO_LOAD, TaskInfo.Status.WAITING_TO_LOAD); @@ -1753,7 +1878,7 @@ public List taskSnapshot() } finally { - lock.unlock(); + unlock(); } result.sort(TaskInfo::compareTo); return result; diff --git a/src/java/org/apache/cassandra/service/accord/AccordExecutorAbstractLockLoop.java b/src/java/org/apache/cassandra/service/accord/AccordExecutorAbstractLockLoop.java index 8d71ab61a57b..768ddebc7578 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordExecutorAbstractLockLoop.java +++ b/src/java/org/apache/cassandra/service/accord/AccordExecutorAbstractLockLoop.java @@ -84,14 +84,14 @@ public boolean hasTasks() if (tasks > 0 || !submitted.isEmpty() || runningThreads > 0) return true; - lock.lock(); + lock(); try { return tasks > 0 || !submitted.isEmpty() || runningThreads > 0; } finally { - lock.unlock(); + unlock(); } } @@ -157,7 +157,7 @@ public void run() Task task; while (true) { - lock.lock(); + lock(); try { resumeExclusive(); @@ -210,7 +210,7 @@ public void run() } finally { - lock.unlock(); + unlock(); } } } @@ -227,7 +227,7 @@ public void run() Task task = null; while (true) { - lock.lock(); + lock(); try { if (task != null) @@ -288,7 +288,7 @@ public void run() } finally { - lock.unlock(); + unlock(); } try diff --git a/src/java/org/apache/cassandra/service/accord/AccordExecutorAbstractSemiSyncSubmit.java b/src/java/org/apache/cassandra/service/accord/AccordExecutorAbstractSemiSyncSubmit.java index 428270fc6bf5..ccba5f7085be 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordExecutorAbstractSemiSyncSubmit.java +++ b/src/java/org/apache/cassandra/service/accord/AccordExecutorAbstractSemiSyncSubmit.java @@ -35,7 +35,7 @@ abstract class AccordExecutorAbstractSemiSyncSubmit extends AccordExecutorAbstra void submitExternal(QuintConsumer sync, QuadFunction async, P1s p1s, P1a p1a, P2 p2, P3 p3, P4 p4) { - if (!lock.tryLock()) + if (!tryLock()) { submitted.push(async.apply(p1a, p2, p3, p4)); notifyWork(); @@ -48,7 +48,7 @@ void submitExternal(QuintConsumer keyCRCBytes = ThreadLocal.withInitial(() -> new byte[JournalKeySupport.TOTAL_SIZE]); - - @VisibleForTesting - protected final Journal journal; - @VisibleForTesting - protected final AccordJournalTable journalTable; - private final Params params; - Node node; - - enum Status { INITIALIZED, STARTING, REPLAY, STARTED, TERMINATING, TERMINATED } - private volatile Status status = Status.INITIALIZED; - - public AccordJournal(Params params) - { - this(params, new File(DatabaseDescriptor.getAccordJournalDirectory()), Keyspace.open(AccordKeyspace.metadata().name).getColumnFamilyStore(AccordKeyspace.JOURNAL)); - } - - @VisibleForTesting - public AccordJournal(Params params, File directory, ColumnFamilyStore cfs) - { - Version userVersion = Version.fromVersion(params.userVersion()); - this.journal = new Journal<>("AccordJournal", directory, params, JournalKey.SUPPORT, - // In Accord, we are using streaming serialization, i.e. Reader/Writer interfaces instead of materializing objects - new ValueSerializer<>() - { - @Override - public void serialize(JournalKey key, Object value, DataOutputPlus out, int userVersion) - { - throw new UnsupportedOperationException(); - } - - @Override - public Object deserialize(JournalKey key, DataInputPlus in, int userVersion) - { - throw new UnsupportedOperationException(); - } - }, - compactor(cfs, userVersion), - cfs.readOrdering); - this.journalTable = new AccordJournalTable<>(journal, JournalKey.SUPPORT, cfs, userVersion); - this.params = params; - } - - protected SegmentCompactor compactor(ColumnFamilyStore cfs, Version userVersion) - { - return new AccordSegmentCompactor<>(userVersion, cfs) { - @Nullable - @Override - public Collection> compact(Collection> staticSegments) - { - if (journalTable == null) - throw new IllegalStateException("Unsafe access to AccordJournal during ; journalTable was touched before it was published"); - Collection> result = super.compact(staticSegments); - journalTable.safeNotify(index -> index.remove(staticSegments)); - return result; - } - }; - } - - @VisibleForTesting - public int inMemorySize() - { - return journal.currentActiveSegment().index().size(); - } - - public void start(Node node) - { - Invariants.require(status == Status.INITIALIZED); - this.node = node; - status = Status.STARTING; - // start table first to scrub directories before compactor starts - journalTable.start(); - journal.start(); - } - - public boolean started() - { - return status == Status.STARTED; - } - - public Params configuration() - { - return params; - } - - public Compactor compactor() - { - return journal.compactor(); - } - - @Override - public boolean isTerminated() - { - return status == Status.TERMINATED; - } - - @Override - public void shutdown() - { - Invariants.require(status == Status.REPLAY || status == Status.STARTED, "%s", status); - status = Status.TERMINATING; - journal.shutdown(); - status = Status.TERMINATED; - } - - @Override - public Object shutdownNow() - { - shutdown(); - return null; - } - - @Override - public boolean awaitTermination(long timeout, TimeUnit units) throws InterruptedException - { - try - { - ExecutorUtils.awaitTermination(timeout, units, Collections.singletonList(journal)); - return true; - } - catch (TimeoutException e) - { - return false; - } - } - - @Override - public Command loadCommand(int commandStoreId, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore) - { - Builder builder = load(commandStoreId, txnId); - builder.maybeCleanup(true, FULL, redundantBefore, durableBefore); - return builder.construct(redundantBefore); - } - - public static class DebugEntry implements Supplier - { - public final long segment; - public final int position; - public final Builder builder; - - public DebugEntry(long segment, int position, Builder builder) - { - this.segment = segment; - this.position = position; - this.builder = builder; - } - - @Override - public CommandChange.Builder get() - { - return builder; - } - } - - @Override - public List debugCommand(int commandStoreId, TxnId txnId) - { - JournalKey key = new JournalKey(txnId, COMMAND_DIFF, commandStoreId); - List result = new ArrayList<>(); - journalTable.readAll(key, (long segment, int position, JournalKey k, ByteBuffer buffer, int userVersion) -> { - Builder builder = new Builder(txnId); - new AccordJournalTable.RecordConsumerAdapter<>(builder::deserializeNext).accept(segment, position, k, buffer, userVersion); - result.add(new DebugEntry(segment, position, builder)); - }); - return result; - } - - // applies cleanup and returns null if no command should be returned - public static Builder cleanupAndFilter(Builder builder, RedundantBefore redundantBefore, DurableBefore durableBefore) - { - if (builder.isEmpty()) - return null; - - Cleanup cleanup = builder.shouldCleanup(FULL, redundantBefore, durableBefore); - switch (cleanup) - { - case VESTIGIAL: - case EXPUNGE: - case ERASE: - return null; - } - Invariants.require(builder.saveStatus() != null, "No saveSatus loaded, but next was called and cleanup was not: %s", builder); - return builder; - } - - @Override - public Command.Minimal loadMinimal(int commandStoreId, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore) - { - Builder builder = cleanupAndFilter(loadDiffs(commandStoreId, txnId, MINIMAL), redundantBefore, durableBefore); - return builder == null ? null : builder.asMinimal(); - } - - @Override - public Command.MinimalWithDeps loadMinimalWithDeps(int commandStoreId, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore) - { - Builder builder = cleanupAndFilter(loadDiffs(commandStoreId, txnId, MINIMAL_WITH_DEPS), redundantBefore, durableBefore); - return builder == null ? null : builder.asMinimalWithDeps(); - } - - @Override - public RedundantBefore loadRedundantBefore(int commandStoreId) - { - IdentityAccumulator accumulator = readLast(new JournalKey(TxnId.NONE, JournalKey.Type.REDUNDANT_BEFORE, commandStoreId)); - return accumulator.get(); - } - - @Override - public NavigableMap loadBootstrapBeganAt(int commandStoreId) - { - IdentityAccumulator> accumulator = readLast(new JournalKey(TxnId.NONE, JournalKey.Type.BOOTSTRAP_BEGAN_AT, commandStoreId)); - return accumulator.get(); - } - - @Override - public NavigableMap loadSafeToRead(int commandStoreId) - { - IdentityAccumulator> accumulator = readLast(new JournalKey(TxnId.NONE, JournalKey.Type.SAFE_TO_READ, commandStoreId)); - return accumulator.get(); - } - - @Override - public CommandStores.RangesForEpoch loadRangesForEpoch(int commandStoreId) - { - IdentityAccumulator accumulator = readLast(new JournalKey(TxnId.NONE, JournalKey.Type.RANGES_FOR_EPOCH, commandStoreId)); - return accumulator.get(); - } - - @Override - public void saveCommand(int commandStoreId, CommandUpdate update, @Nullable Runnable onFlush) - { - Writer diff = Writer.make(update.before, update.after); - if (diff == null) - { - if (onFlush != null) - onFlush.run(); - return; - } - - JournalKey key = new JournalKey(update.txnId, COMMAND_DIFF, commandStoreId); - RecordPointer pointer = journal.asyncWrite(key, diff); - if (journalTable.shouldIndex(key) - && diff.hasParticipants() - && diff.after.route() != null) - journal.onDurable(pointer, () -> - journalTable.safeNotify(index -> - index.update(pointer.segment, key.commandStoreId, key.id, diff.after.route()))); - if (onFlush != null) - journal.onDurable(pointer, onFlush); - } - - @Override - public List replayTopologies() - { - List images = new ArrayList<>(); - try (CloseableIterator iter = new CloseableIterator<>() - { - final CloseableIterator> iter = journalTable.keyIterator(topologyUpdateKey(0L), - topologyUpdateKey(Timestamp.MAX_EPOCH), - true); - TopologyImage prev = null; - - @Override - public boolean hasNext() - { - return iter.hasNext(); - } - - @Override - public TopologyUpdate next() - { - Journal.KeyRefs ref = iter.next(); - Accumulator reader = readAll(ref.key()); - if (reader.read().kind() == Kind.Repeat) - { - if (prev == null) - { - logger.error("Encountered TopologyImage Repeat record for epoch {}, but no prior image record was found", ref.key().id.epoch()); - return null; - } - prev = reader.read().asImage(Invariants.nonNull(prev.getUpdate())); - } - else prev = reader.read(); - - return new TopologyUpdate(prev.getUpdate().commandStores, - prev.getUpdate().global); - } - - @Override - public void close() - { - iter.close(); - } - }) - { - TopologyUpdate prev = null; - while (iter.hasNext()) - { - TopologyUpdate next = iter.next(); - if (next == null) - continue; - - Invariants.require(prev == null || next.global.epoch() > prev.global.epoch()); - // Due to partial compaction, we can clean up only some of the old epochs, creating gaps. We skip these epochs here. - if (prev != null && next.global.epoch() > prev.global.epoch() + 1) - images.clear(); - - images.add(next); - prev = next; - } - } - return images; - } - - @Override - public void saveTopology(TopologyUpdate topologyUpdate, Runnable onFlush) - { - RecordPointer pointer = appendInternal(topologyUpdateKey(topologyUpdate.global.epoch()), - newTopology(topologyUpdate)); - if (onFlush != null) - journal.onDurable(pointer, onFlush); - } - - private static JournalKey topologyUpdateKey(long epoch) - { - return new JournalKey(TxnId.fromValues(epoch, 0L, Node.Id.NONE), - JournalKey.Type.TOPOLOGY_UPDATE, Integer.MAX_VALUE); - } - - private static final JournalKey DURABLE_BEFORE_KEY = new JournalKey(TxnId.NONE, JournalKey.Type.DURABLE_BEFORE, 0); - - @Override - public PersistentField.Persister durableBeforePersister() - { - return new PersistentField.Persister<>() - { - @Override - public AsyncResult persist(DurableBefore addValue, DurableBefore newValue) - { - AsyncResult.Settable result = AsyncResults.settable(); - RecordPointer pointer = appendInternal(DURABLE_BEFORE_KEY, addValue); - // TODO (required): what happens on failure? - journal.onDurable(pointer, () -> result.setSuccess(null)); - return result; - } - - @Override - public DurableBefore load() - { - DurableBeforeAccumulator accumulator = readAll(DURABLE_BEFORE_KEY); - return accumulator.get(); - } - }; - } - - @Override - public void saveStoreState(int commandStoreId, FieldUpdates fieldUpdates, Runnable onFlush) - { - RecordPointer pointer = null; - // TODO: avoid allocating keys - if (fieldUpdates.newRedundantBefore != null) - pointer = appendInternal(new JournalKey(TxnId.NONE, JournalKey.Type.REDUNDANT_BEFORE, commandStoreId), fieldUpdates.newRedundantBefore); - if (fieldUpdates.newBootstrapBeganAt != null) - pointer = appendInternal(new JournalKey(TxnId.NONE, JournalKey.Type.BOOTSTRAP_BEGAN_AT, commandStoreId), fieldUpdates.newBootstrapBeganAt); - if (fieldUpdates.newSafeToRead != null) - pointer = appendInternal(new JournalKey(TxnId.NONE, JournalKey.Type.SAFE_TO_READ, commandStoreId), fieldUpdates.newSafeToRead); - if (fieldUpdates.newRangesForEpoch != null) - pointer = appendInternal(new JournalKey(TxnId.NONE, JournalKey.Type.RANGES_FOR_EPOCH, commandStoreId), fieldUpdates.newRangesForEpoch); - - if (onFlush == null) - return; - - if (pointer != null) - journal.onDurable(pointer, onFlush); - else - onFlush.run(); - } - - private Builder loadDiffs(int commandStoreId, TxnId txnId, Load load) - { - JournalKey key = new JournalKey(txnId, COMMAND_DIFF, commandStoreId); - Builder builder = new Builder(txnId, load); - journalTable.readAll(key, builder::deserializeNext); - return builder; - } - - @VisibleForTesting - public Builder load(int commandStoreId, TxnId txnId) - { - return loadDiffs(commandStoreId, txnId, Load.ALL); - } - - public BUILDER readAll(JournalKey key) - { - BUILDER builder = (BUILDER) key.type.serializer.mergerFor(); - builder.reset(key); - // TODO (expected): this can be further improved to avoid allocating lambdas - AccordJournalValueSerializers.FlyweightSerializer serializer = (AccordJournalValueSerializers.FlyweightSerializer) key.type.serializer; - // TODO (expected): for those where we store an image, read only the first entry we find in DESC order - journalTable.readAll(key, (in, userVersion) -> serializer.deserialize(key, builder, in, userVersion)); - return builder; - } - - public BUILDER readLast(JournalKey key) - { - BUILDER builder = (BUILDER) key.type.serializer.mergerFor(); - builder.reset(key); - // TODO (expected): this can be further improved to avoid allocating lambdas - AccordJournalValueSerializers.FlyweightSerializer serializer = (AccordJournalValueSerializers.FlyweightSerializer) key.type.serializer; - journalTable.readLast(key, (in, userVersion) -> serializer.deserialize(key, builder, in, userVersion)); - return builder; - } - - public void forEachEntry(JournalKey key, AccordJournalTable.Reader reader) - { - journalTable.readAll(key, reader); - } - - private RecordPointer appendInternal(JournalKey key, T write) - { - AccordJournalValueSerializers.FlyweightSerializer serializer = (AccordJournalValueSerializers.FlyweightSerializer) key.type.serializer; - return journal.asyncWrite(key, (out, userVersion) -> serializer.serialize(key, write, out, Version.fromVersion(userVersion))); - } - - @VisibleForTesting - public void closeCurrentSegmentForTestingIfNonEmpty() - { - journal.closeCurrentSegmentForTestingIfNonEmpty(); - } - - public void sanityCheck(int commandStoreId, RedundantBefore redundantBefore, Command orig) - { - Builder builder = load(commandStoreId, orig.txnId()); - builder.forceResult(orig.result()); - // We can only use strict equality if we supply result. - Command reconstructed = builder.construct(redundantBefore); - Invariants.require(orig.equals(reconstructed), - '\n' + - "Original: %s\n" + - "Reconstructed: %s\n" + - "Diffs: %s", orig, reconstructed, builder); - } - - @VisibleForTesting - public void truncateForTesting() - { - journal.truncateForTesting(); - journalTable.safeNotify(JournalSegmentRangeSearcher::truncateForTesting); - } - - @VisibleForTesting - public void runCompactorForTesting() - { - journal.runCompactorForTesting(); - } - - @Override - public void purge(CommandStores commandStores, EpochSupplier minEpoch) - { - journal.closeCurrentSegmentForTestingIfNonEmpty(); - journal.runCompactorForTesting(); - journalTable.forceCompaction(); - } - - public void forEach(Consumer consumer, boolean includeActive) - { - forEach(consumer, null, null, includeActive); - } - - public void forEach(Consumer consumer, @Nullable JournalKey min, @Nullable JournalKey max, boolean includeActive) - { - try (CloseableIterator> iter = journalTable.keyIterator(min, max, includeActive)) - { - while (iter.hasNext()) - { - Journal.KeyRefs ref = iter.next(); - consumer.accept(ref.key()); - } - } - } - - @SuppressWarnings("unchecked") - @Override - public boolean replay(CommandStores commandStores) - { - // TODO (expected): make the parallelisms configurable - // Replay is performed in parallel, where at most X commands can be in flight, accross at most Y commands stores. - // That is, you can limit replay parallelism to 1 command store at a time, but load multiple commands within that data store, - // _or_ have multiple commands being loaded accross multiple data stores. - final Semaphore commandParallelism = Semaphore.newSemaphore(getAvailableProcessors()); - final int commandStoreParallelism = Math.max(Math.max(1, Math.min(getAvailableProcessors(), 4)), getAvailableProcessors() / 4); - final AtomicBoolean abort = new AtomicBoolean(); - // TODO (expected): balance work submission by AccordExecutor - final IntArrayList activeCommandStoreIds = new IntArrayList(); - final ReplayQueue pendingCommandStores = new ReplayQueue(commandStores.all()); - - class ReplayStream implements Closeable - { - final CommandStore commandStore; - final AbstractReplayer replayer; - final CloseableIterator> iter; - JournalKey prev; - - public ReplayStream(CommandStore commandStore) - { - this.commandStore = commandStore; - this.replayer = (AbstractReplayer) commandStore.replayer(); - // Keys in the index are sorted by command store id, so index iteration will be sequential - this.iter = journalTable.keyIterator(new JournalKey(replayer.minReplay.withoutNonIdentityFlags(), COMMAND_DIFF, commandStore.id()), new JournalKey(TxnId.MAX.withoutNonIdentityFlags(), COMMAND_DIFF, commandStore.id()), false); - } - - boolean replay() - { - logger.info("Beginning replay of {} with min={}, {}", commandStore, replayer.minReplay, - replayer.redundantBefore.map(b -> b == null ? null : b.maxBoundBoth(LOCALLY_DURABLE_TO_DATA_STORE, LOCALLY_DURABLE_TO_COMMAND_STORE), TxnId[]::new)); - - JournalKey key; - long[] segments; - while (true) - { - if (!iter.hasNext()) - { - logger.info("Completed replay of {}", commandStore); - return false; - } - - Journal.KeyRefs ref = iter.next(); - if (ref.key().type != COMMAND_DIFF) - continue; - - key = ref.key(); - segments = journalTable.shouldIndex(key) ? ref.copyOfSegments() : null; - break; - } - - TxnId txnId = key.id; - Invariants.require(prev == null || - key.commandStoreId != prev.commandStoreId || - key.id.compareTo(prev.id) != 0, - "duplicate key detected %s == %s", key, prev); - prev = key; - commandParallelism.acquireThrowUncheckedOnInterrupt(1); - replayer.replay(txnId) - .map(route -> { - if (segments != null && route != null) - { - for (long segment : segments) - journalTable.safeNotify(index -> index.update(segment, key.commandStoreId, txnId, (Route) route)); - } - return null; - }).begin((success, fail) -> { - commandParallelism.release(1); - if (fail != null && !journal.handleError("Could not replay command " + txnId, fail)) - abort.set(true); - }); - - return true; - } - - @Override - public void close() - { - iter.close(); - } - } - - // Replay streams by command store id, can hold at most commandStoreParallelism items - final Int2ObjectHashMap replayStreams = new Int2ObjectHashMap<>(); - try - { - // index of the store we're currently pulling from in the activeCommandStoreIds collection - int cur = 0; - while (!abort.get()) - { - if (cur == activeCommandStoreIds.size()) - { - if (activeCommandStoreIds.size() < commandStoreParallelism && !pendingCommandStores.isEmpty()) - { - CommandStore next = pendingCommandStores.next(); - int id = next.id(); - activeCommandStoreIds.add(id); - replayStreams.put(id, new ReplayStream(next)); - } - else if (activeCommandStoreIds.isEmpty()) break; - else cur = 0; - } - - int id = activeCommandStoreIds.get(cur); - ReplayStream replayStream = replayStreams.get(id); - while (!replayStream.replay()) - { - // Replay complete for this command store; close and replace - replayStreams.remove(id).close(); - if (pendingCommandStores.isEmpty()) - { - // no more pending to submit; remove and continue with the next remaining (if any) - activeCommandStoreIds.removeAt(cur); - if (cur == activeCommandStoreIds.size()) - --cur; - if (cur < 0) - break; - id = activeCommandStoreIds.get(cur); - } - else - { - // replace it with a pending command store, and continue processing - CommandStore next = pendingCommandStores.next(streamId(replayStream.commandStore)); - id = next.id(); - activeCommandStoreIds.set(cur, id); - replayStreams.put(id, new ReplayStream(next)); - } - - replayStream = replayStreams.get(id); - } - - ++cur; - } - return true; - } - catch (Throwable t) - { - try { FileUtils.close(replayStreams.values()); } - catch (Throwable t2) { t.addSuppressed(t2); } - throw t; - } - } - - static class ReplayQueue - { - final Int2ObjectHashMap> byExecutor = new Int2ObjectHashMap<>(); - final Deque nextId = new ArrayDeque<>(); - - ReplayQueue(CommandStore[] commandStores) - { - for (CommandStore commandStore : commandStores) - { - byExecutor.computeIfAbsent(streamId(commandStore), ignore -> new ArrayDeque<>()) - .add(commandStore); - } - nextId.addAll(byExecutor.keySet()); - } - - boolean isEmpty() - { - return byExecutor.isEmpty(); - } - - CommandStore next() - { - while (true) - { - if (byExecutor.isEmpty()) - return null; - - Integer id = nextId.poll(); - if (id == null) - { - nextId.addAll(byExecutor.keySet()); - id = nextId.poll(); - } - - Queue queue = byExecutor.get(id); - if (queue != null) - { - CommandStore next = queue.poll(); - if (queue.isEmpty()) - byExecutor.remove(id); - if (next != null) - return next; - } - } - } - - CommandStore next(int streamId) - { - Queue queue = byExecutor.get(streamId); - if (queue == null) - return next(); - - CommandStore next = queue.poll(); - if (queue.isEmpty()) - byExecutor.remove(streamId); - - return next; - } - } - - private static int streamId(CommandStore commandStore) - { - return commandStore instanceof AccordCommandStore ? ((AccordCommandStore) commandStore).executor().executorId() : 1; - } - - public static @Nullable ByteBuffer asSerializedChange(Command before, Command after, Version userVersion) throws IOException - { - // TODO (expected): reusable buffer to build, or pre-size - try (DataOutputBuffer out = new DataOutputBuffer()) - { - Writer writer = Writer.make(before, after); - if (writer == null) - return null; - - writer.write(out, userVersion); - return out.asNewBuffer(); - } - } - - @VisibleForTesting - public void unsafeSetStarted() - { - status = Status.STARTED; - } - - @VisibleForTesting - public Journal unsafeGetJournal() - { - return journal; - } - - @Override - public JournalRangeSearcher rangeSearcher() - { - return journalTable.rangeSearcher(); - } - - public static class Writer implements Journal.Writer - { - private final Command after; - private final int flags; - - private Writer(Command after, int flags) - { - this.after = after; - this.flags = flags; - } - - public static Writer make(Command before, Command after) - { - if (before == after - || after == null - || after.saveStatus() == SaveStatus.Uninitialised) - return null; - - int flags = validateFlags(getFlags(before, after)); - if (!anyFieldChanged(flags)) - return null; - - return new Writer(after, flags); - } - - @Override - public void write(DataOutputPlus out, int userVersion) throws IOException - { - write(out, Version.fromVersion(userVersion)); - } - - public void write(DataOutputPlus out, Version userVersion) throws IOException - { - serialize(after, flags, out, userVersion); - } - - private static void serialize(Command command, int flags, DataOutputPlus out, Version userVersion) throws IOException - { - Invariants.require(flags != 0); - out.writeInt(flags); - - int iterable = toIterableSetFields(flags); - while (iterable != 0) - { - Field field = nextSetField(iterable); - if (isNull(field, flags)) - { - iterable = unsetIterable(field, iterable); - continue; - } - - switch (field) - { - case EXECUTE_AT: - ExecuteAtSerializer.serialize(command.txnId(), command.executeAt(), out); - break; - case EXECUTES_AT_LEAST: - ExecuteAtSerializer.serialize(command.executesAtLeast(), out); - break; - case MIN_UNIQUE_HLC: - Invariants.require(command.waitingOn().minUniqueHlc() != 0); - out.writeUnsignedVInt(command.waitingOn().minUniqueHlc()); - break; - case SAVE_STATUS: - out.writeByte(command.saveStatus().ordinal()); - break; - case DURABILITY: - out.writeByte(command.durability().encoded()); - break; - case ACCEPTED: - CommandSerializers.ballot.serialize(command.acceptedOrCommitted(), out); - break; - case PROMISED: - CommandSerializers.ballot.serialize(command.promised(), out); - break; - case PARTICIPANTS: - CommandSerializers.participants.serialize(command.participants(), out); - break; - case PARTIAL_TXN: - CommandSerializers.partialTxn.serialize(command.partialTxn(), out, userVersion); - break; - case PARTIAL_DEPS: - DepsSerializers.partialDepsById.serialize(command.partialDeps(), out); - break; - case WAITING_ON: - Command.WaitingOn waitingOn = command.waitingOn(); - WaitingOnSerializer.serializeBitSetsOnly(command.txnId(), waitingOn, out); - break; - case WRITES: - CommandSerializers.writes.serialize(command.writes(), out, userVersion); - break; - case RESULT: - ResultSerializers.result.serialize(command.result(), out); - break; - case CLEANUP: - Cleanup cleanup; - switch (command.saveStatus()) - { - default: throw new UnhandledEnum(command.saveStatus()); - case Erased: cleanup = Cleanup.ERASE; break; - case Invalidated: cleanup = Cleanup.INVALIDATE; break; - } - out.writeByte(cleanup.ordinal()); - break; - } - - iterable = unsetIterable(field, iterable); - } - } - - private boolean hasField(Field fields) - { - return !isNull(fields, flags); - } - - public boolean hasParticipants() - { - return hasField(Field.PARTICIPANTS); - } - - @Override - public String toString() - { - return after.saveStatus() + " " + describeFlags(flags); - } - } - - public static class Builder extends CommandChange.Builder implements FlyweightImage - { - private final boolean deserializeDeps; - - public Builder() - { - this(Load.ALL); - } - - public Builder(Load load) - { - this(null, load); - } - - public Builder(TxnId txnId) - { - this(txnId, Load.ALL); - } - - public Builder(TxnId txnId, Load load) - { - super(txnId, load); - deserializeDeps = load == ALL; - } - - @Override - public PartialDeps partialDeps() - { - if (partialDeps instanceof ByteBuffer) - { - try - { - partialDeps = DepsSerializers.partialDepsById.deserialize((ByteBuffer) partialDeps); - } - catch (IOException e) - { - throw new IllegalStateException("Failed to materialise partially deserialised deps", e); - } - } - return (PartialDeps) partialDeps; - } - - public void reset(JournalKey key) - { - reset(key.id); - } - - public ByteBuffer asByteBuffer(Version userVersion) throws IOException - { - try (DataOutputBuffer out = new DataOutputBuffer()) - { - serialize(out, userVersion); - return out.asNewBuffer(); - } - } - - public void serialize(DataOutputPlus out, Version userVersion) throws IOException - { - Invariants.require(mask == 0); - Invariants.require(flags != 0); - - int flags = validateFlags(this.flags); - serialize(flags, out, userVersion); - } - - private void serialize(int flags, DataOutputPlus out, Version userVersion) throws IOException - { - Invariants.require(flags != 0); - out.writeInt(flags); - - int iterable = toIterableNonNullFields(flags); - for (Field field = nextSetField(iterable) ; field != null; iterable = unsetIterable(field, iterable), field = nextSetField(iterable)) - { - switch (field) - { - default: throw new UnhandledEnum(field); - case CLEANUP: - out.writeByte(cleanup.ordinal()); - break; - case EXECUTE_AT: - Invariants.require(txnId != null, "%s", this); - Invariants.require(executeAt != null, "%s", this); - ExecuteAtSerializer.serialize(txnId, executeAt, out); - break; - case EXECUTES_AT_LEAST: - Invariants.require(executesAtLeast != null); - ExecuteAtSerializer.serialize(executesAtLeast, out); - break; - case MIN_UNIQUE_HLC: - Invariants.require(minUniqueHlc != 0, "%s", this); - out.writeUnsignedVInt(minUniqueHlc); - break; - case SAVE_STATUS: - Invariants.require(saveStatus != null, "%s", this); - out.writeByte(saveStatus.ordinal()); - break; - case DURABILITY: - Invariants.require(durability != null, "%s", this); - out.writeByte(durability.encoded()); - break; - case ACCEPTED: - Invariants.require(acceptedOrCommitted != null, "%s", this); - CommandSerializers.ballot.serialize(acceptedOrCommitted, out); - break; - case PROMISED: - Invariants.require(promised != null, "%s", this); - CommandSerializers.ballot.serialize(promised, out); - break; - case PARTICIPANTS: - Invariants.require(participants != null, "%s", this); - CommandSerializers.participants.serialize(participants, out); - break; - case PARTIAL_TXN: - Invariants.require(partialTxn != null, "%s", this); - CommandSerializers.partialTxn.serialize(partialTxn, out, userVersion); - break; - case PARTIAL_DEPS: - Invariants.require(partialDeps != null, "%s", this); - if (partialDeps instanceof ByteBuffer) out.write(((ByteBuffer) partialDeps).duplicate()); - else DepsSerializers.partialDepsById.serialize((PartialDeps) partialDeps, out); - break; - case WAITING_ON: - Invariants.require(waitingOn != null, "%s", this); - ((WaitingOnSerializer.WaitingOnBitSetsAndLength)waitingOn).reserialize(out); - break; - case WRITES: - Invariants.require(writes != null, "%s", this); - CommandSerializers.writes.serialize(writes, out, userVersion); - break; - case RESULT: - Invariants.require(result != null, "%s", this); - ResultSerializers.result.serialize(result, out); - break; - } - } - } - - public void deserializeNext(DataInputPlus in, Version userVersion) throws IOException - { - Invariants.require(txnId != null); - int readFlags = in.readInt(); - Invariants.require(readFlags != 0); - hasUpdate = true; - count++; - - // batch-apply any new nulls - setNulls(false, readFlags); - // iterator sets low 16 bits; low readFlag bits are nulls, so masking with ~readFlags restricts to non-null changed fields - int iterable = toIterableSetFields(readFlags) & ~readFlags; - for (Field field = nextSetField(iterable) ; field != null; field = nextSetField(iterable = unsetIterable(field, iterable))) - { - // Since we are iterating in reverse order, we skip the fields that were - // set by entries written later (i.e. already read ones). - if (isChanged(field, flags | mask) && field != CLEANUP) - skip(txnId, field, in, userVersion); - else - deserialize(field, in, userVersion); - } - - // upper 16 bits are changed flags, lower are nulls; by masking upper by ~lower we restrict to only non-null changed fields - this.flags |= readFlags & (~readFlags << 16); - } - - private void deserialize(Field field, DataInputPlus in, Version userVersion) throws IOException - { - switch (field) - { - case EXECUTE_AT: - executeAt = ExecuteAtSerializer.deserialize(txnId, in); - break; - case EXECUTES_AT_LEAST: - executesAtLeast = ExecuteAtSerializer.deserialize(in); - break; - case MIN_UNIQUE_HLC: - minUniqueHlc = in.readUnsignedVInt(); - break; - case SAVE_STATUS: - saveStatus = SaveStatus.values()[in.readByte()]; - break; - case DURABILITY: - durability = Durability.forEncoded(in.readUnsignedByte()); - break; - case ACCEPTED: - acceptedOrCommitted = CommandSerializers.ballot.deserialize(in); - break; - case PROMISED: - promised = CommandSerializers.ballot.deserialize(in); - break; - case PARTICIPANTS: - participants = CommandSerializers.participants.deserialize(in); - break; - case PARTIAL_TXN: - partialTxn = CommandSerializers.partialTxn.deserialize(in, userVersion); - break; - case PARTIAL_DEPS: - // TODO (expected): this optimisation will be easily disabled; - // should either operate natively on ByteBuffer - // or else use some explicit API for copying bytes while skipping - if (deserializeDeps || !(in instanceof DataInputBuffer)) - { - partialDeps = DepsSerializers.partialDepsById.deserialize(in); - } - else - { - ByteBuffer buf = ((DataInputBuffer)in).buffer(); - int start = buf.position(); - DepsSerializers.partialDepsById.skip(in); - int end = buf.position(); - partialDeps = buf.duplicate().position(start).limit(end); - } - break; - case WAITING_ON: - waitingOn = WaitingOnSerializer.deserializeBitSets(txnId, in); - break; - case WRITES: - writes = CommandSerializers.writes.deserialize(in, userVersion); - break; - case CLEANUP: - Cleanup newCleanup = Cleanup.forOrdinal(in.readByte()); - if (cleanup == null || newCleanup.compareTo(cleanup) > 0) - cleanup = newCleanup; - break; - case RESULT: - result = ResultSerializers.result.deserialize(in); - break; - } - } - - private static void skip(TxnId txnId, Field field, DataInputPlus in, Version userVersion) throws IOException - { - switch (field) - { - default: throw new UnhandledEnum(field); - case EXECUTE_AT: - ExecuteAtSerializer.skip(txnId, in); - break; - case EXECUTES_AT_LEAST: - ExecuteAtSerializer.skip(in); - break; - case MIN_UNIQUE_HLC: - in.readUnsignedVInt(); - break; - case SAVE_STATUS: - case DURABILITY: - case CLEANUP: - in.readByte(); - break; - case ACCEPTED: - case PROMISED: - CommandSerializers.ballot.skip(in); - break; - case PARTICIPANTS: - CommandSerializers.participants.skip(in); - break; - case PARTIAL_TXN: - CommandSerializers.partialTxn.skip(in, userVersion); - break; - case PARTIAL_DEPS: - DepsSerializers.partialDepsById.skip(in); - break; - case WAITING_ON: - WaitingOnSerializer.skip(txnId, in); - break; - case WRITES: - // TODO (expected): skip - CommandSerializers.writes.skip(in, userVersion); - break; - case RESULT: - // TODO (expected): skip - ResultSerializers.result.skip(in); - break; - } - } - } -} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournalTable.java b/src/java/org/apache/cassandra/service/accord/AccordJournalTable.java deleted file mode 100644 index 1af6cb72b3cf..000000000000 --- a/src/java/org/apache/cassandra/service/accord/AccordJournalTable.java +++ /dev/null @@ -1,646 +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.service.accord; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; - -import javax.annotation.CheckForNull; -import javax.annotation.Nullable; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.AbstractIterator; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import accord.local.MaxDecidedRX; -import accord.primitives.Timestamp; -import accord.primitives.TxnId; -import accord.utils.Invariants; -import accord.utils.UncheckedInterruptedException; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.cql3.ColumnIdentifier; -import org.apache.cassandra.cql3.Operator; -import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.ColumnFamilyStore.RefViewFragment; -import org.apache.cassandra.db.DataRange; -import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.EmptyIterators; -import org.apache.cassandra.db.PartitionRangeReadCommand; -import org.apache.cassandra.db.ReadExecutionController; -import org.apache.cassandra.db.Slices; -import org.apache.cassandra.db.StorageHook; -import org.apache.cassandra.db.filter.ColumnFilter; -import org.apache.cassandra.db.filter.DataLimits; -import org.apache.cassandra.db.filter.RowFilter; -import org.apache.cassandra.db.lifecycle.SSTableSet; -import org.apache.cassandra.db.lifecycle.View; -import org.apache.cassandra.db.marshal.AbstractType; -import org.apache.cassandra.db.marshal.BytesType; -import org.apache.cassandra.db.marshal.Int32Type; -import org.apache.cassandra.db.marshal.LongType; -import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; -import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators; -import org.apache.cassandra.db.rows.Row; -import org.apache.cassandra.db.rows.UnfilteredRowIterator; -import org.apache.cassandra.db.rows.UnfilteredRowIterators; -import org.apache.cassandra.dht.Bounds; -import org.apache.cassandra.index.Index; -import org.apache.cassandra.index.accord.OrderedRouteSerializer; -import org.apache.cassandra.index.accord.RouteJournalIndex; -import org.apache.cassandra.io.FSReadError; -import org.apache.cassandra.io.sstable.ISSTableScanner; -import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.io.util.DataInputBuffer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.journal.Journal; -import org.apache.cassandra.journal.KeySupport; -import org.apache.cassandra.journal.RecordConsumer; -import org.apache.cassandra.journal.Segment; -import org.apache.cassandra.journal.Segments; -import org.apache.cassandra.schema.ColumnMetadata; -import org.apache.cassandra.service.RetryStrategy; -import org.apache.cassandra.service.accord.AccordKeyspace.JournalColumns; -import org.apache.cassandra.service.accord.api.TokenKey; -import org.apache.cassandra.service.accord.serializers.CommandSerializers; -import org.apache.cassandra.service.accord.serializers.Version; -import org.apache.cassandra.utils.Closeable; -import org.apache.cassandra.utils.CloseableIterator; -import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.JVMStabilityInspector; -import org.apache.cassandra.utils.MergeIterator; -import org.apache.cassandra.utils.concurrent.OpOrder; - -import static org.apache.cassandra.io.sstable.SSTableReadsListener.NOOP_LISTENER; -import static org.apache.cassandra.service.accord.AccordKeyspace.JournalColumns.getJournalKey; - -public class AccordJournalTable implements JournalRangeSearcher.Supplier -{ - private static final Logger logger = LoggerFactory.getLogger(AccordJournalTable.class); - - private final Journal journal; - private final ColumnFamilyStore cfs; - - private final ColumnMetadata recordColumn; - private final ColumnMetadata versionColumn; - - private final KeySupport keySupport; - /** - * Access to this field should only ever be handled by {@link #safeNotify(Consumer)}. There is an assumption that - * an error in the index should not cause the journal to crash, so {@link #safeNotify(Consumer)} exists to make sure - * this property holds true. - */ - @Nullable - private final JournalSegmentRangeSearcher index; - private final Version accordJournalVersion; - - public AccordJournalTable(Journal journal, KeySupport keySupport, ColumnFamilyStore cfs, Version accordJournalVersion) - { - this.journal = journal; - this.cfs = cfs; - this.recordColumn = cfs.metadata().getColumn(ColumnIdentifier.getInterned("record", false)); - this.versionColumn = cfs.metadata().getColumn(ColumnIdentifier.getInterned("user_version", false)); - this.keySupport = keySupport; - this.accordJournalVersion = accordJournalVersion; - - this.index = cfs.indexManager.getIndexByName(AccordKeyspace.JOURNAL_INDEX_NAME) != null - ? new JournalSegmentRangeSearcher<>() - : null; - } - - boolean shouldIndex(JournalKey key) - { - if (index == null) return false; - return RouteJournalIndex.allowed(key); - } - - void safeNotify(Consumer> fn) - { - if (index == null) - return; - try - { - fn.accept(index); - } - catch (Throwable t) - { - JVMStabilityInspector.inspectThrowable(t); - logger.warn("Failure updating index", t); - } - } - - public void forceCompaction() - { - cfs.forceMajorCompaction(); - } - - @Override - public JournalRangeSearcher rangeSearcher() - { - if (index == null) - return JournalRangeSearcher.NoopJournalRangeSearcher.instance; - return new JournalTableRangeSearcher(); - } - - public void start() - { - if (index == null) return; - Index tableIndex = cfs.indexManager.getIndexByName(AccordKeyspace.JOURNAL_INDEX_NAME); - RetryStrategy retry = DatabaseDescriptor.getAccord().retry_journal_index_ready.retry(); - for (int i = 0; !cfs.indexManager.isIndexQueryable(tableIndex); i++) - { - logger.debug("Journal index {} is not ready wait... waiting", AccordKeyspace.JOURNAL_INDEX_NAME); - maybeWait(retry, i); - } - } - - /** - * This method is here to make it easier for org.apache.cassandra.distributed.test.accord.journal.JournalAccessRouteIndexOnStartupRaceTest - * to check when we need to do waiting - */ - @VisibleForTesting - private static void maybeWait(RetryStrategy retry, int i) - { - long waitTime = retry.computeWait(i, TimeUnit.MICROSECONDS); - if (waitTime == -1) - throw new IllegalStateException("Gave up waiting on journal index to be ready"); - try - { - TimeUnit.MICROSECONDS.sleep(waitTime); - } - catch (InterruptedException e) - { - throw new UncheckedInterruptedException(e); - } - } - - public interface Reader - { - void read(DataInputPlus input, Version userVersion) throws IOException; - } - - static class RecordConsumerAdapter implements RecordConsumer - { - protected final Reader reader; - - RecordConsumerAdapter(Reader reader) - { - this.reader = reader; - } - - private long prevSegment = Long.MAX_VALUE; - private long prevPosition = Long.MAX_VALUE; - - @Override - public void accept(long segment, int position, K key, ByteBuffer buffer, int userVersion) - { - Invariants.require(segment <= prevSegment, - "Records should always be iterated over in a reverse order, but segment %d was seen after %d while reading %s", segment, prevSegment, key); - Invariants.require(segment != prevSegment || position < prevPosition, - "Records should always be iterated over in a reverse order, but position %d was seen after %d for segment %d while reading %s", position, prevPosition, segment, key); - readBuffer(buffer, reader, Version.fromVersion(userVersion)); - prevSegment = segment; - prevPosition = position; - } - } - - /** - * When using {@link PartitionRangeReadCommand} we need to work with {@link RowFilter} which works with columns. - * But the index doesn't care about table based queries and needs to be queried using the fields in the index, to - * support that this enum exists. This enum represents the fields present in the index and can be used to apply - * filters to the index. - */ - public enum SyntheticColumn - { - participants("participants", BytesType.instance), - store_id("store_id", Int32Type.instance), - txn_id("txn_id", BytesType.instance); - - public final ColumnMetadata metadata; - - SyntheticColumn(String name, AbstractType type) - { - this.metadata = new ColumnMetadata("journal", "routes", new ColumnIdentifier(name, false), type, ColumnMetadata.NO_UNIQUE_ID, ColumnMetadata.NO_POSITION, ColumnMetadata.Kind.REGULAR, null); - } - } - - private class JournalTableRangeSearcher implements JournalRangeSearcher - { - private final Index tableIndex; - - private JournalTableRangeSearcher() - { - this.tableIndex = cfs.indexManager.getIndexByName("record"); - if (!cfs.indexManager.isIndexQueryable(tableIndex)) - throw new AssertionError("Journal record index is not queryable"); - } - - @Override - public Result search(int commandStoreId, TokenRange range, TxnId minTxnId, Timestamp maxTxnId, @Nullable MaxDecidedRX.DecidedRX decidedRX) - { - CloseableIterator inMemory = index.search(commandStoreId, range, minTxnId, maxTxnId, decidedRX).results(); - CloseableIterator table = tableSearch(commandStoreId, range.start(), range.end(), minTxnId, maxTxnId, decidedRX); - return new DefaultResult(minTxnId, maxTxnId, decidedRX, MergeIterator.get(Arrays.asList(inMemory, table))); - } - - @Override - public Result search(int commandStoreId, TokenKey key, TxnId minTxnId, Timestamp maxTxnId, @Nullable MaxDecidedRX.DecidedRX decidedRX) - { - CloseableIterator inMemory = index.search(commandStoreId, key, minTxnId, maxTxnId, decidedRX).results(); - CloseableIterator table = tableSearch(commandStoreId, key, minTxnId, maxTxnId); - return new DefaultResult(minTxnId, maxTxnId, decidedRX, MergeIterator.get(Arrays.asList(inMemory, table))); - } - - private CloseableIterator tableSearch(int store, TokenKey start, TokenKey end, TxnId minTxnId, Timestamp maxTxnId, @Nullable MaxDecidedRX.DecidedRX decidedRX) - { - RowFilter rowFilter = RowFilter.create(false); - rowFilter.add(AccordJournalTable.SyntheticColumn.participants.metadata, Operator.GT, OrderedRouteSerializer.serialize(start)); - rowFilter.add(AccordJournalTable.SyntheticColumn.participants.metadata, Operator.LTE, OrderedRouteSerializer.serialize(end)); - rowFilter.add(AccordJournalTable.SyntheticColumn.store_id.metadata, Operator.EQ, Int32Type.instance.decompose(store)); - rowFilter.add(AccordJournalTable.SyntheticColumn.txn_id.metadata, Operator.GTE, CommandSerializers.txnId.serialize(minTxnId)); - rowFilter.add(AccordJournalTable.SyntheticColumn.txn_id.metadata, Operator.LTE, CommandSerializers.timestamp.serialize(maxTxnId)); - return process(store, rowFilter); - } - - private CloseableIterator tableSearch(int store, TokenKey key, TxnId minTxnId, Timestamp maxTxnId) - { - RowFilter rowFilter = RowFilter.create(false); - rowFilter.add(AccordJournalTable.SyntheticColumn.participants.metadata, Operator.GTE, OrderedRouteSerializer.serialize(key)); - rowFilter.add(AccordJournalTable.SyntheticColumn.participants.metadata, Operator.LTE, OrderedRouteSerializer.serialize(key)); - rowFilter.add(AccordJournalTable.SyntheticColumn.store_id.metadata, Operator.EQ, Int32Type.instance.decompose(store)); - rowFilter.add(AccordJournalTable.SyntheticColumn.txn_id.metadata, Operator.GTE, CommandSerializers.txnId.serialize(minTxnId)); - rowFilter.add(AccordJournalTable.SyntheticColumn.txn_id.metadata, Operator.LTE, CommandSerializers.timestamp.serialize(maxTxnId)); - return process(store, rowFilter); - } - - private CloseableIterator process(int storeId, RowFilter rowFilter) - { - PartitionRangeReadCommand cmd = PartitionRangeReadCommand.create(cfs.metadata(), - FBUtilities.nowInSeconds(), - ColumnFilter.selectionBuilder() - .add(AccordJournalTable.SyntheticColumn.store_id.metadata) - .add(AccordJournalTable.SyntheticColumn.txn_id.metadata) - .build(), - rowFilter, - DataLimits.NONE, - DataRange.allData(cfs.getPartitioner())); - Index.Searcher s = tableIndex.searcherFor(cmd); - try (ReadExecutionController controller = cmd.executionController()) - { - UnfilteredPartitionIterator partitionIterator = s.search(controller); - return new CloseableIterator<>() - { - - @Override - public void close() - { - partitionIterator.close(); - } - - @Override - public boolean hasNext() - { - return partitionIterator.hasNext(); - } - - @Override - public TxnId next() - { - UnfilteredRowIterator next = partitionIterator.next(); - JournalKey partitionKeyComponents = getJournalKey(next.partitionKey()); - Invariants.require(partitionKeyComponents.commandStoreId == storeId, - () -> String.format("table index returned a command store other than the exepcted one; expected %d != %d", storeId, partitionKeyComponents.commandStoreId)); - return partitionKeyComponents.id; - } - }; - } - } - } - - /** - * Perform a read from Journal table, followed by the reads from all journal segments. - *

- * When reading from journal segments, skip descriptors that were read from the table. - */ - public void readAll(K key, Reader reader) - { - readAll(key, new RecordConsumerAdapter<>(reader)); - } - - public void readAll(K key, RecordConsumer reader) - { - try (OpOrder.Group readOrder = cfs.readOrdering.start()) - { - // SELECT segments first, to avoid missing segments due to races compacting segment->sstable - Segments segments = journal.segments(); - try (TableKeyIterator table = readAllFromTable(key, readOrder)) - { - boolean hasTableData = table.advance(); - long minSegment = hasTableData ? table.segment : Long.MIN_VALUE; - // First, read all journal entries newer than anything flushed into sstables - Journal.readAll(key, (segment, position, key1, buffer, userVersion) -> { - if (segment > minSegment) - reader.accept(segment, position, key1, buffer, userVersion); - }, readOrder, segments); - - // Then, read SSTables - while (hasTableData) - { - reader.accept(table.segment, table.offset, key, table.value, table.userVersion); - hasTableData = table.advance(); - } - } - } - } - - public void readLast(K key, Reader reader) - { - readLast(key, new RecordConsumerAdapter<>(reader)); - } - - public void readLast(K key, RecordConsumer reader) - { - try (OpOrder.Group readOrder = cfs.readOrdering.start()) - { - Segments segments = journal.segments(); - try (TableKeyIterator table = readAllFromTable(key, readOrder)) - { - boolean hasTableData = table.advance(); - long minSegment = hasTableData ? table.segment : Long.MIN_VALUE; - - class JournalReader implements RecordConsumer - { - boolean read; - @Override - public void accept(long segment, int position, K key, ByteBuffer buffer, int userVersion) - { - if (segment > minSegment) - { - reader.accept(segment, position, key, buffer, userVersion); - read = true; - } - } - } - - // First, read all journal entries newer than anything flushed into sstables - JournalReader journalReader = new JournalReader(); - Journal.readLast(key, journalReader, readOrder, segments); - - // Then, read SSTables, if we haven't found a record already - if (hasTableData && !journalReader.read) - reader.accept(table.segment, table.offset, key, table.value, table.userVersion); - } - } - } - - // TODO (expected): why are recordColumn and versionColumn instance fields, so that this cannot be a static class? - class TableKeyIterator implements Closeable, RecordConsumer - { - final K key; - final List unmerged; - final UnfilteredRowIterator merged; - - long segment; - int offset; - ByteBuffer value; - int userVersion; - - TableKeyIterator(K key, List unmerged, UnfilteredRowIterator merged) - { - this.key = key; - this.unmerged = unmerged; - this.merged = merged; - } - - @Override - public void accept(long segment, int offset, K key, ByteBuffer buffer, int userVersion) - { - this.segment = segment; - this.offset = offset; - this.value = buffer; - this.userVersion = userVersion; - } - - boolean advance() - { - if (merged == null || !merged.hasNext()) - return false; - - try - { - Row row = (Row) merged.next(); - segment = LongType.instance.compose(ByteBuffer.wrap((byte[]) row.clustering().get(0))); - offset = Int32Type.instance.compose(ByteBuffer.wrap((byte[]) row.clustering().get(1))); - value = row.getCell(recordColumn).buffer(); - userVersion = Int32Type.instance.compose(row.getCell(versionColumn).buffer()); - return true; - } - catch (Throwable t) - { - throw new FSReadError("Failed to read from " + unmerged, t); - } - } - - @Override - public void close() - { - if (merged != null) - merged.close(); - } - } - - private TableKeyIterator readAllFromTable(K key, OpOrder.Group readOrder) - { - DecoratedKey pk = JournalColumns.decorate(key); - List iters = new ArrayList<>(3); - try - { - ColumnFamilyStore.ViewFragment view = cfs.select(View.select(SSTableSet.LIVE, pk)); - for (SSTableReader sstable : view.sstables) - { - if (!sstable.mayContainAssumingKeyIsInRange(pk)) - continue; - - UnfilteredRowIterator iter = StorageHook.instance.makeRowIterator(cfs, sstable, pk, Slices.ALL, ColumnFilter.all(cfs.metadata()), false, NOOP_LISTENER); - if (iter.getClass() != EmptyIterators.EmptyUnfilteredRowIterator.class) - iters.add(iter); - } - - return new TableKeyIterator(key, iters, iters.isEmpty() ? null : UnfilteredRowIterators.merge(iters)); - } - catch (Throwable t) - { - for (UnfilteredRowIterator iter : iters) - { - try { iter.close(); } - catch (Throwable t2) { t.addSuppressed(t2); } - } - throw t; - } - } - - @SuppressWarnings("resource") // Auto-closeable iterator will release related resources - public CloseableIterator> keyIterator(@Nullable K min, @Nullable K max, boolean includeActive) - { - try (OpOrder.Group readOrder = cfs.readOrdering.start()) - { - return new JournalAndTableKeyIterator(min, max, includeActive); - } - } - - private class TableIterator extends AbstractIterator implements CloseableIterator - { - private final UnfilteredPartitionIterator mergeIterator; - private final RefViewFragment view; - - private TableIterator(JournalKey min, JournalKey max) - { - Invariants.require((min != null && max != null) || min == max); - view = cfs.selectAndReference(View.select(SSTableSet.LIVE, r -> (max == null || JournalKey.SUPPORT.compare(getJournalKey(r.getFirst()), max) <= 0) - && (min == null || JournalKey.SUPPORT.compare(getJournalKey(r.getLast()), min) >= 0))); - List scanners = new ArrayList<>(); - for (SSTableReader sstable : view.sstables) - { - if (min == null) scanners.add(sstable.getScanner()); - else scanners.add(sstable.getScanner(new Bounds(JournalColumns.decorate(min), JournalColumns.decorate(max)))); - } - - mergeIterator = view.sstables.isEmpty() - ? EmptyIterators.unfilteredPartition(cfs.metadata()) - : UnfilteredPartitionIterators.merge(scanners, UnfilteredPartitionIterators.MergeListener.NOOP); - } - - @CheckForNull - protected K computeNext() - { - K ret = null; - if (mergeIterator.hasNext()) - { - try (UnfilteredRowIterator partition = mergeIterator.next()) - { - ret = (K) getJournalKey(partition.partitionKey()); - while (partition.hasNext()) - partition.next(); - } - } - - if (ret != null) - return ret; - else - return endOfData(); - } - - @Override - public void close() - { - mergeIterator.close(); - view.close(); - } - } - - private class JournalAndTableKeyIterator extends AbstractIterator> implements CloseableIterator> - { - final Journal.SegmentKeyIterator journalIterator; - final TableIterator tableIterator; - - private JournalAndTableKeyIterator(K min, K max, boolean includeActive) - { - // We must initialise journal reader first, else we may race with segment->table compaction and miss some data - // that is, the following sequence could happen: - // - Select sstables to read - // - Segments compacted; segments removed and sstables added - // - Segment iterator created - // TODO (expected): segments should be sstables on creation - this.journalIterator = journal.segmentKeyIterator(min, max, includeActive ? ignore -> true : Segment::isStatic); - this.tableIterator = new TableIterator(min, max); - } - - K prevFromTable = null; - K prevFromJournal = null; - - @Override - protected Journal.KeyRefs computeNext() - { - K tableKey = tableIterator.hasNext() ? tableIterator.peek() : null; - K journalKey = journalIterator.hasNext() ? journalIterator.peek().key() : null; - - if (journalKey != null) - { - Invariants.require(prevFromJournal == null || keySupport.compare(journalKey, prevFromJournal) >= 0, // == for case where we have not consumed previous on prev iteration - "Incorrect sort order in journal segments: %s should strictrly follow %s " + this, journalKey, prevFromJournal); - prevFromJournal = journalKey; - } - else - { - prevFromJournal = null; - } - - if (tableKey != null) - { - Invariants.require(prevFromTable == null || keySupport.compare(tableKey, prevFromTable) >= 0, // == for case where we have not consumed previous on prev iteration - "Incorrect sort order in journal table: %s should strictrly follow %s " + this, tableKey, prevFromTable); - prevFromTable = tableKey; - } - else - { - prevFromTable = null; - } - - if (tableKey == null) - return journalKey == null ? endOfData() : journalIterator.next(); - - if (journalKey == null) - return new Journal.KeyRefs<>(tableIterator.next()); - - int cmp = keySupport.compare(tableKey, journalKey); - if (cmp == 0) - { - tableIterator.next(); - return journalIterator.next(); - } - - return cmp < 0 ? new Journal.KeyRefs<>(tableIterator.next()) : journalIterator.next(); - } - - public void close() - { - tableIterator.close(); - journalIterator.close(); - } - } - - public static void readBuffer(ByteBuffer buffer, Reader reader, Version userVersion) - { - try (DataInputBuffer in = new DataInputBuffer(buffer, false)) - { - reader.read(in, userVersion); - } - catch (IOException e) - { - // can only throw if serializer is buggy or bytes got corrupted - throw new RuntimeException(e); - } - } -} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java b/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java deleted file mode 100644 index 3c1d71fd483f..000000000000 --- a/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java +++ /dev/null @@ -1,343 +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.service.accord; - -import java.io.IOException; -import java.util.NavigableMap; - -import com.google.common.collect.ImmutableSortedMap; - -import accord.local.DurableBefore; -import accord.local.RedundantBefore; -import accord.primitives.Ranges; -import accord.primitives.Timestamp; -import accord.primitives.TxnId; - -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.service.accord.journal.AccordTopologyUpdate; -import org.apache.cassandra.service.accord.serializers.CommandStoreSerializers; -import org.apache.cassandra.service.accord.serializers.Version; - -import static accord.local.CommandStores.RangesForEpoch; - -public class AccordJournalValueSerializers -{ - public interface FlyweightImage - { - void reset(JournalKey key); - } - - public interface FlyweightSerializer - { - IMAGE mergerFor(); - - void serialize(JournalKey key, ENTRY from, DataOutputPlus out, Version userVersion) throws IOException; - - void reserialize(JournalKey key, IMAGE from, DataOutputPlus out, Version userVersion) throws IOException; - - void deserialize(JournalKey key, IMAGE into, DataInputPlus in, Version userVersion) throws IOException; - - default IMAGE deserialize(JournalKey key, DataInputPlus in, Version userVersion) throws IOException - { - IMAGE image = mergerFor(); - deserialize(key, image, in, userVersion); - return image; - } - } - - public static class CommandDiffSerializer - implements FlyweightSerializer - { - @Override - public AccordJournal.Builder mergerFor() - { - return new AccordJournal.Builder(); - } - - @Override - public void serialize(JournalKey key, AccordJournal.Writer writer, DataOutputPlus out, Version userVersion) - { - try - { - writer.write(out, userVersion); - } - catch (IOException e) - { - throw new RuntimeException(e); - } - } - - @Override - public void reserialize(JournalKey key, AccordJournal.Builder from, DataOutputPlus out, Version userVersion) throws IOException - { - from.serialize(out, - // In CompactionIterator, we are dealing with relatively recent records, so we do not pass redundant before here. - // However, we do on load and during Journal SSTable compaction. - userVersion); - } - - @Override - public void deserialize(JournalKey journalKey, AccordJournal.Builder into, DataInputPlus in, Version userVersion) throws IOException - { - into.deserializeNext(in, userVersion); - } - } - - public abstract static class Accumulator implements FlyweightImage - { - protected A accumulated; - - public Accumulator(A initial) - { - this.accumulated = initial; - } - - public void update(V newValue) - { - accumulated = accumulate(accumulated, newValue); - } - - protected abstract A accumulate(A oldValue, V newValue); - - public A get() - { - return accumulated; - } - } - - public static class IdentityAccumulator extends Accumulator - { - final T initial; - boolean hasRead; - public IdentityAccumulator(T initial) - { - super(initial); - this.initial = initial; - } - - @Override - public void reset(JournalKey key) - { - hasRead = false; - accumulated = initial; - } - - @Override - protected T accumulate(T oldValue, T newValue) - { - if (hasRead) - return oldValue; - hasRead = true; - return newValue; - } - - @Override - public String toString() - { - return "IdentityAccumulator{" + - initial + - '}'; - } - } - - public static class RedundantBeforeSerializer - implements FlyweightSerializer> - { - @Override - public IdentityAccumulator mergerFor() - { - return new IdentityAccumulator<>(RedundantBefore.EMPTY); - } - - @Override - public void serialize(JournalKey key, RedundantBefore entry, DataOutputPlus out, Version userVersion) - { - try - { - if (entry == RedundantBefore.EMPTY) - { - out.writeInt(0); - return; - } - out.writeInt(1); - CommandStoreSerializers.redundantBefore.serialize(entry, out); - } - catch (IOException e) - { - throw new RuntimeException(e); - } - } - - @Override - public void reserialize(JournalKey key, IdentityAccumulator from, DataOutputPlus out, Version userVersion) throws IOException - { - serialize(key, from.get(), out, userVersion); - } - - @Override - public void deserialize(JournalKey journalKey, IdentityAccumulator into, DataInputPlus in, Version userVersion) throws IOException - { - if (in.readInt() == 0) - { - into.update(RedundantBefore.EMPTY); - return; - } - into.update(CommandStoreSerializers.redundantBefore.deserialize(in)); - } - } - - public static class DurableBeforeAccumulator extends Accumulator - { - public DurableBeforeAccumulator() - { - super(DurableBefore.EMPTY); - } - - @Override - public void reset(JournalKey key) - { - accumulated = DurableBefore.EMPTY; - } - - @Override - protected DurableBefore accumulate(DurableBefore oldValue, DurableBefore newValue) - { - return DurableBefore.merge(oldValue, newValue); - } - } - - public static class DurableBeforeSerializer - implements FlyweightSerializer - { - public DurableBeforeAccumulator mergerFor() - { - return new DurableBeforeAccumulator(); - } - - @Override - public void serialize(JournalKey key, DurableBefore entry, DataOutputPlus out, Version userVersion) - { - try - { - CommandStoreSerializers.durableBefore.serialize(entry, out); - } - catch (IOException e) - { - throw new RuntimeException(e); - } - } - - @Override - public void reserialize(JournalKey key, DurableBeforeAccumulator from, DataOutputPlus out, Version userVersion) throws IOException - { - serialize(key, from.get(), out, userVersion); - } - - @Override - public void deserialize(JournalKey journalKey, DurableBeforeAccumulator into, DataInputPlus in, Version userVersion) throws IOException - { - into.update(CommandStoreSerializers.durableBefore.deserialize(in)); - } - } - - public static class BootstrapBeganAtSerializer - implements FlyweightSerializer, IdentityAccumulator>> - { - @Override - public IdentityAccumulator> mergerFor() - { - return new IdentityAccumulator<>(ImmutableSortedMap.of(TxnId.NONE, Ranges.EMPTY)); - } - - @Override - public void serialize(JournalKey key, NavigableMap entry, DataOutputPlus out, Version userVersion) throws IOException - { - CommandStoreSerializers.bootstrapBeganAt.serialize(entry, out); - } - - @Override - public void reserialize(JournalKey key, IdentityAccumulator> image, DataOutputPlus out, Version userVersion) throws IOException - { - serialize(key, image.get(), out, userVersion); - } - - @Override - public void deserialize(JournalKey key, IdentityAccumulator> into, DataInputPlus in, Version userVersion) throws IOException - { - into.update(CommandStoreSerializers.bootstrapBeganAt.deserialize(in)); - } - } - - public static class SafeToReadSerializer - implements FlyweightSerializer, IdentityAccumulator>> - { - @Override - public IdentityAccumulator> mergerFor() - { - return new IdentityAccumulator<>(ImmutableSortedMap.of(Timestamp.NONE, Ranges.EMPTY)); - } - - @Override - public void serialize(JournalKey key, NavigableMap from, DataOutputPlus out, Version userVersion) throws IOException - { - CommandStoreSerializers.safeToRead.serialize(from, out); - } - - @Override - public void reserialize(JournalKey key, IdentityAccumulator> from, DataOutputPlus out, Version userVersion) throws IOException - { - serialize(key, from.get(), out, userVersion); - } - - @Override - public void deserialize(JournalKey key, IdentityAccumulator> into, DataInputPlus in, Version userVersion) throws IOException - { - into.update(CommandStoreSerializers.safeToRead.deserialize(in)); - } - } - - public static class RangesForEpochSerializer - implements FlyweightSerializer> - { - public static final RangesForEpochSerializer instance = new RangesForEpochSerializer(); - public IdentityAccumulator mergerFor() - { - return new IdentityAccumulator<>(null); - } - - @Override - public void serialize(JournalKey key, RangesForEpoch from, DataOutputPlus out, Version userVersion) throws IOException - { - AccordTopologyUpdate.RangesForEpochSerializer.instance.serialize(from, out); - } - - @Override - public void reserialize(JournalKey key, Accumulator from, DataOutputPlus out, Version userVersion) throws IOException - { - serialize(key, from.get(), out, userVersion); - } - - @Override - public void deserialize(JournalKey key, Accumulator into, DataInputPlus in, Version userVersion) throws IOException - { - into.update(AccordTopologyUpdate.RangesForEpochSerializer.instance.deserialize(in)); - } - } -} diff --git a/src/java/org/apache/cassandra/service/accord/AccordKeyspace.java b/src/java/org/apache/cassandra/service/accord/AccordKeyspace.java index 55d06ec04767..935cf397a61a 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordKeyspace.java +++ b/src/java/org/apache/cassandra/service/accord/AccordKeyspace.java @@ -240,7 +240,7 @@ public static LocalToken makeSystemTableToken(int commandStore, TokenKey key) public static ByteBuffer makeSystemTableKeyBytes(int commandStore, TokenKey key) { - ByteBuffer result = ByteBuffer.allocate(4 + TokenKey.serializer.serializedSizeWithoutPrefix(key)); + ByteBuffer result = ByteBuffer.allocate(4 + TokenKey.serializer.serializedSizeWithoutPrefixOrLength(key)); result.putInt(commandStore); TokenKey.serializer.serializeWithoutPrefixOrLength(key, result); result.flip(); diff --git a/src/java/org/apache/cassandra/service/accord/AccordMessageSink.java b/src/java/org/apache/cassandra/service/accord/AccordMessageSink.java index 5e3168d15b9b..2afd8d02a3ea 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordMessageSink.java +++ b/src/java/org/apache/cassandra/service/accord/AccordMessageSink.java @@ -47,6 +47,7 @@ import org.apache.cassandra.net.ResponseContext; import org.apache.cassandra.net.Verb; import org.apache.cassandra.service.TimeoutStrategy; +import org.apache.cassandra.service.accord.topology.AccordEndpointMapper; import org.apache.cassandra.utils.Clock; import static accord.messages.MessageType.StandardMessage.ACCEPT_REQ; diff --git a/src/java/org/apache/cassandra/service/accord/AccordObjectSizes.java b/src/java/org/apache/cassandra/service/accord/AccordObjectSizes.java index fc1b35e2bb1a..8e282873d176 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordObjectSizes.java +++ b/src/java/org/apache/cassandra/service/accord/AccordObjectSizes.java @@ -275,8 +275,14 @@ public static long dependencies(Deps dependencies) size += dependencies.keyDeps.txnIdCount() * TIMESTAMP_SIZE; size += dependencies.rangeDeps.txnIdCount() * TIMESTAMP_SIZE; - size += KeyDeps.SerializerSupport.keysToTxnIds(dependencies.keyDeps).length * 4L; - size += RangeDeps.SerializerSupport.rangesToTxnIds(dependencies.rangeDeps).length * 4L; + if (dependencies.keyDeps.hasByKey()) + size += KeyDeps.SerializerSupport.keysToTxnIds(dependencies.keyDeps).length * 4L; + if (dependencies.keyDeps.hasByTxnId()) + size += KeyDeps.SerializerSupport.txnIdsToKeys(dependencies.keyDeps).length * 4L; + if (dependencies.rangeDeps.hasByRange()) + size += RangeDeps.SerializerSupport.rangesToTxnIds(dependencies.rangeDeps).length * 4L; + if (dependencies.rangeDeps.hasByTxnId()) + size += RangeDeps.SerializerSupport.txnIdsToRanges(dependencies.rangeDeps).length * 4L; return size; } diff --git a/src/java/org/apache/cassandra/service/accord/AccordResponseVerbHandler.java b/src/java/org/apache/cassandra/service/accord/AccordResponseVerbHandler.java index b5df0863f49a..767f92fd60ba 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordResponseVerbHandler.java +++ b/src/java/org/apache/cassandra/service/accord/AccordResponseVerbHandler.java @@ -31,6 +31,7 @@ import org.apache.cassandra.net.IVerbHandler; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.service.accord.topology.AccordEndpointMapper; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.NoSpamLogger; import org.apache.cassandra.utils.NoSpamLogger.NoSpamLogStatement; diff --git a/src/java/org/apache/cassandra/service/accord/AccordResult.java b/src/java/org/apache/cassandra/service/accord/AccordResult.java index 04b474ffa8bc..013af99b877b 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordResult.java +++ b/src/java/org/apache/cassandra/service/accord/AccordResult.java @@ -21,6 +21,7 @@ import java.util.HashSet; import java.util.Iterator; import java.util.Set; +import java.util.concurrent.CancellationException; import java.util.concurrent.TimeoutException; import java.util.function.BiConsumer; @@ -186,7 +187,7 @@ else if (isTxnRequest && fail instanceof TopologyMismatch) trySuccess((V) RetryWithNewProtocolResult.instance); return false; } - else if (fail instanceof RequestTimeoutException || fail instanceof TimeoutException) + else if (fail instanceof RequestTimeoutException || fail instanceof TimeoutException || fail instanceof CancellationException) { report = bookkeeping.newTimeout(txnId, keysOrRanges); } diff --git a/src/java/org/apache/cassandra/service/accord/AccordSafeCommand.java b/src/java/org/apache/cassandra/service/accord/AccordSafeCommand.java index 361e0e38ca7f..488ec61cb0a8 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordSafeCommand.java +++ b/src/java/org/apache/cassandra/service/accord/AccordSafeCommand.java @@ -42,9 +42,9 @@ public DebugAccordSafeCommand(AccordCacheEntry global) } @Override - public void invalidate() + public void markUnsafe() { - super.invalidate(); + super.markUnsafe(); selfRef.release(); } @@ -54,7 +54,7 @@ public static void trace(AccordSafeCommand safeCommand, String message) } } - private boolean invalidated; + private boolean unsafe; private final AccordCacheEntry global; private Command original; private Command current; @@ -86,7 +86,7 @@ public int hashCode() public String toString() { return "AccordSafeCommand{" + - "invalidated=" + invalidated + + "invalidated=" + unsafe + ", global=" + global + ", original=" + original + ", current=" + current + @@ -138,14 +138,14 @@ public void preExecute() } @Override - public void invalidate() + public void markUnsafe() { - invalidated = true; + unsafe = true; } @Override - public boolean invalidated() + public boolean isUnsafe() { - return invalidated; + return unsafe; } } diff --git a/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java b/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java index 29eb6988d06c..8800ea3cf085 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java +++ b/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java @@ -37,6 +37,7 @@ import accord.local.CommandStores; import accord.local.CommandSummaries; import accord.local.NodeCommandStoreService; +import accord.local.RedundantBefore; import accord.local.cfk.CommandsForKey; import accord.local.cfk.SafeCommandsForKey; import accord.primitives.Timestamp; @@ -48,6 +49,7 @@ import org.apache.cassandra.metrics.LogLinearDecayingHistograms; import org.apache.cassandra.service.accord.AccordCommandStore.ExclusiveCaches; import org.apache.cassandra.service.accord.AccordCommandStore.SafeRedundantBefore; +import org.apache.cassandra.service.accord.AccordDurableOnFlush.ReportDurable; import org.apache.cassandra.service.paxos.PaxosState; import static accord.utils.Invariants.illegalState; @@ -189,6 +191,13 @@ public void updateCommandsForRanges(Command prev, Command updated, boolean force commandStore().rangeIndex().update(prev, updated, force); } + @Override + public void reportDurable(RedundantBefore addRedundantBefore, int flags) + { + upsertRedundantBefore(addRedundantBefore); + commandStore.maybeTerminated(ReportDurable.isCommandStoreFlush(flags), ReportDurable.isDataStoreFlush(flags)); + } + @Override public AccordCommandStore commandStore() { diff --git a/src/java/org/apache/cassandra/service/accord/AccordSafeCommandsForKey.java b/src/java/org/apache/cassandra/service/accord/AccordSafeCommandsForKey.java index 634abda68b18..3d34d463dca4 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordSafeCommandsForKey.java +++ b/src/java/org/apache/cassandra/service/accord/AccordSafeCommandsForKey.java @@ -120,13 +120,13 @@ public void preExecute() } @Override - public void invalidate() + public void markUnsafe() { invalidated = true; } @Override - public boolean invalidated() + public boolean isUnsafe() { return invalidated; } diff --git a/src/java/org/apache/cassandra/service/accord/AccordSafeState.java b/src/java/org/apache/cassandra/service/accord/AccordSafeState.java index 50ce897c0363..a254c1154d08 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordSafeState.java +++ b/src/java/org/apache/cassandra/service/accord/AccordSafeState.java @@ -23,8 +23,8 @@ public interface AccordSafeState extends SafeState { void set(V update); V original(); - void invalidate(); - boolean invalidated(); + void markUnsafe(); + boolean isUnsafe(); void preExecute(); AccordCacheEntry global(); @@ -51,7 +51,7 @@ default Throwable failure() default void checkNotInvalidated() { - if (invalidated()) + if (isUnsafe()) throw new IllegalStateException("Cannot access invalidated " + this); } } diff --git a/src/java/org/apache/cassandra/service/accord/AccordService.java b/src/java/org/apache/cassandra/service/accord/AccordService.java index 92c4556bce09..2f0d575b31c5 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordService.java +++ b/src/java/org/apache/cassandra/service/accord/AccordService.java @@ -23,6 +23,7 @@ import java.util.Collection; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -39,6 +40,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.primitives.Ints; +import org.agrona.collections.Long2LongHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,6 +55,8 @@ import accord.impl.progresslog.DefaultProgressLog; import accord.impl.progresslog.DefaultProgressLogs; import accord.local.Catchup; +import accord.local.CatchupHard; +import accord.local.CommandStores; import accord.local.Node; import accord.local.Node.Id; import accord.local.ShardDistributor.EvenSplit; @@ -76,18 +80,25 @@ import accord.topology.TopologyRange; import accord.utils.DefaultRandom; import accord.utils.Invariants; +import accord.utils.Reduce; +import accord.utils.UnhandledEnum; import accord.utils.async.AsyncChain; import accord.utils.async.AsyncResult; import accord.utils.async.AsyncResults; import org.apache.cassandra.concurrent.Shutdownable; import org.apache.cassandra.config.AccordSpec; +import org.apache.cassandra.config.AccordSpec.CatchupMode; +import org.apache.cassandra.config.AccordSpec.JournalSpec.ReplaySavePoint; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.db.SystemKeyspace; import org.apache.cassandra.db.SystemKeyspace.BootstrapState; +import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.journal.Descriptor; import org.apache.cassandra.journal.Params; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.metrics.AccordExecutorMetrics; @@ -98,9 +109,10 @@ import org.apache.cassandra.net.MessageDelivery; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.SharedContext; +import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.accord.AccordSyncPropagator.Notification; +import org.apache.cassandra.service.accord.AccordKeyspace.AccordColumnFamilyStores; import org.apache.cassandra.service.accord.TimeOnlyRequestBookkeeping.LatencyRequestBookkeeping; import org.apache.cassandra.service.accord.api.AccordAgent; import org.apache.cassandra.service.accord.api.AccordRoutableKey; @@ -111,8 +123,19 @@ import org.apache.cassandra.service.accord.api.CompositeTopologySorter; import org.apache.cassandra.service.accord.api.TokenKey.KeyspaceSplitter; import org.apache.cassandra.service.accord.interop.AccordInteropAdapter.AccordInteropFactory; +import org.apache.cassandra.service.accord.journal.AccordJournal; +import org.apache.cassandra.service.accord.journal.ReplayMarkers; import org.apache.cassandra.service.accord.serializers.TableMetadatas; import org.apache.cassandra.service.accord.serializers.TableMetadatasAndKeys; +import org.apache.cassandra.service.accord.topology.AccordEndpointMapper; +import org.apache.cassandra.service.accord.topology.AccordFastPathCoordinator; +import org.apache.cassandra.service.accord.topology.AccordSyncPropagator; +import org.apache.cassandra.service.accord.topology.AccordSyncPropagator.Notification; +import org.apache.cassandra.service.accord.topology.AccordTopology; +import org.apache.cassandra.service.accord.topology.AccordTopologyService; +import org.apache.cassandra.service.accord.topology.EndpointMapping; +import org.apache.cassandra.service.accord.topology.FetchTopologies; +import org.apache.cassandra.service.accord.topology.WatermarkCollector; import org.apache.cassandra.service.accord.txn.TxnQuery; import org.apache.cassandra.service.accord.txn.TxnRead; import org.apache.cassandra.service.accord.txn.TxnResult; @@ -129,6 +152,7 @@ import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.concurrent.AsyncFuture; import org.apache.cassandra.utils.concurrent.AsyncPromise; +import org.apache.cassandra.utils.concurrent.Condition; import org.apache.cassandra.utils.concurrent.Future; import org.apache.cassandra.utils.concurrent.ImmediateFuture; import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; @@ -136,26 +160,33 @@ import static accord.api.Journal.TopologyUpdate; import static accord.api.ProtocolModifiers.Toggles.FastExec.MAY_BYPASS_SAFESTORE; import static accord.impl.progresslog.DefaultProgressLog.ModeFlag.CATCH_UP; -import static accord.local.RedundantStatus.Property.LOCALLY_DURABLE_TO_COMMAND_STORE; -import static accord.local.RedundantStatus.Property.LOCALLY_DURABLE_TO_DATA_STORE; import static accord.local.durability.DurabilityService.SyncLocal.Self; import static accord.local.durability.DurabilityService.SyncRemote.All; import static accord.messages.SimpleReply.Ok; import static accord.primitives.Txn.Kind.ExclusiveSyncPoint; import static accord.primitives.Txn.Kind.Write; +import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.NANOSECONDS; import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; +import static org.apache.cassandra.concurrent.ExecutorFactory.SimulatorThreadTag.JOB; +import static org.apache.cassandra.concurrent.ExecutorFactory.SystemThreadTag.DAEMON; +import static org.apache.cassandra.config.AccordSpec.CatchupMode.DISABLED; +import static org.apache.cassandra.config.AccordSpec.CatchupMode.FALLBACK_TO_HARD; +import static org.apache.cassandra.config.AccordSpec.CatchupMode.HARD; +import static org.apache.cassandra.config.AccordSpec.JournalSpec.ReplayMode.RESET; +import static org.apache.cassandra.config.DatabaseDescriptor.getAccord; import static org.apache.cassandra.config.DatabaseDescriptor.getAccordCommandStoreShardCount; import static org.apache.cassandra.config.DatabaseDescriptor.getAccordGlobalDurabilityCycle; import static org.apache.cassandra.config.DatabaseDescriptor.getAccordShardDurabilityCycle; import static org.apache.cassandra.config.DatabaseDescriptor.getAccordShardDurabilityMaxSplits; import static org.apache.cassandra.config.DatabaseDescriptor.getAccordShardDurabilityTargetSplits; import static org.apache.cassandra.config.DatabaseDescriptor.getPartitioner; +import static org.apache.cassandra.db.ColumnFamilyStore.FlushReason.DRAIN; import static org.apache.cassandra.db.SystemKeyspace.BootstrapState.COMPLETED; -import static org.apache.cassandra.journal.Params.ReplayMode.RESET; import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.accordReadBookkeeping; import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.accordWriteBookkeeping; -import static org.apache.cassandra.service.accord.AccordTopology.tcmIdToAccord; +import static org.apache.cassandra.service.accord.topology.AccordTopology.tcmIdToAccord; import static org.apache.cassandra.service.consensus.migration.ConsensusRequestRouter.getTableMetadata; import static org.apache.cassandra.utils.Clock.Global.nanoTime; @@ -253,7 +284,7 @@ public void unsafeResetForTesting(ClusterMetadata metadata) ProtocolModifiers.Toggles.setDataStoreDetectsFutureReads(true); } - private enum State { INIT, STARTED, SHUTTING_DOWN, SHUTDOWN } + private enum State { INIT, STARTING, STARTED, STOPPED, SHUTTING_DOWN, SHUTDOWN } private final Node node; private final AccordMessageSink messageSink; @@ -268,6 +299,7 @@ private enum State { INIT, STARTED, SHUTTING_DOWN, SHUTDOWN } @GuardedBy("this") private volatile State state = State.INIT; + private final Condition isShutdown = Condition.newOneTimeCondition(); private static final IAccordService NOOP_SERVICE = new NoOpAccordService(); @@ -275,17 +307,19 @@ private enum State { INIT, STARTED, SHUTTING_DOWN, SHUTDOWN } // tests can specify a DelegatingService if they want to override private static IAccordService instance; private static IAccordService unsafeInstance; + private static volatile IAccordService requestInstance; + private static volatile IAccordService replyInstance; @VisibleForTesting public static void unsafeSetNewAccordService(IAccordService service) { - unsafeInstance = instance = service; + unsafeInstance = instance = requestInstance = replyInstance = service; } @VisibleForTesting public static void unsafeSetNoop() { - unsafeInstance = instance = NOOP_SERVICE; + unsafeInstance = instance = requestInstance = replyInstance = NOOP_SERVICE; } public static IAccordService tryGetUnsafe() @@ -317,14 +351,16 @@ public static IVerbHandler watermarkHandlerOrNoop() public static IVerbHandler requestHandlerOrNoop() { - if (instance == null) return ignore -> {}; - return instance.requestHandler(); + IAccordService accord = requestInstance; + if (accord == null) return ignore -> {}; + return accord.requestHandler(); } public static IVerbHandler responseHandlerOrNoop() { - if (unsafeInstance == null) return ignore -> {}; - return unsafeInstance.responseHandler(); + IAccordService accord = replyInstance; + if (accord == null) return ignore -> {}; + return accord.responseHandler(); } @VisibleForTesting @@ -333,12 +369,12 @@ public synchronized static void localStartup(NodeId tcmId) Invariants.require(instance == null); if (!DatabaseDescriptor.getAccordTransactionsEnabled()) { - unsafeInstance = instance = NOOP_SERVICE; + unsafeSetNoop(); } else { AccordService as = new AccordService(tcmIdToAccord(tcmId)); - unsafeInstance = as; + unsafeInstance = replyInstance = as; as.localStartup(); } } @@ -349,11 +385,10 @@ public synchronized static AccordService distributedStartup() return null; AccordService as = (AccordService) unsafeInstance; - if (as.state != State.INIT) + if (as.state != State.STARTING) return as; as.distributedStartupInternal(); - instance = as; AccordReplicaMetrics.touch(); AccordSystemMetrics.touch(); @@ -363,24 +398,25 @@ public synchronized static AccordService distributedStartup() } @VisibleForTesting - public static boolean replayJournal(AccordService as) + private boolean replayJournal(Long2LongHashMap minSegments) { - logger.info("Starting journal replay."); - long start = nanoTime(); - if (as.journalConfiguration().replayMode() == RESET) + if (getAccord().journal.replay == RESET) + { + if (getAccord().journal.replaySavePoint != ReplaySavePoint.NO) + throw new IllegalArgumentException("Cannot reset state and replay from a save point; must modify either accord.journal.replay or accord.journal.replay_save_point"); AccordKeyspace.truncateCommandsForKey(); + } - as.node.commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().stop()); - as.journal().replay(as.node.commandStores()); + logger.info("Starting journal replay."); + long start = nanoTime(); + boolean success = journal().replay(node.commandStores(), minSegments); logger.info("Waiting for command stores to quiesce."); - ((AccordCommandStores)as.node.commandStores()).waitForQuiescence(); - getBlocking(as.node.commandStores().forAll("Post Replay", safeStore -> ((AccordCommandStore)safeStore.commandStore()).rangeIndex().postReplay())); - as.journal.unsafeSetStarted(); - as.node.commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().start()); + ((AccordCommandStores)node.commandStores()).waitForQuiescence(); + getBlocking(node.commandStores().forAll("Post Replay", safeStore -> ((AccordCommandStore)safeStore.commandStore()).rangeIndex().postReplay())); long end = nanoTime(); logger.info("Finished journal replay. {}s elapsed", String.format("%.2f", NANOSECONDS.toMillis(end - start)/1000.0)); - return true; + return success; } public static IAccordService instance() @@ -410,7 +446,7 @@ public AccordService(Id localId) Invariants.require(localId != null, "static localId must be set before instantiating AccordService"); logger.info("Starting accord with nodeId {}", localId); AccordAgent agent = FBUtilities.construct(CassandraRelevantProperties.ACCORD_AGENT_CLASS.getString(AccordAgent.class.getName()), "AccordAgent"); - agent.setNodeId(localId); + agent.setup(localId); AccordTimeService time = new AccordTimeService(); final RequestCallbacks callbacks = new RequestCallbacks(time); this.scheduler = new AccordScheduler(); @@ -449,39 +485,127 @@ public synchronized void localStartup() if (state != State.INIT) return; + boolean rebootstrap = false; + { + long startMarker = ReplayMarkers.readStartMarker(); + long stopMarker = ReplayMarkers.readStopMarker(); + if (stopMarker < startMarker) + { + switch (getAccord().journal.stopMarkerFailurePolicy) + { + default: throw new UnhandledEnum(getAccord().journal.stopMarkerFailurePolicy); + case EXIT: + throw new RuntimeException("Stop marker is older than start marker (" + stopMarker + '<' + startMarker + ") , so cannot assume we have a complete log of our votes in any consensus groups. Exiting."); + + case UNSAFE_STARTUP: + logger.warn("Stop marker is older than start marker ({}<{}), so cannot assume we have a complete log of our votes in any consensus groups. Continuing to startup as configured.", stopMarker, startMarker); + break; + + case REBOOTSTRAP: + logger.info("Stop marker is older than start marker ({}<{}). Rebootstrapping.", stopMarker, startMarker); + rebootstrap = true; + } + } + } + + logger.info("Starting background compaction of system_accord"); + // We control this ourselves to ensure it starts when we need it, as especially commands_for_key + // can accumulate a lot of state and degrade replay performance significantly + scheduler.recurring(() -> { + CompactionManager.instance.submitBackground(AccordColumnFamilyStores.commandsForKey); + CompactionManager.instance.submitBackground(AccordColumnFamilyStores.journal); + }, 1L, MINUTES); + + state = State.STARTING; node.unsafeSetReplaying(true); try { - journal.start(node); - node.load(); + node.durability().stop(); + journal.open(node); + node.load(); ClusterMetadata metadata = ClusterMetadata.current(); endpointMapper.updateMapping(metadata); - List images = journal.replayTopologies(); - if (!images.isEmpty()) + // Initialise command stores using latest topology from the log; + // if there are no local command stores, don't report any topologies and simply fetch the latest known in the cluster + // this avoids a registered (not joined) node learning of topologies, then later restarting with some intervening + // epochs having been garbage collected by the other nodes in the cluster + + topologyService.onStartup(node); + List images = journal.loadTopologies(); + TopologyUpdate last = images.isEmpty() ? null : images.get(images.size() - 1); + boolean initialiseCommandStores = last != null && !last.commandStores.isEmpty(); + if (initialiseCommandStores) + node.commandStores().initializeTopologyUnsafe(last); + + node.commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().stop()); + + // restore save points before starting the journal so we can validate consistency between journal and save point state (where possible) + Long2LongHashMap minSegments; + if (rebootstrap) minSegments = null; + else { - // Initialise command stores using latest topology from the log; - // if there are no local command stores, don't report any topologies and simply fetch the latest known in the cluster - // this avoids a registered (not joined) node learning of topologies, then later restarting with some intervening - // epochs having been garbage collected by the other nodes in the cluster - TopologyUpdate last = images.get(images.size() - 1); - if (!last.commandStores.isEmpty()) + switch (getAccord().journal.replaySavePoint) { - node.commandStores().initializeTopologyUnsafe(last); - - // Replay local epochs - for (TopologyUpdate image : images) - node.topology().reportTopology(image.global); + default: throw new UnhandledEnum(getAccord().journal.replaySavePoint); + case NO: minSegments = new Long2LongHashMap(0L); break; + case LATEST: minSegments = restoreFromSavePoints(node.commandStores()); } } - replayJournal(this); + + // now start the journal before we replay, as replay may trigger its own new journal writes + journal.start(node); + + if (initialiseCommandStores) + { + // Replay local topologies + for (TopologyUpdate image : images) + node.topology().reportTopology(image.global); + } + + node.commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().start()); + if (rebootstrap) + { + // rebootstrap expects the durability service to process visibility sync points for command stores + node.durability().start(); + getBlocking(node.commandStores().rebootstrap(node)); + } + else + { + replayJournal(minSegments); + + logger.info("Try to execute pending transactions..."); + List> results = new ArrayList<>(); + node.commandStores().forAllUnsafe(commandStore -> results.add(commandStore.tryToExecuteListeningTxns(false))); + if (!results.isEmpty()) + getBlocking(AsyncResults.reduce(results, Reduce.toNull())); + } } finally { node.unsafeSetReplaying(false); } + node.commandStores().forAllUnsafe(commandStore -> ((AccordCommandStore)commandStore).ensureDurable()); + } + + private Long2LongHashMap restoreFromSavePoints(CommandStores commandStores) + { + Long2LongHashMap result = new Long2LongHashMap(0); + Future>> future = toFuture(((AccordCommandStores)commandStores).restoreState()); + future.awaitThrowUncheckedOnInterrupt(); + List> success = future.getNow(); + if (success != null) + { + for (Map.Entry e : success) + { + if (e != null && e.getValue() > 0) + result.put(e.getKey(), e.getValue()); + } + } + + return result; } private void distributedStartupInternal() @@ -494,29 +618,34 @@ private void distributedStartupInternal() // we set ourselves to STARTED before starting progress logs as this is the condition we use to decide if we // start the progress log on command store initialisation (so creates a synchronisation point) + journal.writeStartMarker(); state = State.STARTED; - node.commandStores().forAll("", safeStore -> safeStore.progressLog().start()); - - // trigger catchup only after our progress mechanisms are initialised - catchup(); + instance = requestInstance = this; + node.commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().start()); node.durability().shards().reconfigure(Ints.checkedCast(getAccordShardDurabilityTargetSplits()), Ints.checkedCast(getAccordShardDurabilityMaxSplits()), Ints.checkedCast(getAccordShardDurabilityCycle(SECONDS)), SECONDS); node.durability().global().setGlobalCycleTime(Ints.checkedCast(getAccordGlobalDurabilityCycle(SECONDS)), SECONDS); - // Only enable durability scheduling and progress logs _after_ we have fully replayed journal - node.durability().start(); + + // Only enable durability scheduling + if (!node.durability().isStarted()) + node.durability().start(); + + // trigger catchup only after our progress mechanisms are initialised + catchup(); } void catchup() { AccordSpec spec = DatabaseDescriptor.getAccord(); - if (!spec.catchup_on_start) + if (spec.catchup_on_start == DISABLED) { logger.info("Catchup disabled; continuing to startup"); return; } + CatchupMode mode = spec.catchup_on_start; BootstrapState bootstrapState = SystemKeyspace.getBootstrapState(); if (bootstrapState == COMPLETED) { @@ -527,10 +656,21 @@ void catchup() int attempts = 1; while (true) { - logger.info("Catchup with quorum..."); + logger.info("Catchup ({}) with quorum...", mode); long start = nanoTime(); long failAt = start + maxLatencyNanos; - Future f = toFuture(Catchup.catchup(node)); + Future f; + { + AsyncChain submit; + if (mode == HARD) + { + if (!node.durability().isStarted()) + node.durability().start(); + submit = CatchupHard.catchup(node); + } + else submit = Catchup.catchup(node); + f = toFuture(submit); + } if (!f.awaitUntilThrowUncheckedOnInterrupt(failAt)) { if (spec.catchup_on_start_exit_on_failure) @@ -572,6 +712,8 @@ void catchup() } logger.info("Catchup was slow, so we may behind again; retrying"); + if (mode == FALLBACK_TO_HARD) + mode = HARD; } } finally @@ -978,14 +1120,15 @@ public boolean isTerminated() return scheduler.isTerminated(); } + static class FlushingCacheEntries extends AsyncResults.CountingResult implements Runnable + { + public FlushingCacheEntries() { super(1); } + @Override public void run() { decrement(); } + } + public synchronized Future flushCaches() { - class Ready extends AsyncResults.CountingResult implements Runnable - { - public Ready() { super(1); } - @Override public void run() { decrement(); } - } - Ready ready = new Ready(); + FlushingCacheEntries flushing = new FlushingCacheEntries(); AccordCommandStores commandStores = (AccordCommandStores) node.commandStores(); commandStores.forAllUnsafe(unsafeStore -> { AccordCommandStore commandStore = (AccordCommandStore)unsafeStore; @@ -994,34 +1137,87 @@ class Ready extends AsyncResults.CountingResult implements Runnable caches.commandsForKeys().forEach(entry -> { if (entry.isModified()) { - ready.increment(); - caches.global().saveWhenReadyExclusive(entry, ready); + flushing.increment(); + caches.global().saveWhenReadyExclusive(entry, flushing); } }); } }); - ready.decrement(); - AsyncPromise result = new AsyncPromise<>(); - ready.invoke((success, fail) -> { - if (fail != null) result.tryFailure(fail); - else result.trySuccess(null); - }); - return result; + flushing.decrement(); + return toFuture(flushing); } - public synchronized void markShuttingDown() + public synchronized void stop() { - state = State.SHUTTING_DOWN; + if (state == State.INIT) + return; + + logger.info("Stopping Accord"); + requestInstance = replyInstance = null; + node.durability().stop(); + // TODO (expected): stop TopologyManager from reporting new topologies + topologyService.shutdown(); + AccordCommandStores commandStores = (AccordCommandStores)node.commandStores(); + Set tableIds = commandStores.shutdownStores(); + commandStores.waitForQuiescence(); + journal.writeSafeStopMarker(); + scheduler.shutdownNow(); + toFuture(flushCaches()).map(ignore -> { + return AccordColumnFamilyStores.commandsForKey.forceFlush(DRAIN); + }); + for (TableId tableId : tableIds) + { + ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(tableId); + if (cfs != null) + cfs.forceFlush(DRAIN); + } + + state = State.STOPPED; + logger.info("Stopped Accord"); } @Override public synchronized void shutdown() { - if (state != State.STARTED && state != State.SHUTTING_DOWN) + if (state.compareTo(State.SHUTDOWN) >= 0) return; + + if (state.compareTo(State.STOPPED) < 0) + stop(); + + logger.info("Shutting down Accord"); state = State.SHUTTING_DOWN; - shutdownAndWait(1, TimeUnit.MINUTES); - state = State.SHUTDOWN; + long deadlineNanos = nanoTime() + DatabaseDescriptor.getAccord().shutdown_grace_period.toDuration().toNanos(); + executorFactory().startThread("ShutdownAccord", () -> { + AccordCommandStores commandStores = (AccordCommandStores)node.commandStores(); + boolean safeShutdown = commandStores.awaitStoreTermination(deadlineNanos); + if (!safeShutdown) + logger.warn("Cannot write safe replay marker as not all command stores terminated promptly"); + + commandStores.waitForQuiescence(); + Descriptor lastSegment = journal.stop(); + if (lastSegment == null && safeShutdown) + logger.warn("Cannot write safe replay marker as no segment descriptor reported by journal"); + + if (safeShutdown && lastSegment != null) + { + Future save = toFuture(commandStores.saveState(lastSegment)); + if (!save.awaitUntilThrowUncheckedOnInterrupt(deadlineNanos)) + logger.error("Timeout waiting to write safe replay markers"); + else if (save.cause() != null) + logger.error("Failed to write some safe replay markers", save.cause()); + else if (!save.getNow()) + logger.error("Failed to write some safe replay markers"); + else + logger.info("Written safe replay markers"); + } + + commandStores.shutdownExecutors(); + journal.close(); + state = State.SHUTDOWN; + isShutdown.signalAll(); + logger.info("Accord Shutdown"); + }, DAEMON, JOB); } @Override @@ -1034,32 +1230,32 @@ public Object shutdownNow() @Override public boolean awaitTermination(long timeout, TimeUnit units) throws InterruptedException { - try + long deadlineNanos = nanoTime() + units.toNanos(timeout); + boolean success = true; + if (!isShutdown.awaitUntil(deadlineNanos)) { - ExecutorUtils.awaitTermination(timeout, units, shutdownableSubsystems()); - return true; + logger.error("Accord command stores did not terminate before timeout elapsed"); + success = false; } - catch (TimeoutException e) + try { ExecutorUtils.awaitTerminationUntil(deadlineNanos, Arrays.asList(scheduler, node.commandStores())); } + catch (InterruptedException | TimeoutException e) { - return false; + logger.error("Accord executors did not terminate before timeout elapsed"); + success = false; } - } - - private List shutdownableSubsystems() - { - return Arrays.asList((AccordCommandStores)node.commandStores(), journal, topologyService, scheduler); + if (!journal.awaitTerminationUntil(deadlineNanos)) + { + logger.error("Accord journal did not terminate before timeout elapsed"); + success = false; + } + return success; } @VisibleForTesting @Override - public void shutdownAndWait(long timeout, TimeUnit unit) + public void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { - if (!ExecutorUtils.shutdownThenWait(shutdownableSubsystems(), timeout, unit)) - logger.error("One or more subsystems did not shut down cleanly."); - - node.commandStores().forAllUnsafe(commandStore -> { - logger.info("{} stopping with durability: {}", commandStore, commandStore.unsafeGetRedundantBefore().map(b -> b == null ? null : b.maxBoundBoth(LOCALLY_DURABLE_TO_DATA_STORE, LOCALLY_DURABLE_TO_COMMAND_STORE), TxnId[]::new)); - }); + ExecutorUtils.shutdownAndWait(timeout, unit, this); } @Override @@ -1120,11 +1316,7 @@ public void receive(Message message) public static void receive(MessageDelivery sink, TopologyManager topologyManager, Message message) { AccordSyncPropagator.Notification notification = message.payload; - notification.readyToCoordinate.forEach(id -> topologyManager.onReadyToCoordinate(id, notification.epoch)); - if (!notification.closed.isEmpty()) - topologyManager.onEpochClosed(notification.closed, notification.epoch); - if (!notification.retired.isEmpty()) - topologyManager.onEpochRetired(notification.retired, notification.epoch); + notification.process(topologyManager); sink.respond(Ok, message); } diff --git a/src/java/org/apache/cassandra/service/accord/AccordTask.java b/src/java/org/apache/cassandra/service/accord/AccordTask.java index 63790d5afe6c..d8311ba8c3b7 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordTask.java +++ b/src/java/org/apache/cassandra/service/accord/AccordTask.java @@ -90,7 +90,6 @@ import static org.apache.cassandra.config.DatabaseDescriptor.getPartitioner; import static org.apache.cassandra.service.accord.AccordTask.State.CANCELLED; import static org.apache.cassandra.service.accord.AccordTask.State.FAILED; -import static org.apache.cassandra.service.accord.AccordTask.State.FAILING; import static org.apache.cassandra.service.accord.AccordTask.State.FINISHED; import static org.apache.cassandra.service.accord.AccordTask.State.INITIALIZED; import static org.apache.cassandra.service.accord.AccordTask.State.LOADING; @@ -164,10 +163,9 @@ public enum State WAITING_TO_RUN(INITIALIZED, SCANNING_RANGES, WAITING_TO_LOAD, LOADING), RUNNING(WAITING_TO_RUN), PERSISTING(RUNNING), - FAILING(WAITING_TO_SCAN_RANGES, SCANNING_RANGES, WAITING_TO_LOAD, LOADING, WAITING_TO_RUN, RUNNING, PERSISTING), FINISHED(RUNNING, PERSISTING), CANCELLED(WAITING_TO_SCAN_RANGES, SCANNING_RANGES, WAITING_TO_LOAD, LOADING, WAITING_TO_RUN), - FAILED(WAITING_TO_SCAN_RANGES, SCANNING_RANGES, WAITING_TO_LOAD, LOADING, WAITING_TO_RUN, RUNNING, PERSISTING, FAILING); + FAILED(WAITING_TO_SCAN_RANGES, SCANNING_RANGES, WAITING_TO_LOAD, LOADING, WAITING_TO_RUN, RUNNING, PERSISTING); private final int permittedFrom; @@ -739,46 +737,28 @@ public void runInternal() public void fail(Throwable throwable) { - commandStore.agent().onException(throwable); if (state.isComplete()) return; - if (commandStore.hasSafeStore()) - commandStore.agent().onException(new IllegalStateException(String.format("Failure to cleanup safe store for %s; status=%s", this, state), throwable)); - - state(FAILING); - if (callback != null) - callback.accept(null, throwable); - } - - public void failExclusive(Throwable throwable) - { - boolean newFailure = state != FAILING; try { - if (newFailure) - { - commandStore.agent().onException(throwable); - if (state.isComplete()) - return; - - if (commandStore.hasSafeStore()) - commandStore.agent().onException(new IllegalStateException(String.format("Failure to cleanup safe store for %s; status=%s", this, state), throwable)); - } - + commandStore.agent().onException(throwable); state(FAILED); } finally { - if (newFailure && callback != null) + if (callback != null) callback.accept(null, throwable); } } + public void failExclusive(Throwable throwable) + { + fail(throwable); + } + protected void cleanupExclusive() { - if (state == FAILING) - state(FAILED); Invariants.expect(state.isExecuted()); releaseResources(commandStore.cachesExclusive()); if (runningAt != 0) @@ -815,7 +795,7 @@ public void cancelExclusive() if (rangeScanner != null) rangeScanner.cancelled = true; if (callback != null) - callback.accept(null, new CancellationException()); + commandStore.executor().submit(() -> callback.accept(null, new CancellationException())); } void cancelExclusive(AccordExecutor owner) @@ -912,7 +892,7 @@ private void safeRelease(Map> map, Acc { for (AccordSafeState safeState : map.values()) { - if (safeState.invalidated()) continue; + if (safeState.isUnsafe()) continue; try { cache.release(safeState, this); } catch (Throwable t) { suppressedBy.addSuppressed(t); } } @@ -922,7 +902,7 @@ private void safeRelease(Map> map, AccordCach { for (AccordSafeState safeState : map.values()) { - if (safeState.invalidated()) continue; + if (safeState.isUnsafe()) continue; try { cache.release(safeState, this); } catch (Throwable t) { suppressedBy.addSuppressed(t); } } @@ -1144,7 +1124,8 @@ void scannedInternal() void cleanup(Caches caches) { - loader.cleanupExclusive(caches); + if (loader != null) + loader.cleanupExclusive(caches); } CommandSummaries finish(Caches caches) diff --git a/src/java/org/apache/cassandra/service/accord/AccordVerbHandler.java b/src/java/org/apache/cassandra/service/accord/AccordVerbHandler.java index 1fa311fca03b..91adfafd5822 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordVerbHandler.java +++ b/src/java/org/apache/cassandra/service/accord/AccordVerbHandler.java @@ -28,6 +28,7 @@ import org.apache.cassandra.net.IVerbHandler; import org.apache.cassandra.net.Message; +import org.apache.cassandra.service.accord.topology.AccordEndpointMapper; import org.apache.cassandra.utils.NoSpamLogger; public class AccordVerbHandler implements IVerbHandler diff --git a/src/java/org/apache/cassandra/service/accord/IAccordService.java b/src/java/org/apache/cassandra/service/accord/IAccordService.java index bd1358f9b34a..954f9ae66b86 100644 --- a/src/java/org/apache/cassandra/service/accord/IAccordService.java +++ b/src/java/org/apache/cassandra/service/accord/IAccordService.java @@ -61,9 +61,12 @@ import org.apache.cassandra.net.IVerbHandler; import org.apache.cassandra.net.Message; import org.apache.cassandra.schema.TableId; -import org.apache.cassandra.service.accord.AccordSyncPropagator.Notification; import org.apache.cassandra.service.accord.api.AccordScheduler; import org.apache.cassandra.service.accord.api.AccordTopologySorter; +import org.apache.cassandra.service.accord.topology.AccordEndpointMapper; +import org.apache.cassandra.service.accord.topology.AccordSyncPropagator; +import org.apache.cassandra.service.accord.topology.AccordSyncPropagator.Notification; +import org.apache.cassandra.service.accord.topology.AccordTopologyService; import org.apache.cassandra.service.accord.txn.TxnResult; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Epoch; @@ -122,7 +125,7 @@ interface IAccordResult void localStartup(); Future flushCaches(); - void markShuttingDown(); + void stop(); void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException; AccordScheduler scheduler(); @@ -289,7 +292,7 @@ public void localStartup() } @Override - public void markShuttingDown() + public void stop() { } @@ -498,9 +501,9 @@ public Future flushCaches() } @Override - public void markShuttingDown() + public void stop() { - delegate.markShuttingDown(); + delegate.stop(); } @Override diff --git a/src/java/org/apache/cassandra/service/accord/InMemoryRangeIndex.java b/src/java/org/apache/cassandra/service/accord/InMemoryRangeIndex.java index feca42bd2f59..b4b607ded359 100644 --- a/src/java/org/apache/cassandra/service/accord/InMemoryRangeIndex.java +++ b/src/java/org/apache/cassandra/service/accord/InMemoryRangeIndex.java @@ -18,6 +18,7 @@ package org.apache.cassandra.service.accord; +import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.List; @@ -27,6 +28,7 @@ import javax.annotation.Nullable; +import accord.impl.cfr.IdEntry; import accord.impl.cfr.InMemoryRangeSummaryIndex; import accord.impl.cfr.LoadListener; import accord.local.Command; @@ -41,6 +43,12 @@ import accord.primitives.Unseekables; import accord.utils.async.Cancellable; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.service.accord.serializers.CommandStoreSerializers; + +import static org.apache.cassandra.io.util.CompressedFrameDataInputPlus.readList; +import static org.apache.cassandra.io.util.CompressedFrameDataOutputPlus.writeList; + public class InMemoryRangeIndex extends InMemoryRangeSummaryIndex implements RangeIndex { public static class Loader extends RangeIndex.Loader @@ -122,7 +130,7 @@ public void cleanupExclusive(AccordCommandStore.Caches caches) } @Override - AccordCommandStore commandStore() + protected AccordCommandStore commandStore() { return owner.commandStore; } @@ -152,4 +160,22 @@ private RangeIndex.Loader newLoader(RedundantBefore redundantBefore, MaxDecidedR { return new Loader(this, redundantBefore, maxDecidedRX, primaryTxnId, searchKeysOrRanges, testKind, minTxnId, maxTxnId, loadKeysFor); } + + @Override + public void save(File file) throws IOException + { + writeList(file, snapshot(), CommandStoreSerializers.rangeIndexIdEntry); + } + + @Override + public List load(File file) throws IOException + { + return readList(file, CommandStoreSerializers.rangeIndexIdEntry); + } + + @Override + public void restore(Object loaded) + { + restore((List)loaded); + } } diff --git a/src/java/org/apache/cassandra/service/accord/JournalKey.java b/src/java/org/apache/cassandra/service/accord/JournalKey.java index e45c8b95fcb1..b4731e249619 100644 --- a/src/java/org/apache/cassandra/service/accord/JournalKey.java +++ b/src/java/org/apache/cassandra/service/accord/JournalKey.java @@ -31,22 +31,25 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.journal.KeySupport; -import org.apache.cassandra.service.accord.journal.AccordTopologyUpdate; +import org.apache.cassandra.service.accord.journal.MergeSerializer; import org.apache.cassandra.utils.ByteArrayUtil; import static org.apache.cassandra.db.TypeSizes.BYTE_SIZE; import static org.apache.cassandra.db.TypeSizes.INT_SIZE; import static org.apache.cassandra.db.TypeSizes.LONG_SIZE; -import static org.apache.cassandra.service.accord.AccordJournalValueSerializers.BootstrapBeganAtSerializer; -import static org.apache.cassandra.service.accord.AccordJournalValueSerializers.CommandDiffSerializer; -import static org.apache.cassandra.service.accord.AccordJournalValueSerializers.DurableBeforeSerializer; -import static org.apache.cassandra.service.accord.AccordJournalValueSerializers.FlyweightSerializer; -import static org.apache.cassandra.service.accord.AccordJournalValueSerializers.RangesForEpochSerializer; -import static org.apache.cassandra.service.accord.AccordJournalValueSerializers.RedundantBeforeSerializer; -import static org.apache.cassandra.service.accord.AccordJournalValueSerializers.SafeToReadSerializer; +import static org.apache.cassandra.service.accord.journal.MergeSerializers.BootstrapBeganAtSerializer; +import static org.apache.cassandra.service.accord.journal.MergeSerializers.CommandChangeSerializer; +import static org.apache.cassandra.service.accord.journal.MergeSerializers.DurableBeforeSerializer; +import static org.apache.cassandra.service.accord.journal.MergeSerializers.RangesForEpochSerializer; +import static org.apache.cassandra.service.accord.journal.MergeSerializers.RedundantBeforeSerializer; +import static org.apache.cassandra.service.accord.journal.MergeSerializers.SafeToReadSerializer; +import static org.apache.cassandra.service.accord.journal.MergeSerializers.TopologySerializer; public final class JournalKey { + // TODO (expected): do we need a dedicated buffer here? + private static final ThreadLocal keyCRCBytes = ThreadLocal.withInitial(() -> new byte[JournalKeySupport.TOTAL_SIZE]); + public final Type type; public final TxnId id; public final int commandStoreId; @@ -186,7 +189,7 @@ private TxnId deserializeTxnId(ByteBuffer buffer, int position) @Override public void updateChecksum(Checksum crc, JournalKey key, int userVersion) { - byte[] out = AccordJournal.keyCRCBytes.get(); + byte[] out = keyCRCBytes.get(); serialize(key, out); crc.update(out, 0, out.length); } @@ -229,7 +232,7 @@ public int compare(JournalKey k1, JournalKey k2) if (cmp == 0) cmp = k1.id.compareTo(k2.id); return cmp; } - }; + } @Override public boolean equals(Object other) @@ -264,20 +267,20 @@ public String toString() public enum Type { - COMMAND_DIFF (0, new CommandDiffSerializer(), true), + COMMAND_DIFF (0, new CommandChangeSerializer(), true), REDUNDANT_BEFORE (1, new RedundantBeforeSerializer(), false), DURABLE_BEFORE (2, new DurableBeforeSerializer(), false), SAFE_TO_READ (3, new SafeToReadSerializer(), false), BOOTSTRAP_BEGAN_AT (4, new BootstrapBeganAtSerializer(), false), RANGES_FOR_EPOCH (5, new RangesForEpochSerializer(), false), - TOPOLOGY_UPDATE (6, new AccordTopologyUpdate.FlyweightSerializer(), true), + TOPOLOGY_UPDATE (6, new TopologySerializer(), true), ; public final int id; - public final FlyweightSerializer serializer; + public final MergeSerializer serializer; public final boolean usesTxnId; - Type(int id, FlyweightSerializer serializer, boolean usesTxnId) + Type(int id, MergeSerializer serializer, boolean usesTxnId) { this.id = id; this.serializer = serializer; diff --git a/src/java/org/apache/cassandra/service/accord/RangeIndex.java b/src/java/org/apache/cassandra/service/accord/RangeIndex.java index f5a623535c4a..1d744b2d4df4 100644 --- a/src/java/org/apache/cassandra/service/accord/RangeIndex.java +++ b/src/java/org/apache/cassandra/service/accord/RangeIndex.java @@ -28,6 +28,7 @@ import accord.local.LoadKeysFor; import accord.local.MaxDecidedRX; import accord.local.RedundantBefore; +import accord.primitives.Ranges; import accord.primitives.Routable; import accord.primitives.Timestamp; import accord.primitives.Txn; @@ -37,6 +38,8 @@ import org.apache.cassandra.exceptions.UnknownTableException; import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.service.accord.journal.CommandChanges; import org.apache.cassandra.service.accord.serializers.Version; import static accord.api.Journal.Load.MINIMAL; @@ -52,12 +55,12 @@ public Loader(RedundantBefore redundantBefore, MaxDecidedRX maxDecidedRX, TxnId super(redundantBefore, maxDecidedRX, primaryTxnId, searchKeysOrRanges, testKinds, minTxnId, maxTxnId, loadKeysFor); } - abstract AccordCommandStore commandStore(); + protected abstract AccordCommandStore commandStore(); - abstract void loadExclusive(Map into, AccordCommandStore.Caches caches); - abstract void load(Map into, BooleanSupplier abort); - abstract void finish(Map into); - abstract void cleanupExclusive(AccordCommandStore.Caches caches); + protected abstract void loadExclusive(Map into, AccordCommandStore.Caches caches); + protected abstract void load(Map into, BooleanSupplier abort); + protected abstract void finish(Map into); + protected abstract void cleanupExclusive(AccordCommandStore.Caches caches); protected CommandSummaries.Summary loadFromDisk(TxnId txnId) { @@ -110,7 +113,7 @@ public CommandSummaries.Summary ifRelevant(AccordCacheEntry stat return ifRelevant((Command) command); Invariants.require(command instanceof ByteBuffer); - AccordJournal.Builder builder = new AccordJournal.Builder(txnId, loadKeysFor != RECOVERY ? MINIMAL : MINIMAL_WITH_DEPS); + CommandChanges builder = new CommandChanges(txnId, loadKeysFor != RECOVERY ? MINIMAL : MINIMAL_WITH_DEPS); ByteBuffer buffer = (ByteBuffer) command; buffer.mark(); try (DataInputBuffer buf = new DataInputBuffer(buffer, false)) @@ -135,6 +138,10 @@ public CommandSummaries.Summary ifRelevant(AccordCacheEntry stat } Loader loader(TxnId primaryTxnId, Timestamp primaryExecuteAt, LoadKeysFor loadKeysFor, Unseekables keysOrRanges); - void update(Command prev, Command updated, boolean force); - void postReplay(); + default void update(Command prev, Command updated, boolean force) {} + default void postReplay() {} + default void prune(TxnId syncId, Ranges ranges, RedundantBefore redundantBefore) {} + default void save(File file) throws IOException {} + default Object load(File file) throws IOException { return null; } + default void restore(Object loaded) {} } diff --git a/src/java/org/apache/cassandra/service/accord/TokenRange.java b/src/java/org/apache/cassandra/service/accord/TokenRange.java index 3d20639f39fd..d890e5d012ea 100644 --- a/src/java/org/apache/cassandra/service/accord/TokenRange.java +++ b/src/java/org/apache/cassandra/service/accord/TokenRange.java @@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting; +import accord.api.ProtocolModifiers.RangeSpec; import accord.api.RoutingKey; import accord.primitives.Range; import accord.utils.Invariants; @@ -37,9 +38,13 @@ import org.apache.cassandra.service.accord.api.TokenKey; import org.apache.cassandra.utils.ObjectSizes; -public class TokenRange extends Range.EndInclusive +public class TokenRange extends Range { public static final long EMPTY_SIZE = ObjectSizes.measure(new TokenRange(TokenKey.min(TableId.fromLong(0), Murmur3Partitioner.instance), TokenKey.max(TableId.fromLong(0), Murmur3Partitioner.instance))); + static + { + Invariants.require(RangeSpec.isEndInclusive()); + } // Don't make this public use create or createUnsafe protected TokenRange(TokenKey start, TokenKey end) @@ -124,6 +129,14 @@ public org.apache.cassandra.dht.Range toKeyspaceRange() return new org.apache.cassandra.dht.Range<>(left, right); } + public static TokenRange fromKeyspaceRange(TableId tableId, org.apache.cassandra.dht.Range range) + { + Token left = range.left, right = range.right; + TokenKey start = left.isMinimum() ? TokenKey.min(tableId, left.getPartitioner()) : new TokenKey(tableId, left); + TokenKey end = right.isMinimum() ? TokenKey.max(tableId, right.getPartitioner()) : new TokenKey(tableId, right); + return create(start, end); + } + public static final Serializer serializer = new Serializer(); public static final class Serializer implements UnversionedSerializer diff --git a/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java b/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java index 88f6debf2b06..8d1a7ab90ae7 100644 --- a/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java +++ b/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java @@ -38,9 +38,11 @@ import accord.api.RoutingKey; import accord.api.Tracing; import accord.coordinate.Coordination; +import accord.coordinate.Exhausted; import accord.coordinate.Preempted; import accord.coordinate.Timeout; import accord.local.Command; +import accord.local.LogUnavailableException; import accord.local.Node; import accord.local.SafeCommand; import accord.local.SafeCommandStore; @@ -66,6 +68,7 @@ import accord.utils.async.AsyncChains; import accord.utils.async.Cancellable; +import org.apache.cassandra.config.AccordSpec; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.SystemKeyspace; import org.apache.cassandra.exceptions.RequestTimeoutException; @@ -121,6 +124,7 @@ public static void onFailedBarrier(TxnId txnId, Throwable cause) private final AccordTracing tracing = new AccordTracing(); private final RandomSource random = new DefaultRandom(); protected Node.Id self; + protected AccordSpec config; public AccordAgent() { @@ -143,9 +147,10 @@ public OwnershipEventListener ownershipEvents() return this; } - public void setNodeId(Node.Id id) + public void setup(Node.Id id) { self = id; + config = DatabaseDescriptor.getAccord(); } @Override @@ -186,7 +191,7 @@ public void onFailedBootstrap(int attempts, String phase, Ranges ranges, Runnabl else { logger.error(message, phase, ranges, ". Retrying in " + retryDelayMicros + "us.", failure); - AccordService.instance().scheduler().once(() -> { + AccordService.unsafeInstance().scheduler().once(() -> { logger.info("Retrying bootstrap of {}", ranges); retry.run(); }, retryDelayMicros, MICROSECONDS); @@ -205,7 +210,7 @@ public static void handleException(Throwable t) return; AccordSystemMetrics.metrics.errors.inc(); - if (t instanceof CancellationException || t instanceof TimeoutException || t instanceof Timeout || t instanceof Preempted) + if (t instanceof CancellationException || t instanceof TimeoutException || t instanceof Timeout || t instanceof Preempted || t instanceof Exhausted || t instanceof LogUnavailableException) // TODO (required): leaky logger, permitting multiple messages per time period and reporting how many were dropped noSpamLogger.warn("", t); else @@ -266,6 +271,21 @@ public long maxConflictsPruneInterval() return 1024; } + @Override + public boolean softReject(long unappliedCount, long maxUnappliedAge, long cumulativeUnappliedAge) + { + return unappliedCount > config.min_soft_reject_count + && (unappliedCount > config.max_soft_reject_count + || maxUnappliedAge > config.soft_reject_age.toMicroseconds() + || cumulativeUnappliedAge > config.soft_reject_cumulative_age.toMicroseconds()); + } + + @Override + public boolean hardReject(int softRejectCount, int totalCount) + { + return (softRejectCount / (float) totalCount) >= config.hard_reject_ratio; + } + /** * Create an empty transaction that Accord can use for its internal transactions. This is not suitable * for tests since it skips validation done by regular transactions. @@ -295,10 +315,19 @@ public ReplicaEventListener replicaEvents() public long slowCoordinatorDelay(Node node, SafeCommandStore safeStore, TxnId txnId, TimeUnit units, int attempt) { SafeCommand safeCommand = safeStore.unsafeGetNoCleanup(txnId); - Invariants.nonNull(safeCommand); + if (safeCommand == null) + { + noSpamLogger.warn("{} invoked slowCoordinatorDelay for {} without having it in cache", safeStore.commandStore(), txnId, new RuntimeException()); + return recover(txnId).computeWait(attempt, units); + } Command command = safeCommand.current(); - Invariants.nonNull(command); + if (command == null) + { + noSpamLogger.warn("{} invoked slowCoordinatorDelay for {} without knowing the command", safeStore.commandStore(), txnId, new RuntimeException()); + return recover(txnId).computeWait(attempt, units); + } + // TODO (expected): make this a configurable calculation on normal request latencies (like ContentionStrategy) long nowMicros = MILLISECONDS.toMicros(Clock.Global.currentTimeMillis()); @@ -368,7 +397,20 @@ public static long nonClashingStartTime(long startTime, SortedList node @Override public long slowReplicaDelay(Node node, SafeCommandStore safeStore, TxnId txnId, int attempt, BlockedUntil blockedUntil, TimeUnit units) { - Command command = Invariants.nonNull(safeStore.unsafeGetNoCleanup(txnId).current()); + SafeCommand safeCommand = safeStore.unsafeGetNoCleanup(txnId); + if (safeCommand == null) + { + noSpamLogger.warn("{} invoked slowReplicaDelay for {} without having it in cache", safeStore.commandStore(), txnId, new RuntimeException()); + return fetch(txnId).computeWait(attempt, units); + } + + Command command = safeCommand.current(); + if (command == null) + { + noSpamLogger.warn("{} invoked slowReplicaDelay for {} without knowing the command", safeStore.commandStore(), txnId, new RuntimeException()); + return fetch(txnId).computeWait(attempt, units); + } + long nowMicros = MILLISECONDS.toMicros(Clock.Global.currentTimeMillis()); long mostRecentStart = mostRecentStart(command, nowMicros); long waitMicros = fetch(txnId).computeWait(attempt, units); diff --git a/src/java/org/apache/cassandra/service/accord/api/AccordRoutableKey.java b/src/java/org/apache/cassandra/service/accord/api/AccordRoutableKey.java index aa9317a2ca87..13c1f7bc5bb5 100644 --- a/src/java/org/apache/cassandra/service/accord/api/AccordRoutableKey.java +++ b/src/java/org/apache/cassandra/service/accord/api/AccordRoutableKey.java @@ -42,7 +42,7 @@ public interface AccordSearchableKeySerializer extends AccordKeySerializer // -1 means dynamic int fixedKeyLengthForPrefix(Object prefix); int serializedSizeOfPrefix(Object prefix); - int serializedSizeWithoutPrefix(K key); + int serializedSizeWithoutPrefixOrLength(K key); void serializePrefix(Object prefix, DataOutputPlus out) throws IOException; void serializeWithoutPrefixOrLength(K key, DataOutputPlus out) throws IOException; Object deserializePrefix(DataInputPlus in) throws IOException; diff --git a/src/java/org/apache/cassandra/service/accord/api/AccordTopologySorter.java b/src/java/org/apache/cassandra/service/accord/api/AccordTopologySorter.java index d6ca0ab8253a..26e6f69e1022 100644 --- a/src/java/org/apache/cassandra/service/accord/api/AccordTopologySorter.java +++ b/src/java/org/apache/cassandra/service/accord/api/AccordTopologySorter.java @@ -35,11 +35,11 @@ import org.apache.cassandra.locator.Endpoint; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.NodeProximity; -import org.apache.cassandra.service.accord.AccordEndpointMapper; +import org.apache.cassandra.service.accord.topology.AccordEndpointMapper; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.Sortable; -import static org.apache.cassandra.service.accord.AccordEndpointMapper.NodeStatus.HEALTHY; +import static org.apache.cassandra.service.accord.topology.AccordEndpointMapper.NodeStatus.HEALTHY; public class AccordTopologySorter implements TopologySorter { diff --git a/src/java/org/apache/cassandra/service/accord/api/TokenKey.java b/src/java/org/apache/cassandra/service/accord/api/TokenKey.java index bd1887094807..7666f7048075 100644 --- a/src/java/org/apache/cassandra/service/accord/api/TokenKey.java +++ b/src/java/org/apache/cassandra/service/accord/api/TokenKey.java @@ -375,7 +375,7 @@ public void skip(DataInputPlus in, IPartitioner partitioner) throws IOException public ByteBuffer serialize(TokenKey key) { - int size = key.table.serializedCompactComparableSize() + serializedSizeWithoutPrefix(key); + int size = key.table.serializedCompactComparableSize() + serializedSizeWithoutPrefixOrLength(key); ByteBuffer result = ByteBuffer.allocate(size); result.position(key.table.serializeCompactComparable(result, ByteBufferAccessor.instance, 0)); serializeWithoutPrefixOrLength(key, result); @@ -417,7 +417,7 @@ public int fixedKeyLengthForPrefix(Object prefix) } @Override - public int serializedSizeWithoutPrefix(TokenKey key) + public int serializedSizeWithoutPrefixOrLength(TokenKey key) { return 2 + key.token.getPartitioner().accordSerializedSize(key.token); } @@ -445,7 +445,7 @@ public void serializeWithoutPrefixOrLength(TokenKey key, DataOutputPlus out) thr public ByteBuffer serializeWithoutPrefixOrLength(TokenKey key) { IPartitioner partitioner = key.token.getPartitioner(); - ByteBuffer result = ByteBuffer.allocate(serializedSizeWithoutPrefix(key)); + ByteBuffer result = ByteBuffer.allocate(serializedSizeWithoutPrefixOrLength(key)); serializeWithoutPrefixOrLength(key, result, partitioner); result.flip(); return result; diff --git a/src/java/org/apache/cassandra/service/accord/debug/DebugTxnGraph.java b/src/java/org/apache/cassandra/service/accord/debug/DebugTxnGraph.java index e86c3265a122..8bc6a6a53a20 100644 --- a/src/java/org/apache/cassandra/service/accord/debug/DebugTxnGraph.java +++ b/src/java/org/apache/cassandra/service/accord/debug/DebugTxnGraph.java @@ -22,10 +22,12 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Queue; +import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.TimeoutException; import java.util.function.BiConsumer; @@ -171,7 +173,7 @@ public DebugTxnGraph(IAccordService service, TxnId root, TxnKindsAndDomains kind protected AsyncChain> visitRoot(SafeCommandStore safeStore, Command command, P param) { - return visitParent(safeStore, command, param, new HashMap<>(), 0); + return visitParent(safeStore, command, param, new HashMap<>(), new HashSet<>(), 0); } void visit(long deadlineNanos) throws TimeoutException @@ -227,17 +229,18 @@ private AsyncChain> submitRoot(CommandStore commandStore, TxnId txnI }).flatMap(i -> i); } - private AsyncChain> submitParent(CommandStore commandStore, TxnId txnId, P param, Map infos, int depth) + private AsyncChain> submitParent(CommandStore commandStore, TxnId txnId, P param, Map infos, Set visitedParent, int depth) { + return commandStore.chain(PreLoadContext.contextFor(txnId, "Populate txn_graph"), safeStore -> { Command command = safeStore.unsafeGetNoCleanup(txnId).current(); if (command == null || command.saveStatus() == SaveStatus.Uninitialised) return AsyncChains.>success(null); - return visitParent(safeStore, command, param, infos, depth); + return visitParent(safeStore, command, param, infos, visitedParent, depth); }).flatMap(i -> i); } - private AsyncChain> visitParent(SafeCommandStore safeStore, Command command, P param, Map infos, int depth) + private AsyncChain> visitParent(SafeCommandStore safeStore, Command command, P param, Map infos, Set visitedParent, int depth) { CommandStore commandStore = safeStore.commandStore(); if (depth < maxDepth) @@ -296,7 +299,8 @@ else if (!infos.containsKey(txnId)) if (!next.saveStatus.hasBeen(Status.Committed) || next.saveStatus.hasBeen(Status.Truncated)) return; - queued.add(submitParent(commandStore, next.txnId, param, infos, depth + 1)); + if (visitedParent.add(next.txnId)) + queued.add(submitParent(commandStore, next.txnId, param, infos, visitedParent, depth + 1)); }); callback.accept(build(commandStore, depth, command, list, intersecting, param), null); return null; diff --git a/src/java/org/apache/cassandra/service/accord/interop/AccordInteropAdapter.java b/src/java/org/apache/cassandra/service/accord/interop/AccordInteropAdapter.java index 001bc9afee52..61de28c26c1e 100644 --- a/src/java/org/apache/cassandra/service/accord/interop/AccordInteropAdapter.java +++ b/src/java/org/apache/cassandra/service/accord/interop/AccordInteropAdapter.java @@ -43,7 +43,7 @@ import accord.topology.Topologies; import org.apache.cassandra.db.ConsistencyLevel; -import org.apache.cassandra.service.accord.AccordEndpointMapper; +import org.apache.cassandra.service.accord.topology.AccordEndpointMapper; import org.apache.cassandra.service.accord.txn.AccordUpdate; import org.apache.cassandra.service.accord.txn.TxnRead; diff --git a/src/java/org/apache/cassandra/service/accord/interop/AccordInteropExecution.java b/src/java/org/apache/cassandra/service/accord/interop/AccordInteropExecution.java index ab6fbb0d88eb..22a0a7932e18 100644 --- a/src/java/org/apache/cassandra/service/accord/interop/AccordInteropExecution.java +++ b/src/java/org/apache/cassandra/service/accord/interop/AccordInteropExecution.java @@ -79,10 +79,10 @@ import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.StorageProxy; -import org.apache.cassandra.service.accord.AccordEndpointMapper; import org.apache.cassandra.service.accord.TokenRange; import org.apache.cassandra.service.accord.api.TokenKey; import org.apache.cassandra.service.accord.serializers.TableMetadatasAndKeys; +import org.apache.cassandra.service.accord.topology.AccordEndpointMapper; import org.apache.cassandra.service.accord.txn.AccordUpdate; import org.apache.cassandra.service.accord.txn.TxnData; import org.apache.cassandra.service.accord.txn.TxnDataKeyValue; diff --git a/src/java/org/apache/cassandra/service/accord/AbstractAccordSegmentCompactor.java b/src/java/org/apache/cassandra/service/accord/journal/AbstractSegmentCompactor.java similarity index 85% rename from src/java/org/apache/cassandra/service/accord/AbstractAccordSegmentCompactor.java rename to src/java/org/apache/cassandra/service/accord/journal/AbstractSegmentCompactor.java index dafee34101b2..f2caf355371a 100644 --- a/src/java/org/apache/cassandra/service/accord/AbstractAccordSegmentCompactor.java +++ b/src/java/org/apache/cassandra/service/accord/journal/AbstractSegmentCompactor.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.journal; import java.io.IOException; import java.util.Collection; @@ -46,8 +46,8 @@ import org.apache.cassandra.journal.StaticSegment; import org.apache.cassandra.journal.StaticSegment.KeyOrderReader; import org.apache.cassandra.service.ClientState; -import org.apache.cassandra.service.accord.AccordJournalValueSerializers.FlyweightImage; -import org.apache.cassandra.service.accord.AccordJournalValueSerializers.FlyweightSerializer; +import org.apache.cassandra.service.accord.AccordKeyspace; +import org.apache.cassandra.service.accord.JournalKey; import org.apache.cassandra.service.accord.serializers.Version; import org.apache.cassandra.utils.BulkIterator; import org.apache.cassandra.utils.NoSpamLogger; @@ -59,9 +59,9 @@ /** * Segment compactor: takes static segments and compacts them into a single SSTable. */ -public abstract class AbstractAccordSegmentCompactor implements SegmentCompactor +public abstract class AbstractSegmentCompactor implements SegmentCompactor { - protected static final Logger logger = LoggerFactory.getLogger(AbstractAccordSegmentCompactor.class); + protected static final Logger logger = LoggerFactory.getLogger(AbstractSegmentCompactor.class); private static final NoSpamLogger.NoSpamLogStatement unknownTable = NoSpamLogger.getStatement(logger, "Unknown (probably dropped) TableId {} reading {}; skipping record", 1L, MINUTES); static final Object[] rowTemplate = BTree.build(BulkIterator.of(new Object[2]), 2, UpdateFunction.noOp); @@ -71,7 +71,7 @@ public abstract class AbstractAccordSegmentCompactor implements SegmentCompac protected final ColumnFamilyStore cfs; protected final long timestamp = ClientState.getTimestamp(); - public AbstractAccordSegmentCompactor(Version userVersion, ColumnFamilyStore cfs) + public AbstractSegmentCompactor(Version userVersion, ColumnFamilyStore cfs) { this.userVersion = userVersion; this.userVersionCell = BufferCell.live(AccordKeyspace.JournalColumns.user_version, timestamp, Int32Type.instance.decompose(userVersion.version)); @@ -120,8 +120,8 @@ public Collection> compact(Collection serializer = null; + Merger merger = null; + MergeSerializer serializer = null; long firstDescriptor = -1, lastDescriptor = -1; int firstOffset = -1, lastOffset = -1; try @@ -131,12 +131,12 @@ public Collection> compact(Collection) key.type.serializer; - builder = serializer.mergerFor(); - builder.reset(key); + serializer = key.type.serializer; + merger = serializer.mergerFor(); + merger.reset(key); firstDescriptor = lastDescriptor = -1; firstOffset = lastOffset = -1; } @@ -146,10 +146,10 @@ public Collection> compact(Collection> compact(Collection> compact(Collection> compact(Collection> compact(Collection serializer, long descriptor, int offset) throws IOException + private void maybeWritePartition(JournalKey key, Merger merger, MergeSerializer serializer, long descriptor, int offset) throws IOException { - if (builder != null) + if (merger != null) { DecoratedKey decoratedKey = AccordKeyspace.JournalColumns.decorate(key); Invariants.requireArgument(prevKey == null || normalize(decoratedKey.compareTo(prevDecoratedKey)) == normalize(JournalKey.SUPPORT.compare(key, prevKey)), @@ -222,7 +222,7 @@ private void maybeWritePartition(JournalKey key, FlyweightImage builder, Flyweig Object[] rowData = rowTemplate.clone(); try (DataOutputBuffer out = DataOutputBuffer.scratchBuffer.get()) { - serializer.reserialize(key, builder, out, userVersion); + serializer.reserialize(key, merger, out, userVersion); rowData[0] = BufferCell.live(AccordKeyspace.JournalColumns.record, timestamp, out.asNewBuffer()); } rowData[1] = userVersionCell; diff --git a/src/java/org/apache/cassandra/service/accord/journal/AccordJournal.java b/src/java/org/apache/cassandra/service/accord/journal/AccordJournal.java new file mode 100644 index 000000000000..f10317f35235 --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/journal/AccordJournal.java @@ -0,0 +1,698 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.service.accord.journal; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.NavigableMap; +import java.util.concurrent.TimeoutException; +import java.util.function.Consumer; +import java.util.function.Supplier; + +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; + +import org.agrona.collections.Long2LongHashMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.impl.CommandChange; +import accord.local.Command; +import accord.local.CommandStore; +import accord.local.CommandStores; +import accord.local.CommandStores.RangesForEpoch; +import accord.local.DurableBefore; +import accord.local.Node; +import accord.local.RedundantBefore; +import accord.primitives.EpochSupplier; +import accord.primitives.Ranges; +import accord.primitives.Timestamp; +import accord.primitives.TxnId; +import accord.utils.Invariants; +import accord.utils.PersistentField; + +import org.apache.cassandra.config.AccordSpec.JournalSpec; +import org.apache.cassandra.config.AccordSpec.JournalSpec.ReplayMode; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.journal.Compactor; +import org.apache.cassandra.journal.Descriptor; +import org.apache.cassandra.journal.Journal; +import org.apache.cassandra.journal.Params; +import org.apache.cassandra.journal.RecordConsumer; +import org.apache.cassandra.journal.RecordPointer; +import org.apache.cassandra.journal.Segments; +import org.apache.cassandra.journal.ValueSerializer; +import org.apache.cassandra.service.accord.AccordKeyspace; +import org.apache.cassandra.service.accord.JournalKey; +import org.apache.cassandra.service.accord.journal.Merger.KeepFirst; +import org.apache.cassandra.service.accord.journal.RangeSearcher.NoopJournalRangeSearcher; +import org.apache.cassandra.service.accord.serializers.Version; +import org.apache.cassandra.utils.CloseableIterator; +import org.apache.cassandra.utils.concurrent.OpOrder; + +import static accord.api.Journal.Load.MINIMAL; +import static accord.api.Journal.Load.MINIMAL_WITH_DEPS; +import static accord.local.Cleanup.Input.FULL; +import static org.apache.cassandra.config.AccordSpec.RangeIndexMode.journal_sai; +import static org.apache.cassandra.config.DatabaseDescriptor.getAccord; +import static org.apache.cassandra.config.DatabaseDescriptor.getAccordJournalDirectory; +import static org.apache.cassandra.service.accord.JournalKey.Type.COMMAND_DIFF; +import static org.apache.cassandra.service.accord.journal.ReplayMarkers.safeStopMarker; +import static org.apache.cassandra.service.accord.journal.ReplayMarkers.startMarker; +import static org.apache.cassandra.service.accord.journal.ReplayMarkers.writeMarker; +import static org.apache.cassandra.service.accord.journal.TopologyRecord.newTopology; + +public class AccordJournal implements accord.api.Journal, RangeSearcher.Supplier +{ + private static final Logger logger = LoggerFactory.getLogger(AccordJournal.class); + + @VisibleForTesting + protected final Journal segments; + protected final ColumnFamilyStore table; + @VisibleForTesting + protected final @Nullable RangeSearchManager rangeSearch; + protected final OpOrder readOrder; + private final Params params; + + public AccordJournal(Params params) + { + this(params, new File(getAccordJournalDirectory()), Keyspace.open(AccordKeyspace.metadata().name).getColumnFamilyStore(AccordKeyspace.JOURNAL)); + } + + @VisibleForTesting + public AccordJournal(Params params, File directory, ColumnFamilyStore table) + { + Version userVersion = Version.fromVersion(params.userVersion()); + this.rangeSearch = RangeSearchManager.ifEnabled(table); + this.table = table; + this.readOrder = table.readOrdering; + this.params = params; + // initialise journal last because we call a self method to initialise its compactor + this.segments = new Journal<>("AccordJournal", directory, params, JournalKey.SUPPORT, + new ValueSerializer.Unsupported<>(), + compactor(table, userVersion), + table.readOrdering); + } + + @Override + public void open(Node node) + { + segments.open(); + } + + public void start(Node node) + { + if (rangeSearch != null) + rangeSearch.start(); + + long maxTableDescriptor = maxTableDescriptor(); + segments.start(maxTableDescriptor); + } + + public Descriptor stop() + { + return segments.stop(); + } + + public void close() + { + segments.close(); + } + + public boolean awaitTerminationUntil(long deadlineNanos) throws InterruptedException + { + try + { + segments.awaitTerminationUntil(deadlineNanos); + return true; + } + catch (TimeoutException e) + { + return false; + } + } + + @Override + public void saveCommand(int commandStoreId, CommandUpdate update, @Nullable Runnable onFlush) + { + CommandChangeWriter change = CommandChangeWriter.make(update.before, update.after); + if (change == null) + { + if (onFlush != null) + onFlush.run(); + return; + } + + JournalKey key = new JournalKey(update.txnId, COMMAND_DIFF, commandStoreId); + RecordPointer pointer = segments.asyncWrite(key, change); + if (rangeSearch != null) + onFlush = merge(onFlush, rangeSearch.maybeIndex(key, pointer, change)); + if (onFlush != null) + segments.onDurable(pointer, onFlush); + } + + void append(JournalKey key, T write, Runnable onFlush) + { + RecordPointer pointer = appendInternal(key, write); + if (onFlush != null) + segments.onDurable(pointer, onFlush); + } + + private RecordPointer appendInternal(JournalKey key, T write) + { + MergeSerializer serializer = (MergeSerializer) key.type.serializer; + return segments.asyncWrite(key, (out, userVersion) -> serializer.serialize(key, write, out, Version.fromVersion(userVersion))); + } + + + public long maxTableDescriptor() + { + return table.getTracker().getView().liveSSTables() + .stream() + .filter(sst -> sst.getSSTableMetadata().totalRows > 0) + .map(sst -> LongType.instance.compose(sst.getSSTableMetadata().coveredClustering.end().bufferAt(0))) + .max(Long::compare).orElse(0L); + } + + public long maxDescriptor() + { + return Math.max(segments.maxDescriptor(), maxTableDescriptor()); + } + + public Params configuration() + { + return params; + } + + @Override + public Command loadCommand(int commandStoreId, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore) + { + CommandChanges builder = load(commandStoreId, txnId); + builder.maybeCleanup(true, FULL, redundantBefore, durableBefore); + return builder.construct(redundantBefore); + } + + @Override + public List debugCommand(int commandStoreId, TxnId txnId) + { + JournalKey key = new JournalKey(txnId, COMMAND_DIFF, commandStoreId); + List result = new ArrayList<>(); + readAll(key, (long segment, int position, JournalKey k, ByteBuffer buffer, int userVersion) -> { + CommandChanges builder = new CommandChanges(txnId); + new RecordConsumerAdapter<>(builder::deserializeNext).accept(segment, position, k, buffer, userVersion); + result.add(new DebugEntry(segment, position, builder)); + }); + return result; + } + + @Override + public Command.Minimal loadMinimal(int commandStoreId, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore) + { + CommandChanges builder = CommandChanges.cleanupAndFilter(loadDiffs(commandStoreId, txnId, MINIMAL), redundantBefore, durableBefore); + return builder == null ? null : builder.asMinimal(); + } + + @Override + public Command.MinimalWithDeps loadMinimalWithDeps(int commandStoreId, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore) + { + CommandChanges builder = CommandChanges.cleanupAndFilter(loadDiffs(commandStoreId, txnId, MINIMAL_WITH_DEPS), redundantBefore, durableBefore); + return builder == null ? null : builder.asMinimalWithDeps(); + } + + private CommandChanges loadDiffs(int commandStoreId, TxnId txnId, Load load) + { + JournalKey key = new JournalKey(txnId, COMMAND_DIFF, commandStoreId); + CommandChanges builder = new CommandChanges(txnId, load); + readAll(key, builder::deserializeNext); + return builder; + } + + @VisibleForTesting + public CommandChanges load(int commandStoreId, TxnId txnId) + { + return loadDiffs(commandStoreId, txnId, Load.ALL); + } + + @Override + public List loadTopologies() + { + List images = new ArrayList<>(); + try (CloseableIterator iter = new CloseableIterator<>() + { + final CloseableIterator> iter = keyIterator(TopologyRecord.journalKey(0L), + TopologyRecord.journalKey(Timestamp.MAX_EPOCH), + true, 0); + TopologyRecord.TopologyImage prev = null; + + @Override + public boolean hasNext() + { + return iter.hasNext(); + } + + @Override + public accord.api.Journal.TopologyUpdate next() + { + Journal.KeyRefs ref = iter.next(); + MergeSerializers.TopologyMerger reader = readAll(ref.key()); + if (reader.read().kind() == TopologyRecord.Kind.Repeat) + { + if (prev == null) + { + logger.error("Encountered TopologyImage Repeat record for epoch {}, but no prior image record was found", ref.key().id.epoch()); + return null; + } + prev = reader.read().asImage(Invariants.nonNull(prev.getUpdate())); + } + else prev = reader.read(); + + return new accord.api.Journal.TopologyUpdate(prev.getUpdate().commandStores, + prev.getUpdate().global); + } + + @Override + public void close() + { + iter.close(); + } + }) + { + accord.api.Journal.TopologyUpdate prev = null; + while (iter.hasNext()) + { + accord.api.Journal.TopologyUpdate next = iter.next(); + if (next == null) + continue; + + Invariants.require(prev == null || next.global.epoch() > prev.global.epoch()); + // Due to partial compaction, we can clean up only some of the old epochs, creating gaps. We skip these epochs here. + if (prev != null && next.global.epoch() > prev.global.epoch() + 1) + images.clear(); + + images.add(next); + prev = next; + } + } + return images; + } + + @Override + public void saveTopology(TopologyUpdate topologyUpdate, Runnable onFlush) + { + append(TopologyRecord.journalKey(topologyUpdate.global.epoch()), + newTopology(topologyUpdate), + onFlush); + } + + @Override + public RedundantBefore loadRedundantBefore(int commandStoreId) + { + KeepFirst accumulator = readLast(new JournalKey(TxnId.NONE, JournalKey.Type.REDUNDANT_BEFORE, commandStoreId)); + return accumulator.get(); + } + + @Override + public NavigableMap loadBootstrapBeganAt(int commandStoreId) + { + KeepFirst> accumulator = readLast(new JournalKey(TxnId.NONE, JournalKey.Type.BOOTSTRAP_BEGAN_AT, commandStoreId)); + return accumulator.get(); + } + + @Override + public NavigableMap loadSafeToRead(int commandStoreId) + { + KeepFirst> accumulator = readLast(new JournalKey(TxnId.NONE, JournalKey.Type.SAFE_TO_READ, commandStoreId)); + return accumulator.get(); + } + + @Override + public CommandStores.RangesForEpoch loadRangesForEpoch(int commandStoreId) + { + KeepFirst accumulator = readLast(new JournalKey(TxnId.NONE, JournalKey.Type.RANGES_FOR_EPOCH, commandStoreId)); + return accumulator.get(); + } + + @Override + public PersistentField.Persister durableBeforePersister() + { + return new DurableBeforePersister(this); + } + + @Override + public void saveStoreState(int commandStoreId, FieldUpdates fieldUpdates, Runnable onFlush) + { + RecordPointer pointer = null; + if (fieldUpdates.newRedundantBefore != null) + pointer = appendInternal(new JournalKey(TxnId.NONE, JournalKey.Type.REDUNDANT_BEFORE, commandStoreId), fieldUpdates.newRedundantBefore); + if (fieldUpdates.newBootstrapBeganAt != null) + pointer = appendInternal(new JournalKey(TxnId.NONE, JournalKey.Type.BOOTSTRAP_BEGAN_AT, commandStoreId), fieldUpdates.newBootstrapBeganAt); + if (fieldUpdates.newSafeToRead != null) + pointer = appendInternal(new JournalKey(TxnId.NONE, JournalKey.Type.SAFE_TO_READ, commandStoreId), fieldUpdates.newSafeToRead); + if (fieldUpdates.newRangesForEpoch != null) + pointer = appendInternal(new JournalKey(TxnId.NONE, JournalKey.Type.RANGES_FOR_EPOCH, commandStoreId), fieldUpdates.newRangesForEpoch); + + if (onFlush == null) + return; + + if (pointer != null) + segments.onDurable(pointer, onFlush); + else + onFlush.run(); + } + + public BUILDER readAll(JournalKey key) + { + Invariants.require(segments.isReadable()); + BUILDER builder = (BUILDER) key.type.serializer.mergerFor(); + builder.reset(key); + // TODO (expected): this can be further improved to avoid allocating lambdas + MergeSerializer serializer = (MergeSerializer) key.type.serializer; + // TODO (expected): for those where we store an image, read only the first entry we find in DESC order + readAll(key, (in, userVersion) -> serializer.deserialize(key, builder, in, userVersion)); + return builder; + } + + public BUILDER readLast(JournalKey key) + { + Invariants.require(segments.isReadable()); + BUILDER builder = (BUILDER) key.type.serializer.mergerFor(); + builder.reset(key); + // TODO (expected): this can be further improved to avoid allocating lambdas + MergeSerializer serializer = (MergeSerializer) key.type.serializer; + readLast(key, (in, userVersion) -> serializer.deserialize(key, builder, in, userVersion)); + return builder; + } + + public void forEachEntry(JournalKey key, Reader reader) + { + readAll(key, reader); + } + + public interface Reader + { + void read(DataInputPlus input, Version userVersion) throws IOException; + + default void read(ByteBuffer buffer, Version userVersion) + { + try (DataInputBuffer in = new DataInputBuffer(buffer, false)) + { + read(in, userVersion); + } + catch (IOException e) + { + // can only throw if serializer is buggy or bytes got corrupted + throw new UncheckedIOException(e); + } + } + } + + static class RecordConsumerAdapter implements RecordConsumer + { + protected final Reader reader; + private long prevSegment = Long.MAX_VALUE; + private long prevPosition = Long.MAX_VALUE; + + RecordConsumerAdapter(Reader reader) + { + this.reader = reader; + } + + @Override + public void accept(long segment, int position, K key, ByteBuffer buffer, int userVersion) + { + Invariants.require(segment <= prevSegment, + "Records should always be iterated over in a reverse order, but segment %d was seen after %d while reading %s", segment, prevSegment, key); + Invariants.require(segment != prevSegment || position < prevPosition, + "Records should always be iterated over in a reverse order, but position %d was seen after %d for segment %d while reading %s", position, prevPosition, segment, key); + reader.read(buffer, Version.fromVersion(userVersion)); + prevSegment = segment; + prevPosition = position; + } + } + + /** + * Perform a read from Journal table, followed by the reads from all journal segments. + *

+ * When reading from journal segments, skip descriptors that were read from the table. + */ + public void readAll(JournalKey key, Reader reader) + { + readAll(key, new RecordConsumerAdapter<>(reader)); + } + + public void readAll(JournalKey key, RecordConsumer reader) + { + try (OpOrder.Group readOrder = table.readOrdering.start()) + { + // SELECT segments first, to avoid missing segments due to races compacting segment->sstable + Segments segments = this.segments.segments(); + try (TableRecordIterator table = TableRecordIterator.all(this.table, key, readOrder)) + { + boolean hasTableData = table.advance(); + long minSegment = hasTableData ? table.segment : Long.MIN_VALUE; + // First, read all journal entries newer than anything flushed into sstables + Journal.readAll(key, (segment, position, key1, buffer, userVersion) -> { + if (segment > minSegment) + reader.accept(segment, position, key1, buffer, userVersion); + }, readOrder, segments); + + // Then, read SSTables + while (hasTableData) + { + reader.accept(table.segment, table.offset, key, table.value, table.userVersion); + hasTableData = table.advance(); + } + } + } + } + + public void readLast(JournalKey key, Reader reader) + { + readLast(key, new RecordConsumerAdapter<>(reader)); + } + + public void readLast(JournalKey key, RecordConsumer reader) + { + try (OpOrder.Group readOrder = table.readOrdering.start()) + { + Segments segments = this.segments.segments(); + try (TableRecordIterator table = TableRecordIterator.all(this.table, key, readOrder)) + { + boolean hasTableData = table.advance(); + long minSegment = hasTableData ? table.segment : Long.MIN_VALUE; + + class JournalReader implements RecordConsumer + { + boolean read; + @Override + public void accept(long segment, int position, JournalKey key, ByteBuffer buffer, int userVersion) + { + if (segment > minSegment) + { + reader.accept(segment, position, key, buffer, userVersion); + read = true; + } + } + } + + // First, read all journal entries newer than anything flushed into sstables + JournalReader journalReader = new JournalReader(); + Journal.readLast(key, journalReader, readOrder, segments); + + // Then, read SSTables, if we haven't found a record already + if (hasTableData && !journalReader.read) + reader.accept(table.segment, table.offset, key, table.value, table.userVersion); + } + } + } + + public CloseableIterator> keyIterator(@Nullable JournalKey min, @Nullable JournalKey max, boolean includeActive, long minSegment) + { + // the readOrder is taken only to ensure we get a consistent snapshot of both segments and sstables; + // we take references to the segments and sstables directly, and do not need to manage the readOrder for the lifetime of the iterator + try (OpOrder.Group readOrder = this.readOrder.start()) + { + return new TableAndSegmentKeyIterator<>(segments, table, min, max, includeActive, minSegment); + } + } + + public void forEach(Consumer consumer, boolean includeActive, long minSegment) + { + forEach(consumer, null, null, includeActive, minSegment); + } + + public void forEach(Consumer consumer, @Nullable JournalKey min, @Nullable JournalKey max, boolean includeActive, long minSegment) + { + try (CloseableIterator> iter = keyIterator(min, max, includeActive, minSegment)) + { + while (iter.hasNext()) + { + Journal.KeyRefs ref = iter.next(); + consumer.accept(ref.key()); + } + } + } + + public Compactor compactor() + { + return segments.compactor(); + } + + protected org.apache.cassandra.journal.SegmentCompactor compactor(ColumnFamilyStore cfs, Version userVersion) + { + if (rangeSearch == null) + { + Invariants.require(getAccord().range_index_mode != journal_sai, "range_index_mode is journal_sai, but the storage attached index was not found on initialisation"); + return new SegmentCompactor<>(userVersion, cfs); + } + + return rangeSearch.compactor(cfs, userVersion); + } + + public void forceCompaction() + { + table.forceMajorCompaction(); + } + + @Override + public void purge(CommandStores commandStores, EpochSupplier minEpoch) + { + segments.closeCurrentSegmentForTestingIfNonEmpty(); + segments.runCompactorForTesting(); + forceCompaction(); + } + + public void replay(CommandStore commandStore, ReplayMode replayMode, long minSegmentId) + { + Long2LongHashMap minSegments = new Long2LongHashMap(0); + minSegments.put(commandStore.id(), minSegmentId); + Replay.replay(this, replayMode, new CommandStore[] {commandStore }, minSegments); + } + + @Override + public boolean replay(CommandStores commandStores, Object param) + { + ReplayMode mode = params instanceof JournalSpec ? ((JournalSpec)params).replay + : getAccord().journal.replay; + return Replay.replay(this, mode, commandStores.all(), param); + } + + @Override + public RangeSearcher rangeSearcher() + { + if (rangeSearch == null) + return NoopJournalRangeSearcher.instance; + return rangeSearch.rangeSearcher(); + } + + public void writeStartMarker() + { + writeMarker(startMarker(), segments.peekSegmentId()); + } + + public void writeSafeStopMarker() + { + segments.fsync(); + writeMarker(safeStopMarker(), segments.peekSegmentId()); + } + + private static Runnable merge(Runnable first, Runnable second) + { + if (first == null) return second; + if (second == null) return first; + return () -> + { + try { first.run(); } + finally { second.run(); } + }; + } + + public static class DebugEntry implements Supplier + { + public final long segment; + public final int position; + public final CommandChanges builder; + + public DebugEntry(long segment, int position, CommandChanges builder) + { + this.segment = segment; + this.position = position; + this.builder = builder; + } + + @Override + public CommandChange.Builder get() + { + return builder; + } + } + + @VisibleForTesting + public Journal unsafeGetJournal() + { + return segments; + } + + @VisibleForTesting + public void closeCurrentSegmentForTestingIfNonEmpty() + { + segments.closeCurrentSegmentForTestingIfNonEmpty(); + } + + public void sanityCheck(int commandStoreId, RedundantBefore redundantBefore, Command orig) + { + CommandChanges builder = load(commandStoreId, orig.txnId()); + builder.forceResult(orig.result()); + // We can only use strict equality if we supply result. + Command reconstructed = builder.construct(redundantBefore); + Invariants.require(orig.equals(reconstructed), + '\n' + + "Original: %s\n" + + "Reconstructed: %s\n" + + "Diffs: %s", orig, reconstructed, builder); + } + + @VisibleForTesting + public void truncateForTesting() + { + segments.truncateForTesting(); + if (rangeSearch != null) + rangeSearch.safeNotify(SegmentRangeSearcher::truncateForTesting); + } + + @VisibleForTesting + public void runCompactorForTesting() + { + segments.runCompactorForTesting(); + } + + @VisibleForTesting + public int inMemorySize() + { + return segments.currentActiveSegment().index().size(); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/accord/journal/CommandChangeWriter.java b/src/java/org/apache/cassandra/service/accord/journal/CommandChangeWriter.java new file mode 100644 index 000000000000..ec8fecfaf4be --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/journal/CommandChangeWriter.java @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.accord.journal; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import javax.annotation.Nullable; + +import accord.impl.CommandChange; +import accord.local.Cleanup; +import accord.local.Command; +import accord.primitives.SaveStatus; +import accord.utils.Invariants; +import accord.utils.UnhandledEnum; + +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.journal.Journal; +import org.apache.cassandra.service.accord.serializers.CommandSerializers; +import org.apache.cassandra.service.accord.serializers.DepsSerializers; +import org.apache.cassandra.service.accord.serializers.ResultSerializers; +import org.apache.cassandra.service.accord.serializers.Version; +import org.apache.cassandra.service.accord.serializers.WaitingOnSerializer; + +import static accord.impl.CommandChange.anyFieldChanged; +import static accord.impl.CommandChange.describeFlags; +import static accord.impl.CommandChange.getFlags; +import static accord.impl.CommandChange.isNull; +import static accord.impl.CommandChange.nextSetField; +import static accord.impl.CommandChange.toIterableSetFields; +import static accord.impl.CommandChange.unsetIterable; +import static accord.impl.CommandChange.validateFlags; + +public class CommandChangeWriter implements Journal.Writer +{ + final Command after; + final int flags; + + private CommandChangeWriter(Command after, int flags) + { + this.after = after; + this.flags = flags; + } + + public static CommandChangeWriter make(Command before, Command after) + { + if (before == after + || after == null + || after.saveStatus() == SaveStatus.Uninitialised) + return null; + + int flags = validateFlags(getFlags(before, after)); + if (!anyFieldChanged(flags)) + return null; + + return new CommandChangeWriter(after, flags); + } + + @Override + public void write(DataOutputPlus out, int userVersion) throws IOException + { + write(out, Version.fromVersion(userVersion)); + } + + public void write(DataOutputPlus out, Version userVersion) throws IOException + { + serialize(after, flags, out, userVersion); + } + + private static void serialize(Command command, int flags, DataOutputPlus out, Version userVersion) throws IOException + { + Invariants.require(flags != 0); + out.writeInt(flags); + + int iterable = toIterableSetFields(flags); + while (iterable != 0) + { + CommandChange.Field field = nextSetField(iterable); + if (isNull(field, flags)) + { + iterable = unsetIterable(field, iterable); + continue; + } + + switch (field) + { + case EXECUTE_AT: + CommandSerializers.ExecuteAtSerializer.serialize(command.txnId(), command.executeAt(), out); + break; + case EXECUTES_AT_LEAST: + CommandSerializers.ExecuteAtSerializer.serialize(command.executesAtLeast(), out); + break; + case MIN_UNIQUE_HLC: + Invariants.require(command.waitingOn().minUniqueHlc() != 0); + out.writeUnsignedVInt(command.waitingOn().minUniqueHlc()); + break; + case SAVE_STATUS: + out.writeByte(command.saveStatus().ordinal()); + break; + case DURABILITY: + out.writeByte(command.durability().encoded()); + break; + case ACCEPTED: + CommandSerializers.ballot.serialize(command.acceptedOrCommitted(), out); + break; + case PROMISED: + CommandSerializers.ballot.serialize(command.promised(), out); + break; + case PARTICIPANTS: + CommandSerializers.participants.serialize(command.participants(), out); + break; + case PARTIAL_TXN: + CommandSerializers.partialTxn.serialize(command.partialTxn(), out, userVersion); + break; + case PARTIAL_DEPS: + DepsSerializers.partialDepsById.serialize(command.partialDeps(), out); + break; + case WAITING_ON: + Command.WaitingOn waitingOn = command.waitingOn(); + WaitingOnSerializer.serializeBitSetsOnly(command.txnId(), waitingOn, out); + break; + case WRITES: + CommandSerializers.writes.serialize(command.writes(), out, userVersion); + break; + case RESULT: + ResultSerializers.result.serialize(command.result(), out); + break; + case CLEANUP: + Cleanup cleanup; + switch (command.saveStatus()) + { + default: + throw new UnhandledEnum(command.saveStatus()); + case Erased: + cleanup = Cleanup.ERASE; + break; + case Invalidated: + cleanup = Cleanup.INVALIDATE; + break; + } + out.writeByte(cleanup.ordinal()); + break; + } + + iterable = unsetIterable(field, iterable); + } + } + + private boolean hasField(CommandChange.Field fields) + { + return !isNull(fields, flags); + } + + public boolean hasParticipants() + { + return hasField(CommandChange.Field.PARTICIPANTS); + } + + @Override + public String toString() + { + return after.saveStatus() + " " + describeFlags(flags); + } + + public static @Nullable ByteBuffer asSerializedChange(Command before, Command after, Version userVersion) throws IOException + { + // TODO (expected): reusable buffer to build, or pre-size + try (DataOutputBuffer out = new DataOutputBuffer()) + { + CommandChangeWriter writer = CommandChangeWriter.make(before, after); + if (writer == null) + return null; + + writer.write(out, userVersion); + return out.asNewBuffer(); + } + } +} diff --git a/src/java/org/apache/cassandra/service/accord/journal/CommandChanges.java b/src/java/org/apache/cassandra/service/accord/journal/CommandChanges.java new file mode 100644 index 000000000000..c0246f25ddb0 --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/journal/CommandChanges.java @@ -0,0 +1,363 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.accord.journal; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; + +import accord.api.Journal; +import accord.impl.CommandChange; +import accord.local.Cleanup; +import accord.local.DurableBefore; +import accord.local.RedundantBefore; +import accord.primitives.PartialDeps; +import accord.primitives.SaveStatus; +import accord.primitives.Status; +import accord.primitives.TxnId; +import accord.utils.Invariants; +import accord.utils.UnhandledEnum; + +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.service.accord.JournalKey; +import org.apache.cassandra.service.accord.serializers.CommandSerializers; +import org.apache.cassandra.service.accord.serializers.DepsSerializers; +import org.apache.cassandra.service.accord.serializers.ResultSerializers; +import org.apache.cassandra.service.accord.serializers.Version; +import org.apache.cassandra.service.accord.serializers.WaitingOnSerializer; + +import static accord.api.Journal.Load.ALL; +import static accord.impl.CommandChange.Field.CLEANUP; +import static accord.impl.CommandChange.isChanged; +import static accord.impl.CommandChange.nextSetField; +import static accord.impl.CommandChange.toIterableNonNullFields; +import static accord.impl.CommandChange.toIterableSetFields; +import static accord.impl.CommandChange.unsetIterable; +import static accord.impl.CommandChange.validateFlags; +import static accord.local.Cleanup.Input.FULL; + +public class CommandChanges extends CommandChange.Builder implements Merger +{ + private final boolean deserializeDeps; + + public CommandChanges() + { + this(Journal.Load.ALL); + } + + public CommandChanges(Journal.Load load) + { + this(null, load); + } + + public CommandChanges(TxnId txnId) + { + this(txnId, Journal.Load.ALL); + } + + public CommandChanges(TxnId txnId, Journal.Load load) + { + super(txnId, load); + deserializeDeps = load == ALL; + } + + // applies cleanup and returns null if no command should be returned + public static CommandChanges cleanupAndFilter(CommandChanges builder, RedundantBefore redundantBefore, DurableBefore durableBefore) + { + if (builder.isEmpty()) + return null; + + Cleanup cleanup = builder.shouldCleanup(FULL, redundantBefore, durableBefore); + switch (cleanup) + { + case VESTIGIAL: + case EXPUNGE: + case ERASE: + return null; + } + Invariants.require(builder.saveStatus() != null, "No saveSatus loaded, but next was called and cleanup was not: %s", builder); + return builder; + } + + @Override + public PartialDeps partialDeps() + { + if (partialDeps instanceof ByteBuffer) + { + try + { + partialDeps = DepsSerializers.partialDepsById.deserialize((ByteBuffer) partialDeps); + } + catch (IOException e) + { + throw new IllegalStateException("Failed to materialise partially deserialised deps", e); + } + } + return (PartialDeps) partialDeps; + } + + public void reset(JournalKey key) + { + reset(key.id); + } + + public ByteBuffer asByteBuffer(Version userVersion) throws IOException + { + try (DataOutputBuffer out = new DataOutputBuffer()) + { + serialize(out, userVersion); + return out.asNewBuffer(); + } + } + + public void serialize(DataOutputPlus out, Version userVersion) throws IOException + { + Invariants.require(mask == 0); + Invariants.require(flags != 0); + + int flags = validateFlags(this.flags); + serialize(flags, out, userVersion); + } + + private void serialize(int flags, DataOutputPlus out, Version userVersion) throws IOException + { + Invariants.require(flags != 0); + out.writeInt(flags); + + int iterable = toIterableNonNullFields(flags); + for (CommandChange.Field field = nextSetField(iterable); field != null; iterable = unsetIterable(field, iterable), field = nextSetField(iterable)) + { + switch (field) + { + default: + throw new UnhandledEnum(field); + case CLEANUP: + out.writeByte(cleanup.ordinal()); + break; + case EXECUTE_AT: + Invariants.require(txnId != null, "%s", this); + Invariants.require(executeAt != null, "%s", this); + CommandSerializers.ExecuteAtSerializer.serialize(txnId, executeAt, out); + break; + case EXECUTES_AT_LEAST: + Invariants.require(executesAtLeast != null); + CommandSerializers.ExecuteAtSerializer.serialize(executesAtLeast, out); + break; + case MIN_UNIQUE_HLC: + Invariants.require(minUniqueHlc != 0, "%s", this); + out.writeUnsignedVInt(minUniqueHlc); + break; + case SAVE_STATUS: + Invariants.require(saveStatus != null, "%s", this); + out.writeByte(saveStatus.ordinal()); + break; + case DURABILITY: + Invariants.require(durability != null, "%s", this); + out.writeByte(durability.encoded()); + break; + case ACCEPTED: + Invariants.require(acceptedOrCommitted != null, "%s", this); + CommandSerializers.ballot.serialize(acceptedOrCommitted, out); + break; + case PROMISED: + Invariants.require(promised != null, "%s", this); + CommandSerializers.ballot.serialize(promised, out); + break; + case PARTICIPANTS: + Invariants.require(participants != null, "%s", this); + CommandSerializers.participants.serialize(participants, out); + break; + case PARTIAL_TXN: + Invariants.require(partialTxn != null, "%s", this); + CommandSerializers.partialTxn.serialize(partialTxn, out, userVersion); + break; + case PARTIAL_DEPS: + Invariants.require(partialDeps != null, "%s", this); + if (partialDeps instanceof ByteBuffer) out.write(((ByteBuffer) partialDeps).duplicate()); + else DepsSerializers.partialDepsById.serialize((PartialDeps) partialDeps, out); + break; + case WAITING_ON: + Invariants.require(waitingOn != null, "%s", this); + ((WaitingOnSerializer.WaitingOnBitSetsAndLength) waitingOn).reserialize(out); + break; + case WRITES: + Invariants.require(writes != null, "%s", this); + CommandSerializers.writes.serialize(writes, out, userVersion); + break; + case RESULT: + Invariants.require(result != null, "%s", this); + ResultSerializers.result.serialize(result, out); + break; + } + } + } + + public void deserializeNext(ByteBuffer buffer, Version userVersion) + { + try (DataInputBuffer in = new DataInputBuffer(buffer, false)) + { + deserializeNext(in, userVersion); + } + catch (IOException e) + { + throw new UncheckedIOException(e); + } + } + + public void deserializeNext(DataInputPlus in, Version userVersion) throws IOException + { + Invariants.require(txnId != null); + int readFlags = in.readInt(); + Invariants.require(readFlags != 0); + hasUpdate = true; + count++; + + // batch-apply any new nulls + setNulls(false, readFlags); + // iterator sets low 16 bits; low readFlag bits are nulls, so masking with ~readFlags restricts to non-null changed fields + int iterable = toIterableSetFields(readFlags) & ~readFlags; + for (CommandChange.Field field = nextSetField(iterable); field != null; field = nextSetField(iterable = unsetIterable(field, iterable))) + { + // Since we are iterating in reverse order, we skip the fields that were + // set by entries written later (i.e. already read ones). + if (isChanged(field, flags | mask) && field != CLEANUP) + skip(txnId, field, in, userVersion); + else + deserialize(field, in, userVersion); + } + + // upper 16 bits are changed flags, lower are nulls; by masking upper by ~lower we restrict to only non-null changed fields + this.flags |= readFlags & (~readFlags << 16); + } + + private void deserialize(CommandChange.Field field, DataInputPlus in, Version userVersion) throws IOException + { + switch (field) + { + case EXECUTE_AT: + executeAt = CommandSerializers.ExecuteAtSerializer.deserialize(txnId, in); + break; + case EXECUTES_AT_LEAST: + executesAtLeast = CommandSerializers.ExecuteAtSerializer.deserialize(in); + break; + case MIN_UNIQUE_HLC: + minUniqueHlc = in.readUnsignedVInt(); + break; + case SAVE_STATUS: + saveStatus = SaveStatus.values()[in.readByte()]; + break; + case DURABILITY: + durability = Status.Durability.forEncoded(in.readUnsignedByte()); + break; + case ACCEPTED: + acceptedOrCommitted = CommandSerializers.ballot.deserialize(in); + break; + case PROMISED: + promised = CommandSerializers.ballot.deserialize(in); + break; + case PARTICIPANTS: + participants = CommandSerializers.participants.deserialize(in); + break; + case PARTIAL_TXN: + partialTxn = CommandSerializers.partialTxn.deserialize(in, userVersion); + break; + case PARTIAL_DEPS: + // TODO (expected): this optimisation will be easily disabled; + // should either operate natively on ByteBuffer + // or else use some explicit API for copying bytes while skipping + if (deserializeDeps || !(in instanceof DataInputBuffer)) + { + partialDeps = DepsSerializers.partialDepsById.deserialize(in); + } + else + { + ByteBuffer buf = ((DataInputBuffer) in).buffer(); + int start = buf.position(); + DepsSerializers.partialDepsById.skip(in); + int end = buf.position(); + partialDeps = buf.duplicate().position(start).limit(end); + } + break; + case WAITING_ON: + waitingOn = WaitingOnSerializer.deserializeBitSets(txnId, in); + break; + case WRITES: + writes = CommandSerializers.writes.deserialize(in, userVersion); + break; + case CLEANUP: + Cleanup newCleanup = Cleanup.forOrdinal(in.readByte()); + if (cleanup == null || newCleanup.compareTo(cleanup) > 0) + cleanup = newCleanup; + break; + case RESULT: + result = ResultSerializers.result.deserialize(in); + break; + } + } + + private static void skip(TxnId txnId, CommandChange.Field field, DataInputPlus in, Version userVersion) throws IOException + { + switch (field) + { + default: + throw new UnhandledEnum(field); + case EXECUTE_AT: + CommandSerializers.ExecuteAtSerializer.skip(txnId, in); + break; + case EXECUTES_AT_LEAST: + CommandSerializers.ExecuteAtSerializer.skip(in); + break; + case MIN_UNIQUE_HLC: + in.readUnsignedVInt(); + break; + case SAVE_STATUS: + case DURABILITY: + case CLEANUP: + in.readByte(); + break; + case ACCEPTED: + case PROMISED: + CommandSerializers.ballot.skip(in); + break; + case PARTICIPANTS: + CommandSerializers.participants.skip(in); + break; + case PARTIAL_TXN: + CommandSerializers.partialTxn.skip(in, userVersion); + break; + case PARTIAL_DEPS: + DepsSerializers.partialDepsById.skip(in); + break; + case WAITING_ON: + WaitingOnSerializer.skip(txnId, in); + break; + case WRITES: + // TODO (expected): skip + CommandSerializers.writes.skip(in, userVersion); + break; + case RESULT: + // TODO (expected): skip + ResultSerializers.result.skip(in); + break; + } + } +} diff --git a/src/java/org/apache/cassandra/service/accord/journal/DurableBeforePersister.java b/src/java/org/apache/cassandra/service/accord/journal/DurableBeforePersister.java new file mode 100644 index 000000000000..43aa096d6b47 --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/journal/DurableBeforePersister.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.accord.journal; + +import accord.local.DurableBefore; +import accord.primitives.TxnId; +import accord.utils.PersistentField; +import accord.utils.async.AsyncResult; +import accord.utils.async.AsyncResults; + +import org.apache.cassandra.service.accord.JournalKey; + +public class DurableBeforePersister implements PersistentField.Persister +{ + private static final JournalKey JOURNAL_KEY = new JournalKey(TxnId.NONE, JournalKey.Type.DURABLE_BEFORE, 0); + + final AccordJournal journal; + + public DurableBeforePersister(AccordJournal journal) + { + this.journal = journal; + } + + @Override + public AsyncResult persist(DurableBefore addValue, DurableBefore newValue) + { + AsyncResult.Settable result = AsyncResults.settable(); + journal.append(JOURNAL_KEY, addValue, () -> result.setSuccess(null)); + return result; + } + + @Override + public DurableBefore load() + { + MergeSerializers.DurableBeforeMerger accumulator = journal.readAll(JOURNAL_KEY); + return accumulator.get(); + } +}; \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/accord/IndexRange.java b/src/java/org/apache/cassandra/service/accord/journal/IndexedRange.java similarity index 87% rename from src/java/org/apache/cassandra/service/accord/IndexRange.java rename to src/java/org/apache/cassandra/service/accord/journal/IndexedRange.java index 426dde5a5b6e..f0bb4164f78b 100644 --- a/src/java/org/apache/cassandra/service/accord/IndexRange.java +++ b/src/java/org/apache/cassandra/service/accord/journal/IndexedRange.java @@ -16,26 +16,26 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.journal; import org.apache.cassandra.cache.IMeasurableMemory; import org.apache.cassandra.utils.ByteArrayUtil; import org.apache.cassandra.utils.ObjectSizes; -public class IndexRange implements Comparable, IMeasurableMemory +class IndexedRange implements Comparable, IMeasurableMemory { - private static final long EMPTY_SIZE = ObjectSizes.measure(new IndexRange(null, null)); + private static final long EMPTY_SIZE = ObjectSizes.measure(new IndexedRange(null, null)); public final byte[] start, end; - public IndexRange(byte[] start, byte[] end) + public IndexedRange(byte[] start, byte[] end) { this.start = start; this.end = end; } @Override - public int compareTo(IndexRange other) + public int compareTo(IndexedRange other) { int rc = ByteArrayUtil.compareUnsigned(start, 0, other.start, 0, start.length); if (rc == 0) diff --git a/src/java/org/apache/cassandra/service/accord/JournalRangeIndex.java b/src/java/org/apache/cassandra/service/accord/journal/JournalRangeIndex.java similarity index 95% rename from src/java/org/apache/cassandra/service/accord/JournalRangeIndex.java rename to src/java/org/apache/cassandra/service/accord/journal/JournalRangeIndex.java index 5b6444eca50a..ca43832657af 100644 --- a/src/java/org/apache/cassandra/service/accord/JournalRangeIndex.java +++ b/src/java/org/apache/cassandra/service/accord/journal/JournalRangeIndex.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.journal; import java.util.Map; import java.util.concurrent.CancellationException; @@ -54,7 +54,12 @@ import accord.utils.btree.BTree; import accord.utils.btree.IntervalBTree; +import org.apache.cassandra.service.accord.AccordCache; +import org.apache.cassandra.service.accord.AccordCacheEntry; +import org.apache.cassandra.service.accord.AccordCommandStore; import org.apache.cassandra.service.accord.AccordCommandStore.Caches; +import org.apache.cassandra.service.accord.RangeIndex; +import org.apache.cassandra.service.accord.TokenRange; import org.apache.cassandra.service.accord.api.TokenKey; import static accord.local.CommandSummaries.Relevance.IRRELEVANT; @@ -194,7 +199,7 @@ public void load(Map into, BooleanSupplier abort) } @Override - void finish(Map into) + protected void finish(Map into) { } @@ -228,7 +233,7 @@ boolean isMaybeRelevant(TxnIdInterval txnIdInterval) } @Override - void cleanupExclusive(Caches caches) + protected void cleanupExclusive(Caches caches) { if (commandWatcher != null) { @@ -239,7 +244,7 @@ void cleanupExclusive(Caches caches) } @Override - AccordCommandStore commandStore() + protected AccordCommandStore commandStore() { return owner.commandStore; } @@ -247,7 +252,7 @@ AccordCommandStore commandStore() private final AccordCommandStore commandStore; // TODO (expected): do we need one of these per command store? - private final JournalRangeSearcher searcher; + private final RangeSearcher searcher; private final Object2ObjectHashMap cachedRangeTxnsById = new Object2ObjectHashMap<>(); public JournalRangeIndex(AccordCommandStore commandStore) @@ -258,7 +263,7 @@ public JournalRangeIndex(AccordCommandStore commandStore) { caches.commands().register(this); } - this.searcher = JournalRangeSearcher.extractRangeSearcher(commandStore.journal); + this.searcher = RangeSearcher.extractRangeSearcher(commandStore.journal); } @Override @@ -366,16 +371,6 @@ public JournalRangeIndex.Loader loader(TxnId primaryTxnId, Timestamp primaryExec return SummaryLoader.loader(redundantBefore, maxDecidedRX, primaryTxnId, primaryExecuteAt, loadKeysFor, keysOrRanges, this::newLoader); } - @Override - public void update(Command prev, Command updated, boolean force) - { - } - - @Override - public void postReplay() - { - } - private Loader newLoader(RedundantBefore redundantBefore, MaxDecidedRX maxDecidedRX, @Nullable TxnId primaryTxnId, Unseekables searchKeysOrRanges, Kinds testKind, TxnId minTxnId, Timestamp maxTxnId, LoadKeysFor loadKeysFor) { return new Loader(this, redundantBefore, maxDecidedRX, primaryTxnId, searchKeysOrRanges, testKind, minTxnId, maxTxnId, loadKeysFor); diff --git a/src/java/org/apache/cassandra/service/accord/journal/MergeSerializer.java b/src/java/org/apache/cassandra/service/accord/journal/MergeSerializer.java new file mode 100644 index 000000000000..b22a31341909 --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/journal/MergeSerializer.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.accord.journal; + +import java.io.IOException; + +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.service.accord.JournalKey; +import org.apache.cassandra.service.accord.serializers.Version; + +public interface MergeSerializer +{ + M mergerFor(); + + void deserialize(JournalKey key, DeserializeInto into, DataInputPlus in, Version userVersion) throws IOException; + + default M deserialize(JournalKey key, DataInputPlus in, Version userVersion) throws IOException + { + M builder = mergerFor(); + deserialize(key, builder, in, userVersion); + return builder; + } + + void serialize(JournalKey key, V from, DataOutputPlus out, Version userVersion) throws IOException; + + void reserialize(JournalKey key, M from, DataOutputPlus out, Version userVersion) throws IOException; + + +} diff --git a/src/java/org/apache/cassandra/service/accord/journal/MergeSerializers.java b/src/java/org/apache/cassandra/service/accord/journal/MergeSerializers.java new file mode 100644 index 000000000000..74b00f740cf3 --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/journal/MergeSerializers.java @@ -0,0 +1,343 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.accord.journal; + +import java.io.IOException; +import java.util.NavigableMap; +import java.util.Objects; + +import com.google.common.collect.ImmutableSortedMap; + +import accord.local.DurableBefore; +import accord.local.RedundantBefore; +import accord.primitives.Ranges; +import accord.primitives.Timestamp; +import accord.primitives.TxnId; +import accord.utils.Invariants; + +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.service.accord.JournalKey; +import org.apache.cassandra.service.accord.journal.Merger.KeepFirst; +import org.apache.cassandra.service.accord.journal.Merger.SimpleMerger; +import org.apache.cassandra.service.accord.serializers.CommandStoreSerializers; +import org.apache.cassandra.service.accord.serializers.Version; + +import static accord.local.CommandStores.RangesForEpoch; + +public class MergeSerializers +{ + public static class CommandChangeSerializer implements MergeSerializer + { + @Override + public CommandChanges mergerFor() + { + return new CommandChanges(); + } + + @Override + public void serialize(JournalKey key, CommandChangeWriter writer, DataOutputPlus out, Version userVersion) + { + try + { + writer.write(out, userVersion); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + } + + @Override + public void reserialize(JournalKey key, CommandChanges from, DataOutputPlus out, Version userVersion) throws IOException + { + from.serialize(out, + // In CompactionIterator, we are dealing with relatively recent records, so we do not pass redundant before here. + // However, we do on load and during Journal SSTable compaction. + userVersion); + } + + @Override + public void deserialize(JournalKey journalKey, CommandChanges into, DataInputPlus in, Version userVersion) throws IOException + { + into.deserializeNext(in, userVersion); + } + } + + public static class RedundantBeforeSerializer + implements MergeSerializer, + SimpleMerger> + { + @Override + public KeepFirst mergerFor() + { + return new KeepFirst<>(RedundantBefore.EMPTY); + } + + @Override + public void serialize(JournalKey key, RedundantBefore entry, DataOutputPlus out, Version userVersion) + { + try + { + if (entry == RedundantBefore.EMPTY) + { + // I am fairly sure this branch was to paper over a bug in the RedundantBefore serializer; it should now be defunct + out.writeInt(0); + return; + } + out.writeInt(1); + CommandStoreSerializers.redundantBefore.serialize(entry, out); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + } + + @Override + public void reserialize(JournalKey key, SimpleMerger from, DataOutputPlus out, Version userVersion) throws IOException + { + serialize(key, from.get(), out, userVersion); + } + + @Override + public void deserialize(JournalKey journalKey, SimpleMerger into, DataInputPlus in, Version userVersion) throws IOException + { + if (in.readInt() == 0) + { + into.update(RedundantBefore.EMPTY); + return; + } + into.update(CommandStoreSerializers.redundantBefore.deserialize(in)); + } + } + + public static class DurableBeforeMerger extends SimpleMerger + { + public DurableBeforeMerger() + { + super(DurableBefore.EMPTY); + } + + @Override + public void reset(JournalKey key) + { + accumulated = DurableBefore.EMPTY; + } + + @Override + protected DurableBefore merge(DurableBefore oldValue, DurableBefore newValue) + { + return DurableBefore.merge(oldValue, newValue); + } + } + + public static class DurableBeforeSerializer + implements MergeSerializer, + DurableBeforeMerger> + { + public DurableBeforeMerger mergerFor() + { + return new DurableBeforeMerger(); + } + + @Override + public void serialize(JournalKey key, DurableBefore entry, DataOutputPlus out, Version userVersion) + { + try + { + CommandStoreSerializers.durableBefore.serialize(entry, out); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + } + + @Override + public void reserialize(JournalKey key, DurableBeforeMerger from, DataOutputPlus out, Version userVersion) throws IOException + { + serialize(key, from.get(), out, userVersion); + } + + @Override + public void deserialize(JournalKey journalKey, SimpleMerger into, DataInputPlus in, Version userVersion) throws IOException + { + into.update(CommandStoreSerializers.durableBefore.deserialize(in)); + } + } + + public static class BootstrapBeganAtSerializer + implements MergeSerializer, + SimpleMerger>, + SimpleMerger, NavigableMap>> + { + @Override + public KeepFirst> mergerFor() + { + return new KeepFirst<>(ImmutableSortedMap.of(TxnId.NONE, Ranges.EMPTY)); + } + + @Override + public void serialize(JournalKey key, NavigableMap entry, DataOutputPlus out, Version userVersion) throws IOException + { + CommandStoreSerializers.bootstrapBeganAt.serialize(entry, out); + } + + @Override + public void reserialize(JournalKey key, SimpleMerger, NavigableMap> image, DataOutputPlus out, Version userVersion) throws IOException + { + serialize(key, image.get(), out, userVersion); + } + + @Override + public void deserialize(JournalKey key, SimpleMerger> into, DataInputPlus in, Version userVersion) throws IOException + { + into.update(CommandStoreSerializers.bootstrapBeganAt.deserialize(in)); + } + } + + public static class SafeToReadSerializer + implements MergeSerializer, + SimpleMerger>, + SimpleMerger, NavigableMap>> + { + @Override + public KeepFirst> mergerFor() + { + return new KeepFirst<>(ImmutableSortedMap.of(Timestamp.NONE, Ranges.EMPTY)); + } + + @Override + public void serialize(JournalKey key, NavigableMap from, DataOutputPlus out, Version userVersion) throws IOException + { + CommandStoreSerializers.safeToRead.serialize(from, out); + } + + @Override + public void reserialize(JournalKey key, SimpleMerger, NavigableMap> from, DataOutputPlus out, Version userVersion) throws IOException + { + serialize(key, from.get(), out, userVersion); + } + + @Override + public void deserialize(JournalKey key, SimpleMerger> into, DataInputPlus in, Version userVersion) throws IOException + { + into.update(CommandStoreSerializers.safeToRead.deserialize(in)); + } + } + + public static class RangesForEpochSerializer + implements MergeSerializer, + SimpleMerger> + { + public static final RangesForEpochSerializer instance = new RangesForEpochSerializer(); + public KeepFirst mergerFor() + { + return new KeepFirst<>(null); + } + + @Override + public void serialize(JournalKey key, RangesForEpoch from, DataOutputPlus out, Version userVersion) throws IOException + { + CommandStoreSerializers.rangesForEpoch.serialize(from, out); + } + + @Override + public void reserialize(JournalKey key, SimpleMerger from, DataOutputPlus out, Version userVersion) throws IOException + { + serialize(key, from.get(), out, userVersion); + } + + @Override + public void deserialize(JournalKey key, SimpleMerger into, DataInputPlus in, Version userVersion) throws IOException + { + into.update(CommandStoreSerializers.rangesForEpoch.deserialize(in)); + } + } + + public static class TopologySerializer implements MergeSerializer + { + public static final TopologySerializer INSTANCE = new TopologySerializer(); + + public TopologySerializer() {} + + @Override + public TopologyMerger mergerFor() + { + return new TopologyMerger(); + } + + @Override + public void serialize(JournalKey key, TopologyRecord from, DataOutputPlus out, Version version) throws IOException + { + TopologyRecord.Serializer.instance.serialize(from, out); + } + + @Override + public void reserialize(JournalKey key, TopologyMerger from, DataOutputPlus out, Version version) throws IOException + { + serialize(key, from.write, out, version); + } + + @Override + public void deserialize(JournalKey key, TopologyMerger into, DataInputPlus in, Version version) throws IOException + { + into.read(TopologyRecord.Serializer.instance.deserialize(in)); + } + } + + public static class TopologyMerger implements Merger + { + TopologyRecord.TopologyImage read, write; + + public TopologyMerger() + { + } + + @Override + public void reset(JournalKey key) + { + read = write = null; + } + + public TopologyRecord.TopologyImage read() + { + return read; + } + + public void read(TopologyRecord update) + { + if (Objects.requireNonNull(update.kind()) == TopologyRecord.Kind.New) + read = new TopologyRecord.TopologyImage(update.epoch(), TopologyRecord.Kind.Image, update.getUpdate()); + else + read = (TopologyRecord.TopologyImage) update; + write = read; + } + + public void write(TopologyRecord.TopologyImage image) + { + Invariants.require(write == read); + this.write = image; + } + } +} diff --git a/src/java/org/apache/cassandra/service/accord/journal/Merger.java b/src/java/org/apache/cassandra/service/accord/journal/Merger.java new file mode 100644 index 000000000000..db214513262e --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/journal/Merger.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.accord.journal; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.cassandra.service.accord.JournalKey; + +public interface Merger +{ + abstract class SimpleMerger implements Merger + { + protected A accumulated; + + public SimpleMerger(A initial) {this.accumulated = initial; } + public void update(V newValue) { accumulated = merge(accumulated, newValue); } + public A get() { return accumulated; } + + protected abstract A merge(A oldValue, V newValue); + } + + class KeepFirst extends SimpleMerger + { + final V ifNone; + boolean hasRead; + public KeepFirst(V ifNone) + { + super(ifNone); + this.ifNone = ifNone; + } + + @Override + public void reset(JournalKey key) + { + hasRead = false; + accumulated = ifNone; + } + + @Override + protected V merge(V oldValue, V newValue) + { + if (hasRead) + return oldValue; + hasRead = true; + return newValue; + } + + @Override + public String toString() + { + return "KeepFirst{" + accumulated + '}'; + } + } + + class KeepList extends SimpleMerger, V> + { + public KeepList(List initial) + { + super(initial); + } + + public KeepList() + { + super(new ArrayList<>()); + } + + @Override + protected List merge(List oldValue, V newValue) + { + oldValue.add(newValue); + return oldValue; + } + + @Override + public void reset(JournalKey key) + { + accumulated.clear(); + } + } + + void reset(JournalKey key); +} diff --git a/src/java/org/apache/cassandra/service/accord/journal/RangeSearchManager.java b/src/java/org/apache/cassandra/service/accord/journal/RangeSearchManager.java new file mode 100644 index 000000000000..9edbdd639cf4 --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/journal/RangeSearchManager.java @@ -0,0 +1,293 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.service.accord.journal; + +import java.util.Arrays; +import java.util.Collection; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; + +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.local.MaxDecidedRX; +import accord.primitives.Timestamp; +import accord.primitives.TxnId; +import accord.utils.Invariants; +import accord.utils.UncheckedInterruptedException; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.PartitionRangeReadCommand; +import org.apache.cassandra.db.ReadExecutionController; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.filter.DataLimits; +import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.index.accord.OrderedRouteSerializer; +import org.apache.cassandra.index.accord.RouteJournalIndex; +import org.apache.cassandra.journal.RecordPointer; +import org.apache.cassandra.journal.StaticSegment; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.service.RetryStrategy; +import org.apache.cassandra.service.accord.AccordKeyspace; +import org.apache.cassandra.service.accord.JournalKey; +import org.apache.cassandra.service.accord.TokenRange; +import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.service.accord.serializers.CommandSerializers; +import org.apache.cassandra.service.accord.serializers.Version; +import org.apache.cassandra.utils.CloseableIterator; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.JVMStabilityInspector; +import org.apache.cassandra.utils.MergeIterator; + +import static org.apache.cassandra.service.accord.AccordKeyspace.JournalColumns.getJournalKey; + +public class RangeSearchManager implements RangeSearcher.Supplier +{ + private static final Logger logger = LoggerFactory.getLogger(RangeSearchManager.class); + + final ColumnFamilyStore cfs; + + /** + * Access to this field should only ever be handled by {@link #safeNotify(Consumer)}. There is an assumption that + * an error in the index should not cause the journal to crash, so {@link #safeNotify(Consumer)} exists to make sure + * this property holds true. + */ + @Nullable + private final SegmentRangeSearcher segmentSearcher; + + private RangeSearchManager(ColumnFamilyStore cfs) + { + this.cfs = cfs; + this.segmentSearcher = new SegmentRangeSearcher<>(); + } + + static @Nullable RangeSearchManager ifEnabled(ColumnFamilyStore cfs) + { + return cfs.indexManager.getIndexByName(AccordKeyspace.JOURNAL_INDEX_NAME) != null ? + new RangeSearchManager(cfs) : null; + } + + org.apache.cassandra.journal.SegmentCompactor compactor(ColumnFamilyStore cfs, Version userVersion) + { + return new SegmentCompactor<>(userVersion, cfs) { + @Nullable + @Override + public Collection> compact(Collection> staticSegments) + { + Collection> result = super.compact(staticSegments); + safeNotify(index -> index.remove(staticSegments)); + return result; + } + }; + } + + Runnable maybeIndex(JournalKey key, RecordPointer pointer, CommandChangeWriter change) + { + if (shouldIndex(key) + && change.hasParticipants() + && change.after.route() != null) + { + return () -> safeNotify(index -> index.update(pointer.segment, key.commandStoreId, key.id, change.after.route())); + } + return null; + } + + boolean shouldIndex(JournalKey key) + { + return RouteJournalIndex.allowed(key); + } + + void safeNotify(Consumer> fn) + { + if (segmentSearcher == null) + return; + try + { + fn.accept(segmentSearcher); + } + catch (Throwable t) + { + JVMStabilityInspector.inspectThrowable(t); + logger.warn("Failure updating index", t); + } + } + + @Override + public RangeSearcher rangeSearcher() + { + return new JournalTableRangeSearcher(); + } + + public void start() + { + Index tableIndex = cfs.indexManager.getIndexByName(AccordKeyspace.JOURNAL_INDEX_NAME); + RetryStrategy retry = DatabaseDescriptor.getAccord().retry_journal_index_ready.retry(); + for (int i = 0; !cfs.indexManager.isIndexQueryable(tableIndex); i++) + { + logger.debug("Journal index {} is not ready wait... waiting", AccordKeyspace.JOURNAL_INDEX_NAME); + maybeWait(retry, i); + } + } + + /** + * This method is here to make it easier for org.apache.cassandra.distributed.test.accord.journal.JournalAccessRouteIndexOnStartupRaceTest + * to check when we need to do waiting + */ + @VisibleForTesting + private static void maybeWait(RetryStrategy retry, int i) + { + long waitTime = retry.computeWait(i, TimeUnit.MICROSECONDS); + if (waitTime == -1) + throw new IllegalStateException("Gave up waiting on journal index to be ready"); + try + { + TimeUnit.MICROSECONDS.sleep(waitTime); + } + catch (InterruptedException e) + { + throw new UncheckedInterruptedException(e); + } + } + + /** + * When using {@link PartitionRangeReadCommand} we need to work with {@link RowFilter} which works with columns. + * But the index doesn't care about table based queries and needs to be queried using the fields in the index, to + * support that this enum exists. This enum represents the fields present in the index and can be used to apply + * filters to the index. + */ + public enum SyntheticColumn + { + participants("participants", BytesType.instance), + store_id("store_id", Int32Type.instance), + txn_id("txn_id", BytesType.instance); + + public final ColumnMetadata metadata; + + SyntheticColumn(String name, AbstractType type) + { + this.metadata = new ColumnMetadata("journal", "routes", new ColumnIdentifier(name, false), type, ColumnMetadata.NO_UNIQUE_ID, ColumnMetadata.NO_POSITION, ColumnMetadata.Kind.REGULAR, null); + } + } + + private class JournalTableRangeSearcher implements RangeSearcher + { + private final Index tableIndex; + + private JournalTableRangeSearcher() + { + this.tableIndex = cfs.indexManager.getIndexByName("record"); + if (!cfs.indexManager.isIndexQueryable(tableIndex)) + throw new AssertionError("Journal record index is not queryable"); + } + + @Override + public Result search(int commandStoreId, TokenRange range, TxnId minTxnId, Timestamp maxTxnId, @Nullable MaxDecidedRX.DecidedRX decidedRX) + { + CloseableIterator inMemory = segmentSearcher.search(commandStoreId, range, minTxnId, maxTxnId, decidedRX).results(); + CloseableIterator table = tableSearch(commandStoreId, range.start(), range.end(), minTxnId, maxTxnId, decidedRX); + return new DefaultResult(minTxnId, maxTxnId, decidedRX, MergeIterator.get(Arrays.asList(inMemory, table))); + } + + @Override + public Result search(int commandStoreId, TokenKey key, TxnId minTxnId, Timestamp maxTxnId, @Nullable MaxDecidedRX.DecidedRX decidedRX) + { + CloseableIterator inMemory = segmentSearcher.search(commandStoreId, key, minTxnId, maxTxnId, decidedRX).results(); + CloseableIterator table = tableSearch(commandStoreId, key, minTxnId, maxTxnId); + return new DefaultResult(minTxnId, maxTxnId, decidedRX, MergeIterator.get(Arrays.asList(inMemory, table))); + } + + private CloseableIterator tableSearch(int store, TokenKey start, TokenKey end, TxnId minTxnId, Timestamp maxTxnId, @Nullable MaxDecidedRX.DecidedRX decidedRX) + { + RowFilter rowFilter = RowFilter.create(false); + rowFilter.add(RangeSearchManager.SyntheticColumn.participants.metadata, Operator.GT, OrderedRouteSerializer.serialize(start)); + rowFilter.add(RangeSearchManager.SyntheticColumn.participants.metadata, Operator.LTE, OrderedRouteSerializer.serialize(end)); + rowFilter.add(RangeSearchManager.SyntheticColumn.store_id.metadata, Operator.EQ, Int32Type.instance.decompose(store)); + rowFilter.add(RangeSearchManager.SyntheticColumn.txn_id.metadata, Operator.GTE, CommandSerializers.txnId.serialize(minTxnId)); + rowFilter.add(RangeSearchManager.SyntheticColumn.txn_id.metadata, Operator.LTE, CommandSerializers.timestamp.serialize(maxTxnId)); + return process(store, rowFilter); + } + + private CloseableIterator tableSearch(int store, TokenKey key, TxnId minTxnId, Timestamp maxTxnId) + { + RowFilter rowFilter = RowFilter.create(false); + rowFilter.add(RangeSearchManager.SyntheticColumn.participants.metadata, Operator.GTE, OrderedRouteSerializer.serialize(key)); + rowFilter.add(RangeSearchManager.SyntheticColumn.participants.metadata, Operator.LTE, OrderedRouteSerializer.serialize(key)); + rowFilter.add(RangeSearchManager.SyntheticColumn.store_id.metadata, Operator.EQ, Int32Type.instance.decompose(store)); + rowFilter.add(RangeSearchManager.SyntheticColumn.txn_id.metadata, Operator.GTE, CommandSerializers.txnId.serialize(minTxnId)); + rowFilter.add(RangeSearchManager.SyntheticColumn.txn_id.metadata, Operator.LTE, CommandSerializers.timestamp.serialize(maxTxnId)); + return process(store, rowFilter); + } + + private CloseableIterator process(int storeId, RowFilter rowFilter) + { + PartitionRangeReadCommand cmd = PartitionRangeReadCommand.create(cfs.metadata(), + FBUtilities.nowInSeconds(), + ColumnFilter.selectionBuilder() + .add(RangeSearchManager.SyntheticColumn.store_id.metadata) + .add(RangeSearchManager.SyntheticColumn.txn_id.metadata) + .build(), + rowFilter, + DataLimits.NONE, + DataRange.allData(cfs.getPartitioner())); + Index.Searcher s = tableIndex.searcherFor(cmd); + try (ReadExecutionController controller = cmd.executionController()) + { + UnfilteredPartitionIterator partitionIterator = s.search(controller); + return new CloseableIterator<>() + { + + @Override + public void close() + { + partitionIterator.close(); + } + + @Override + public boolean hasNext() + { + return partitionIterator.hasNext(); + } + + @Override + public TxnId next() + { + UnfilteredRowIterator next = partitionIterator.next(); + JournalKey partitionKeyComponents = getJournalKey(next.partitionKey()); + Invariants.require(partitionKeyComponents.commandStoreId == storeId, + () -> String.format("table index returned a command store other than the expected one; expected %d != %d", storeId, partitionKeyComponents.commandStoreId)); + return partitionKeyComponents.id; + } + }; + } + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/accord/JournalRangeSearcher.java b/src/java/org/apache/cassandra/service/accord/journal/RangeSearcher.java similarity index 90% rename from src/java/org/apache/cassandra/service/accord/JournalRangeSearcher.java rename to src/java/org/apache/cassandra/service/accord/journal/RangeSearcher.java index 71821e0a9a5d..9cd7d49bb0ae 100644 --- a/src/java/org/apache/cassandra/service/accord/JournalRangeSearcher.java +++ b/src/java/org/apache/cassandra/service/accord/journal/RangeSearcher.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.journal; import java.util.function.Consumer; @@ -27,24 +27,25 @@ import accord.primitives.TxnId; import org.apache.cassandra.index.accord.RouteIndexFormat; +import org.apache.cassandra.service.accord.TokenRange; import org.apache.cassandra.service.accord.api.TokenKey; import org.apache.cassandra.utils.CloseableIterator; -public interface JournalRangeSearcher +public interface RangeSearcher { Result search(int commandStoreId, TokenRange range, TxnId minTxnId, Timestamp maxTxnId, @Nullable MaxDecidedRX.DecidedRX decidedRX); Result search(int commandStoreId, TokenKey key, TxnId minTxnId, Timestamp maxTxnId, @Nullable MaxDecidedRX.DecidedRX decidedRX); - static JournalRangeSearcher extractRangeSearcher(Object o) + static RangeSearcher extractRangeSearcher(Object o) { - if (o instanceof JournalRangeSearcher.Supplier) - return ((JournalRangeSearcher.Supplier) o).rangeSearcher(); + if (o instanceof RangeSearcher.Supplier) + return ((RangeSearcher.Supplier) o).rangeSearcher(); return NoopJournalRangeSearcher.instance; } interface Supplier { - JournalRangeSearcher rangeSearcher(); + RangeSearcher rangeSearcher(); } interface Result @@ -117,7 +118,7 @@ public CloseableIterator results() } } - enum NoopJournalRangeSearcher implements JournalRangeSearcher + enum NoopJournalRangeSearcher implements RangeSearcher { instance; diff --git a/src/java/org/apache/cassandra/service/accord/journal/Replay.java b/src/java/org/apache/cassandra/service/accord/journal/Replay.java new file mode 100644 index 000000000000..ce840ccc275e --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/journal/Replay.java @@ -0,0 +1,283 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.accord.journal; + +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.Queue; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.agrona.collections.Int2ObjectHashMap; +import org.agrona.collections.IntArrayList; +import org.agrona.collections.Long2LongHashMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.impl.AbstractReplayer; +import accord.local.CommandStore; +import accord.primitives.Route; +import accord.primitives.TxnId; +import accord.utils.Invariants; +import accord.utils.UnhandledEnum; + +import org.apache.cassandra.config.AccordSpec.JournalSpec.ReplayMode; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.journal.Journal; +import org.apache.cassandra.service.accord.AccordCommandStore; +import org.apache.cassandra.service.accord.JournalKey; +import org.apache.cassandra.utils.Closeable; +import org.apache.cassandra.utils.CloseableIterator; +import org.apache.cassandra.utils.concurrent.Semaphore; + +import static accord.local.RedundantStatus.Property.LOCALLY_DURABLE_TO_COMMAND_STORE; +import static accord.local.RedundantStatus.Property.LOCALLY_DURABLE_TO_DATA_STORE; +import static org.apache.cassandra.service.accord.JournalKey.Type.COMMAND_DIFF; +import static org.apache.cassandra.utils.FBUtilities.getAvailableProcessors; + +public class Replay +{ + private static final Logger logger = LoggerFactory.getLogger(Replay.class); + + static boolean replay(AccordJournal journal, ReplayMode replayMode, CommandStore[] commandStores, Object param) + { + AbstractReplayer.Mode accordReplayerMode; + switch (replayMode) + { + default: throw new UnhandledEnum(replayMode); + case NON_DURABLE: + accordReplayerMode = AbstractReplayer.Mode.NON_DURABLE; + throw new UnsupportedOperationException("Not yet safe to use NON_DURABLE ReplayMode"); + case PART_NON_DURABLE: + accordReplayerMode = AbstractReplayer.Mode.PART_NON_DURABLE; + break; + case ALL: + case RESET: + accordReplayerMode = AbstractReplayer.Mode.ALL; + } + + Invariants.require(param == null || param.getClass() == Long2LongHashMap.class, "Param should be null or a map of commandStoreId->minSegmentId"); + final Long2LongHashMap minSegments = param == null || journal.rangeSearch != null ? new Long2LongHashMap(0L) : (Long2LongHashMap) param; + if (journal.rangeSearch != null) + logger.warn("journal_sai index enabled, which means safe replay markers do not reduce replay work"); + + // TODO (expected): make the parallelisms configurable + // Replay is performed in parallel, where at most X commands can be in flight, across at most Y commands stores. + // That is, you can limit replay parallelism to 1 command store at a time, but load multiple commands within that data store, + // _or_ have multiple commands being loaded accross multiple data stores. + final Semaphore commandParallelism = Semaphore.newSemaphore(getAvailableProcessors()); + final int commandStoreParallelism = Math.max(Math.max(1, Math.min(getAvailableProcessors(), 4)), getAvailableProcessors() / 4); + final AtomicBoolean abort = new AtomicBoolean(); + final IntArrayList activeCommandStoreIds = new IntArrayList(); + final ReplayQueue pendingCommandStores = new ReplayQueue(commandStores); + + class ReplayStream implements Closeable + { + final CommandStore commandStore; + final AbstractReplayer replayer; + final CloseableIterator> iter; + JournalKey prev; + + public ReplayStream(CommandStore commandStore, long minSegment) + { + this.commandStore = commandStore; + this.replayer = (AbstractReplayer) commandStore.replayer(accordReplayerMode); + // Keys in the index are sorted by command store id, so index iteration will be sequential + this.iter = journal.keyIterator(new JournalKey(replayer.minReplay.withoutNonIdentityFlags(), COMMAND_DIFF, commandStore.id()), new JournalKey(TxnId.MAX.withoutNonIdentityFlags(), COMMAND_DIFF, commandStore.id()), false, minSegment); + logger.info("Beginning replay of {} with min={}, {}", commandStore, replayer.minReplay, + replayer.redundantBefore.map(b -> b == null ? null : b.maxBoundBoth(LOCALLY_DURABLE_TO_DATA_STORE, LOCALLY_DURABLE_TO_COMMAND_STORE), TxnId[]::new)); + } + + boolean replay() + { + JournalKey key; + long[] segments; + while (true) + { + if (!iter.hasNext()) + { + logger.info("Completed replay of {}", commandStore); + return false; + } + + Journal.KeyRefs ref = iter.next(); + if (ref.key().type != COMMAND_DIFF) + continue; + + key = ref.key(); + segments = journal.rangeSearch != null && journal.rangeSearch.shouldIndex(key) ? ref.copyOfSegments() : null; + break; + } + + TxnId txnId = key.id; + Invariants.require(prev == null || + key.commandStoreId != prev.commandStoreId || + key.id.compareTo(prev.id) != 0, + "duplicate key detected %s == %s", key, prev); + prev = key; + commandParallelism.acquireThrowUncheckedOnInterrupt(1); + replayer.replay(txnId) + .map(route -> { + if (segments != null && route != null) + { + for (long segment : segments) + journal.rangeSearch.safeNotify(index -> index.update(segment, key.commandStoreId, txnId, (Route) route)); + } + return null; + }).begin((success, fail) -> { + commandParallelism.release(1); + if (fail != null && !journal.segments.handleError("Could not replay command " + txnId, fail)) + abort.set(true); + }); + + return true; + } + + @Override + public void close() + { + iter.close(); + } + } + + // Replay streams by command store id, can hold at most commandStoreParallelism items + final Int2ObjectHashMap replayStreams = new Int2ObjectHashMap<>(); + try + { + // index of the store we're currently pulling from in the activeCommandStoreIds collection + int cur = 0; + while (!abort.get()) + { + if (cur == activeCommandStoreIds.size()) + { + if (activeCommandStoreIds.size() < commandStoreParallelism && !pendingCommandStores.isEmpty()) + { + CommandStore next = pendingCommandStores.next(); + int id = next.id(); + activeCommandStoreIds.add(id); + replayStreams.put(id, new ReplayStream(next, minSegments.getOrDefault(id, 0))); + } + else if (activeCommandStoreIds.isEmpty()) break; + else cur = 0; + } + + int id = activeCommandStoreIds.get(cur); + ReplayStream replayStream = replayStreams.get(id); + while (!replayStream.replay()) + { + // Replay complete for this command store; close and replace + replayStreams.remove(id).close(); + if (pendingCommandStores.isEmpty()) + { + // no more pending to submit; remove and continue with the next remaining (if any) + activeCommandStoreIds.removeAt(cur); + if (cur == activeCommandStoreIds.size()) + --cur; + if (cur < 0) + break; + id = activeCommandStoreIds.get(cur); + } + else + { + // replace it with a pending command store, and continue processing + CommandStore next = pendingCommandStores.next(streamId(replayStream.commandStore)); + id = next.id(); + activeCommandStoreIds.set(cur, id); + replayStreams.put(id, new ReplayStream(next, minSegments.getOrDefault(id, 0))); + } + + replayStream = replayStreams.get(id); + } + + ++cur; + } + return true; + } + catch (Throwable t) + { + try { FileUtils.close(replayStreams.values()); } + catch (Throwable t2) { t.addSuppressed(t2); } + throw t; + } + } + + static class ReplayQueue + { + final Int2ObjectHashMap> byExecutor = new Int2ObjectHashMap<>(); + final Deque nextId = new ArrayDeque<>(); + + ReplayQueue(CommandStore[] commandStores) + { + for (CommandStore commandStore : commandStores) + { + byExecutor.computeIfAbsent(streamId(commandStore), ignore -> new ArrayDeque<>()) + .add(commandStore); + } + nextId.addAll(byExecutor.keySet()); + } + + boolean isEmpty() + { + return byExecutor.isEmpty(); + } + + CommandStore next() + { + while (true) + { + if (byExecutor.isEmpty()) + return null; + + Integer id = nextId.poll(); + if (id == null) + { + nextId.addAll(byExecutor.keySet()); + id = nextId.poll(); + } + + Queue queue = byExecutor.get(id); + if (queue != null) + { + CommandStore next = queue.poll(); + if (queue.isEmpty()) + byExecutor.remove(id); + if (next != null) + return next; + } + } + } + + CommandStore next(int streamId) + { + Queue queue = byExecutor.get(streamId); + if (queue == null) + return next(); + + CommandStore next = queue.poll(); + if (queue.isEmpty()) + byExecutor.remove(streamId); + + return next; + } + } + + private static int streamId(CommandStore commandStore) + { + return commandStore instanceof AccordCommandStore ? ((AccordCommandStore) commandStore).executor().executorId() : 1; + } +} diff --git a/src/java/org/apache/cassandra/service/accord/journal/ReplayMarkers.java b/src/java/org/apache/cassandra/service/accord/journal/ReplayMarkers.java new file mode 100644 index 000000000000..14845c31ae7f --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/journal/ReplayMarkers.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.accord.journal; + +import java.io.IOException; +import java.io.UncheckedIOException; + +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.FileInputStreamPlus; +import org.apache.cassandra.io.util.FileOutputStreamPlus; +import org.apache.cassandra.utils.NativeLibrary; + +import static org.apache.cassandra.config.DatabaseDescriptor.getAccordJournalDirectory; + +public class ReplayMarkers +{ + public static File startMarker() + { + return new File(getAccordJournalDirectory(), "started"); + } + + public static File safeStopMarker() + { + return new File(getAccordJournalDirectory(), "stopped"); + } + + // TODO (required): add checksummed version and default to this (but support unchecksummed for manual editing) + static void writeMarker(File file, long timestamp) + { + try (FileOutputStreamPlus out = new FileOutputStreamPlus(file)) + { + out.writeBytes(Long.toString(timestamp)); + } + catch (IOException e) + { + throw new UncheckedIOException(e); + } + trySyncJournalDirectory(); + } + + public static long readStartMarker() + { + return readMarker(startMarker()); + } + + public static long readStopMarker() + { + return readMarker(safeStopMarker()); + } + + public static long readMarker(File file) + { + if (!file.exists()) + return -1L; + + try (FileInputStreamPlus in = new FileInputStreamPlus(file)) + { + StringBuilder sb = new StringBuilder(8); + for (int b = in.read(); b >= 0 ; b = in.read()) + sb.append((char)b); + return Long.parseLong(sb.toString()); + } + catch (IOException e) + { + throw new UncheckedIOException(e); + } + } + + private static void trySyncJournalDirectory() + { + trySyncDirectory(getAccordJournalDirectory()); + } + + private static void trySyncDirectory(String path) + { + int fd = NativeLibrary.tryOpenDirectory(path); + NativeLibrary.trySync(fd); + } + + public static File saveDirectory() + { + return new File(getAccordJournalDirectory(), "save_state"); + } +} diff --git a/src/java/org/apache/cassandra/service/accord/AccordSegmentCompactor.java b/src/java/org/apache/cassandra/service/accord/journal/SegmentCompactor.java similarity index 91% rename from src/java/org/apache/cassandra/service/accord/AccordSegmentCompactor.java rename to src/java/org/apache/cassandra/service/accord/journal/SegmentCompactor.java index 7cd047bc0ef3..e4ce93dea2d6 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordSegmentCompactor.java +++ b/src/java/org/apache/cassandra/service/accord/journal/SegmentCompactor.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.journal; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.SerializationHeader; @@ -27,11 +27,11 @@ /** * Segment compactor: takes static segments and compacts them into a single SSTable. */ -public class AccordSegmentCompactor extends AbstractAccordSegmentCompactor +public class SegmentCompactor extends AbstractSegmentCompactor { private SSTableTxnWriter writer; - public AccordSegmentCompactor(Version userVersion, ColumnFamilyStore cfs) + public SegmentCompactor(Version userVersion, ColumnFamilyStore cfs) { super(userVersion, cfs); } diff --git a/src/java/org/apache/cassandra/service/accord/JournalSegmentRangeSearcher.java b/src/java/org/apache/cassandra/service/accord/journal/SegmentRangeSearcher.java similarity index 87% rename from src/java/org/apache/cassandra/service/accord/JournalSegmentRangeSearcher.java rename to src/java/org/apache/cassandra/service/accord/journal/SegmentRangeSearcher.java index e2b69f977b32..9121f2380a50 100644 --- a/src/java/org/apache/cassandra/service/accord/JournalSegmentRangeSearcher.java +++ b/src/java/org/apache/cassandra/service/accord/journal/SegmentRangeSearcher.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.journal; import java.util.Collection; import java.util.Collections; @@ -49,6 +49,8 @@ import org.apache.cassandra.index.accord.RouteJournalIndex; import org.apache.cassandra.journal.StaticSegment; import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.service.accord.JournalKey; +import org.apache.cassandra.service.accord.TokenRange; import org.apache.cassandra.service.accord.api.TokenKey; import org.apache.cassandra.utils.ByteArrayUtil; import org.apache.cassandra.utils.CloseableIterator; @@ -58,7 +60,7 @@ import static accord.primitives.Routable.Domain.Range; -public class JournalSegmentRangeSearcher implements JournalRangeSearcher +public class SegmentRangeSearcher implements RangeSearcher { private final Long2ObjectHashMap segmentIndexes = new Long2ObjectHashMap<>(); @@ -85,7 +87,7 @@ public synchronized void removeForTests(Collection oldSegments) } @Override - public JournalRangeSearcher.Result search(int commandStoreId, TokenRange range, TxnId minTxnId, Timestamp maxTxnId, @Nullable DecidedRX decidedRX) + public RangeSearcher.Result search(int commandStoreId, TokenRange range, TxnId minTxnId, Timestamp maxTxnId, @Nullable DecidedRX decidedRX) { NavigableSet result = search(commandStoreId, range.table(), OrderedRouteSerializer.serializeTokenOnly(range.start()), @@ -103,7 +105,7 @@ private synchronized NavigableSet search(int storeId, TableId tableId, by } @Override - public JournalRangeSearcher.Result search(int commandStoreId, TokenKey key, TxnId minTxnId, Timestamp maxTxnId, @Nullable MaxDecidedRX.DecidedRX decidedRX) + public RangeSearcher.Result search(int commandStoreId, TokenKey key, TxnId minTxnId, Timestamp maxTxnId, @Nullable MaxDecidedRX.DecidedRX decidedRX) { NavigableSet result = search(commandStoreId, key.table(), OrderedRouteSerializer.serializeTokenOnly(key), minTxnId, maxTxnId, decidedRX); return new DefaultResult(minTxnId, maxTxnId, decidedRX, CloseableIterator.wrap(result.iterator())); @@ -138,7 +140,7 @@ private void add(int commandStoreId, TxnId id, Route route) private void search(int storeId, TableId tableId, byte[] start, byte[] end, TxnId minTxnId, Timestamp maxTxnId, @Nullable DecidedRX decidedRX, - Consumer> fn) + Consumer> fn) { StoreIndex idx = storeIndexes.get(storeId); if (idx == null) return; @@ -148,7 +150,7 @@ private void search(int storeId, TableId tableId, private void search(int storeId, TableId tableId, byte[] key, TxnId minTxnId, Timestamp maxTxnId, @Nullable DecidedRX decidedRX, - Consumer> fn) + Consumer> fn) { StoreIndex idx = storeIndexes.get(storeId); if (idx == null) return; @@ -181,7 +183,7 @@ private void add(TxnId id, Unseekable keyOrRange) public void search(TableId tableId, byte[] start, byte[] end, TxnId minTxnId, Timestamp maxTxnId, @Nullable DecidedRX decidedRX, - Consumer> fn) + Consumer> fn) { TableIndex index = tableIndex.get(tableId); if (index == null) return; @@ -191,7 +193,7 @@ public void search(TableId tableId, public void search(TableId tableId, byte[] key, TxnId minTxnId, Timestamp maxTxnId, @Nullable MaxDecidedRX.DecidedRX decidedRX, - Consumer> fn) + Consumer> fn) { TableIndex index = tableIndex.get(tableId); if (index == null) return; @@ -201,7 +203,7 @@ public void search(TableId tableId, private static class TableIndex { - private final RangeTree index = createRangeTree(); + private final RangeTree index = createRangeTree(); private TxnId min = TxnId.MAX; private TxnId max = TxnId.NONE; private @Nullable TxnId maxRX = TxnId.NONE; @@ -214,7 +216,7 @@ private void add(TxnId id, TokenRange ts) { byte[] start = OrderedRouteSerializer.serializeTokenOnly(ts.start()); byte[] end = OrderedRouteSerializer.serializeTokenOnly(ts.end()); - IndexRange range = new IndexRange(start, end); + IndexedRange range = new IndexedRange(start, end); index.add(range, id); if (min.compareTo(id) > 0) @@ -230,12 +232,12 @@ private void add(TxnId id, TokenRange ts) private void search(byte[] start, byte[] end, TxnId minTxnId, Timestamp maxTxnId, @Nullable DecidedRX decidedRX, - Consumer> fn) + Consumer> fn) { if (minTxnId.compareTo(max) > 0) return; if (maxTxnId.compareTo(min) < 0) return; if (maxRX != null && !RouteIndexFormat.includeByDecidedRX(decidedRX, maxRX)) return; - index.search(new IndexRange(start, end), e -> { + index.search(new IndexedRange(start, end), e -> { if (minTxnId.compareTo(e.getValue()) > 0) return; if (maxTxnId.compareTo(e.getValue()) < 0) return; fn.accept(e); @@ -244,7 +246,7 @@ private void search(byte[] start, byte[] end, private void search(byte[] key, TxnId minTxnId, Timestamp maxTxnId, @Nullable MaxDecidedRX.DecidedRX decidedRX, - Consumer> fn) + Consumer> fn) { if (minTxnId.compareTo(max) > 0) return; if (maxTxnId.compareTo(min) < 0) return; @@ -257,18 +259,18 @@ private void search(byte[] key, } } - private static RangeTree createRangeTree() + private static RangeTree createRangeTree() { return new RTree<>((a, b) -> ByteArrayUtil.compareUnsigned(a, 0, b, 0, a.length), new RangeTree.Accessor<>() { @Override - public byte[] start(IndexRange range) + public byte[] start(IndexedRange range) { return range.start; } @Override - public byte[] end(IndexRange range) + public byte[] end(IndexedRange range) { return range.end; } @@ -282,13 +284,13 @@ public boolean contains(byte[] start, byte[] end, byte[] bytes) } @Override - public boolean intersects(IndexRange range, byte[] start, byte[] end) + public boolean intersects(IndexedRange range, byte[] start, byte[] end) { return range.intersects(start, end); } @Override - public boolean intersects(IndexRange left, IndexRange right) + public boolean intersects(IndexedRange left, IndexedRange right) { return left.intersects(right.start, right.end); } diff --git a/src/java/org/apache/cassandra/service/accord/journal/TableAndSegmentKeyIterator.java b/src/java/org/apache/cassandra/service/accord/journal/TableAndSegmentKeyIterator.java new file mode 100644 index 000000000000..e41a25433ed9 --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/journal/TableAndSegmentKeyIterator.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.accord.journal; + +import com.google.common.collect.AbstractIterator; + +import accord.utils.Invariants; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.journal.Journal; +import org.apache.cassandra.service.accord.JournalKey; +import org.apache.cassandra.utils.CloseableIterator; + +import static org.apache.cassandra.service.accord.JournalKey.SUPPORT; + +class TableAndSegmentKeyIterator extends AbstractIterator> implements CloseableIterator> +{ + final Journal.SegmentKeyIterator journalIterator; + final TableKeyIterator tableIterator; + + TableAndSegmentKeyIterator(Journal journal, ColumnFamilyStore table, JournalKey min, JournalKey max, boolean includeActive, long minSegment) + { + // We must initialise journal reader first, else we may race with segment->table compaction and miss some data + // that is, the following sequence could happen: + // - Select sstables to read + // - Segments compacted; segments removed and sstables added + // - Segment iterator created + // TODO (expected): segments should be sstables on creation + this.journalIterator = journal.segmentKeyIterator(min, max, segment -> segment.id() >= minSegment && (includeActive || segment.isStatic())); + this.tableIterator = new TableKeyIterator(table, min, max, minSegment); + } + + JournalKey prevFromTable = null; + JournalKey prevFromJournal = null; + + @Override + protected Journal.KeyRefs computeNext() + { + JournalKey tableKey = tableIterator.hasNext() ? tableIterator.peek() : null; + JournalKey journalKey = journalIterator.hasNext() ? journalIterator.peek().key() : null; + + if (journalKey != null) + { + Invariants.require(prevFromJournal == null || SUPPORT.compare(journalKey, prevFromJournal) >= 0, // == for case where we have not consumed previous on prev iteration + "Incorrect sort order in journal segments: %s should strictly follow %s", journalKey, prevFromJournal); + prevFromJournal = journalKey; + } + else + { + prevFromJournal = null; + } + + if (tableKey != null) + { + Invariants.require(prevFromTable == null || SUPPORT.compare(tableKey, prevFromTable) >= 0, // == for case where we have not consumed previous on prev iteration + "Incorrect sort order in journal table: %s should strictly follow %s", tableKey, prevFromTable); + prevFromTable = tableKey; + } + else + { + prevFromTable = null; + } + + if (tableKey == null) + return journalKey == null ? endOfData() : journalIterator.next(); + + if (journalKey == null) + return new Journal.KeyRefs<>(tableIterator.next()); + + int cmp = SUPPORT.compare(tableKey, journalKey); + if (cmp == 0) + { + tableIterator.next(); + return journalIterator.next(); + } + + return cmp < 0 ? new Journal.KeyRefs<>(tableIterator.next()) : journalIterator.next(); + } + + public void close() + { + tableIterator.close(); + journalIterator.close(); + } +} diff --git a/src/java/org/apache/cassandra/service/accord/journal/TableKeyIterator.java b/src/java/org/apache/cassandra/service/accord/journal/TableKeyIterator.java new file mode 100644 index 000000000000..99a181317282 --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/journal/TableKeyIterator.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.accord.journal; + +import java.util.ArrayList; +import java.util.List; + +import javax.annotation.CheckForNull; + +import com.google.common.collect.AbstractIterator; + +import accord.utils.Invariants; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.EmptyIterators; +import org.apache.cassandra.db.lifecycle.SSTableSet; +import org.apache.cassandra.db.lifecycle.View; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.dht.Bounds; +import org.apache.cassandra.io.sstable.ISSTableScanner; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.service.accord.AccordKeyspace; +import org.apache.cassandra.service.accord.JournalKey; +import org.apache.cassandra.utils.CloseableIterator; + +import static org.apache.cassandra.service.accord.AccordKeyspace.JournalColumns.getJournalKey; +import static org.apache.cassandra.service.accord.JournalKey.SUPPORT; + +class TableKeyIterator extends AbstractIterator implements CloseableIterator +{ + private final UnfilteredPartitionIterator mergeIterator; + private final ColumnFamilyStore.RefViewFragment view; + + TableKeyIterator(ColumnFamilyStore table, JournalKey min, JournalKey max, long minSegment) + { + Invariants.require((min != null && max != null) || min == max); + view = table.selectAndReference(View.select(SSTableSet.LIVE, r -> (max == null || SUPPORT.compare(getJournalKey(r.getFirst()), max) <= 0) + && (min == null || SUPPORT.compare(getJournalKey(r.getLast()), min) >= 0) + && (r.getSSTableMetadata().coveredClustering.end().isArtificial() || LongType.instance.compose(r.getSSTableMetadata().coveredClustering.end().bufferAt(0)) >= minSegment) + )); + List scanners = new ArrayList<>(); + for (SSTableReader sstable : view.sstables) + { + + if (min == null) scanners.add(sstable.getScanner()); + else + scanners.add(sstable.getScanner(new Bounds(AccordKeyspace.JournalColumns.decorate(min), AccordKeyspace.JournalColumns.decorate(max)))); + } + + mergeIterator = view.sstables.isEmpty() + ? EmptyIterators.unfilteredPartition(table.metadata()) + : UnfilteredPartitionIterators.merge(scanners, UnfilteredPartitionIterators.MergeListener.NOOP); + } + + @CheckForNull + protected JournalKey computeNext() + { + JournalKey ret = null; + if (mergeIterator.hasNext()) + { + try (UnfilteredRowIterator partition = mergeIterator.next()) + { + ret = getJournalKey(partition.partitionKey()); + while (partition.hasNext()) + partition.next(); + } + } + + if (ret != null) + return ret; + else + return endOfData(); + } + + @Override + public void close() + { + mergeIterator.close(); + view.close(); + } +} diff --git a/src/java/org/apache/cassandra/service/accord/journal/TableRecordIterator.java b/src/java/org/apache/cassandra/service/accord/journal/TableRecordIterator.java new file mode 100644 index 000000000000..764e9ac77e0f --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/journal/TableRecordIterator.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.service.accord.journal; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.EmptyIterators; +import org.apache.cassandra.db.Slices; +import org.apache.cassandra.db.StorageHook; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.lifecycle.SSTableSet; +import org.apache.cassandra.db.lifecycle.View; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.rows.UnfilteredRowIterators; +import org.apache.cassandra.io.FSReadError; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.journal.RecordConsumer; +import org.apache.cassandra.service.accord.AccordKeyspace; +import org.apache.cassandra.service.accord.JournalKey; +import org.apache.cassandra.utils.Closeable; +import org.apache.cassandra.utils.concurrent.OpOrder; + +import static org.apache.cassandra.io.sstable.SSTableReadsListener.NOOP_LISTENER; + +class TableRecordIterator implements Closeable, RecordConsumer +{ + final JournalKey key; + final List unmerged; + final UnfilteredRowIterator merged; + + long segment; + int offset; + ByteBuffer value; + int userVersion; + + TableRecordIterator(JournalKey key, List unmerged, UnfilteredRowIterator merged) + { + this.key = key; + this.unmerged = unmerged; + this.merged = merged; + } + + static TableRecordIterator all(ColumnFamilyStore cfs, JournalKey key, OpOrder.Group readOrder) + { + DecoratedKey pk = AccordKeyspace.JournalColumns.decorate(key); + List iters = new ArrayList<>(3); + try + { + ColumnFamilyStore.ViewFragment view = cfs.select(View.select(SSTableSet.LIVE, pk)); + for (SSTableReader sstable : view.sstables) + { + if (!sstable.mayContainAssumingKeyIsInRange(pk)) + continue; + + UnfilteredRowIterator iter = StorageHook.instance.makeRowIterator(cfs, sstable, pk, Slices.ALL, ColumnFilter.all(cfs.metadata()), false, NOOP_LISTENER); + if (iter.getClass() != EmptyIterators.EmptyUnfilteredRowIterator.class) + iters.add(iter); + } + + return new TableRecordIterator(key, iters, iters.isEmpty() ? null : UnfilteredRowIterators.merge(iters)); + } + catch (Throwable t) + { + for (UnfilteredRowIterator iter : iters) + { + try { iter.close(); } + catch (Throwable t2) { t.addSuppressed(t2); } + } + throw t; + } + } + + @Override + public void accept(long segment, int offset, JournalKey key, ByteBuffer buffer, int userVersion) + { + this.segment = segment; + this.offset = offset; + this.value = buffer; + this.userVersion = userVersion; + } + + boolean advance() + { + if (merged == null || !merged.hasNext()) + return false; + + try + { + Row row = (Row) merged.next(); + segment = LongType.instance.compose(ByteBuffer.wrap((byte[]) row.clustering().get(0))); + offset = Int32Type.instance.compose(ByteBuffer.wrap((byte[]) row.clustering().get(1))); + value = row.getCell(AccordKeyspace.JournalColumns.record).buffer(); + userVersion = Int32Type.instance.compose(row.getCell(AccordKeyspace.JournalColumns.user_version).buffer()); + return true; + } + catch (Throwable t) + { + throw new FSReadError("Failed to read from " + unmerged, t); + } + } + + @Override + public void close() + { + if (merged != null) + merged.close(); + } +} diff --git a/src/java/org/apache/cassandra/service/accord/journal/AccordTopologyUpdate.java b/src/java/org/apache/cassandra/service/accord/journal/TopologyRecord.java similarity index 71% rename from src/java/org/apache/cassandra/service/accord/journal/AccordTopologyUpdate.java rename to src/java/org/apache/cassandra/service/accord/journal/TopologyRecord.java index efb92c2df15d..c59a184170a5 100644 --- a/src/java/org/apache/cassandra/service/accord/journal/AccordTopologyUpdate.java +++ b/src/java/org/apache/cassandra/service/accord/journal/TopologyRecord.java @@ -26,7 +26,9 @@ import accord.api.Journal; import accord.local.CommandStores; +import accord.local.Node; import accord.primitives.Ranges; +import accord.primitives.TxnId; import accord.topology.Topology; import accord.utils.Invariants; import accord.utils.UnhandledEnum; @@ -35,198 +37,29 @@ import org.apache.cassandra.io.UnversionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.service.accord.AccordJournalValueSerializers; import org.apache.cassandra.service.accord.JournalKey; import org.apache.cassandra.service.accord.serializers.KeySerializers; import org.apache.cassandra.service.accord.serializers.TopologySerializers; -import org.apache.cassandra.service.accord.serializers.Version; -public interface AccordTopologyUpdate +import static org.apache.cassandra.service.accord.serializers.CommandStoreSerializers.rangesForEpoch; + +public interface TopologyRecord { Kind kind(); void applyTo(TopologyImage accumulator); long epoch(); - AccordTopologyUpdate asRepeat(); + TopologyRecord asRepeat(); Journal.TopologyUpdate getUpdate(); - static AccordTopologyUpdate newTopology(Journal.TopologyUpdate update) + static TopologyRecord newTopology(Journal.TopologyUpdate update) { return new NewTopology(update); } - class RangesForEpochSerializer implements UnversionedSerializer - { - public static final RangesForEpochSerializer instance = new RangesForEpochSerializer(); - - @Override - public void serialize(CommandStores.RangesForEpoch from, DataOutputPlus out) throws IOException - { - out.writeUnsignedVInt32(from.size()); - for (int i = 0; i < from.size(); i++) - { - out.writeLong(from.epochAtIndex(i)); - KeySerializers.ranges.serialize(from.rangesAtIndex(i), out); - } - } - - @Override - public CommandStores.RangesForEpoch deserialize(DataInputPlus in) throws IOException - { - int size = in.readUnsignedVInt32(); - Ranges[] ranges = new Ranges[size]; - long[] epochs = new long[size]; - for (int i = 0; i < ranges.length; i++) - { - epochs[i] = in.readLong(); - ranges[i] = KeySerializers.ranges.deserialize(in); - } - return new CommandStores.RangesForEpoch(epochs, ranges); - } - - @Override - public long serializedSize(CommandStores.RangesForEpoch from) - { - long size = TypeSizes.sizeofUnsignedVInt(from.size()); - for (int i = 0; i < from.size(); i++) - { - size += TypeSizes.LONG_SIZE; - size += KeySerializers.ranges.serializedSize(from.rangesAtIndex(i)); - } - return size; - } - } - class TopologyUpdateSerializer implements UnversionedSerializer + static JournalKey journalKey(long epoch) { - public static final TopologyUpdateSerializer instance = new TopologyUpdateSerializer(); - - @Override - public void serialize(Journal.TopologyUpdate from, DataOutputPlus out) throws IOException - { - out.writeUnsignedVInt32(from.commandStores.size()); - for (Map.Entry e : from.commandStores.entrySet()) - { - out.writeUnsignedVInt32(e.getKey()); - RangesForEpochSerializer.instance.serialize(e.getValue(), out); - } - TopologySerializers.compactTopology.serialize(from.global, out); - } - - @Override - public Journal.TopologyUpdate deserialize(DataInputPlus in) throws IOException - { - int commandStoresSize = in.readUnsignedVInt32(); - Int2ObjectHashMap commandStores = new Int2ObjectHashMap<>(); - for (int j = 0; j < commandStoresSize; j++) - { - int commandStoreId = in.readUnsignedVInt32(); - CommandStores.RangesForEpoch rangesForEpoch = RangesForEpochSerializer.instance.deserialize(in); - commandStores.put(commandStoreId, rangesForEpoch); - } - Topology global = TopologySerializers.compactTopology.deserialize(in); - return new Journal.TopologyUpdate(commandStores, global); - } - - @Override - public long serializedSize(Journal.TopologyUpdate from) - { - long size = TypeSizes.sizeofUnsignedVInt(from.commandStores.size()); - for (Map.Entry e : from.commandStores.entrySet()) - { - size += TypeSizes.sizeofUnsignedVInt(e.getKey()); - size += RangesForEpochSerializer.instance.serializedSize(e.getValue()); - } - - size += TopologySerializers.compactTopology.serializedSize(from.global); - return size; - } - } - - class Serializer implements UnversionedSerializer - { - public static Serializer instance = new Serializer(); - - @Override - public void serialize(AccordTopologyUpdate t, DataOutputPlus out) throws IOException - { - out.writeUnsignedVInt(t.epoch()); - out.writeUnsignedVInt32(t.kind().ordinal()); - switch (t.kind()) - { - case New: - { - TopologyUpdateSerializer.instance.serialize(((NewTopology) t).update, out); - break; - } - case Repeat: - case Image: - TopologyImage image = (TopologyImage) t; - out.writeBoolean(image.update != null); - if (image.update != null) - TopologyUpdateSerializer.instance.serialize(image.update, out); - out.writeByte(0); // defunct enum byte - - KeySerializers.ranges.serialize(image.closed, out); - KeySerializers.ranges.serialize(image.retired, out); - break; - default: - throw new UnhandledEnum(t.kind()); - } - } - - @Override - public AccordTopologyUpdate deserialize(DataInputPlus in) throws IOException - { - long epoch = in.readUnsignedVInt(); - Kind kind = Kind.values()[in.readUnsignedVInt32()]; - - switch (kind) - { - case New: - return new NewTopology(TopologyUpdateSerializer.instance.deserialize(in)); - case Repeat: - case Image: - Journal.TopologyUpdate update = null; - if (in.readBoolean()) - update = TopologyUpdateSerializer.instance.deserialize(in); - - TopologyImage image = new TopologyImage(epoch, kind, update); - in.readByte(); // defunct enum byte - - image.closed = KeySerializers.ranges.deserialize(in); - image.retired = KeySerializers.ranges.deserialize(in); - return image; - default: - throw new UnhandledEnum(kind); - } - } - - @Override - public long serializedSize(AccordTopologyUpdate t) - { - long size = TypeSizes.sizeofUnsignedVInt(t.epoch()); - size += TypeSizes.sizeofUnsignedVInt(t.kind().ordinal()); - - switch (t.kind()) - { - case New: - size += TopologyUpdateSerializer.instance.serializedSize(((NewTopology) t).update); - break; - case Image: - case Repeat: - TopologyImage image = (TopologyImage) t; - size += TypeSizes.BOOL_SIZE; - if (image.update != null) - size += TopologyUpdateSerializer.instance.serializedSize(image.update); - - size += TypeSizes.BYTE_SIZE; - size += KeySerializers.ranges.serializedSize(image.closed); - size += KeySerializers.ranges.serializedSize(image.retired); - break; - default: - throw new UnhandledEnum(t.kind()); - } - return size; - } + return new JournalKey(TxnId.fromValues(epoch, 0L, Node.Id.NONE), + JournalKey.Type.TOPOLOGY_UPDATE, Integer.MAX_VALUE); } enum Kind @@ -242,8 +75,7 @@ enum Kind Repeat } - - class TopologyImage implements AccordTopologyUpdate + class TopologyImage implements TopologyRecord { private final long epoch; private final Kind kind; @@ -332,7 +164,7 @@ public int hashCode() } } - class NewTopology implements AccordTopologyUpdate + class NewTopology implements TopologyRecord { public final Journal.TopologyUpdate update; private final long epoch; @@ -370,7 +202,7 @@ public void applyTo(TopologyImage accumulator) } @Override - public AccordTopologyUpdate asRepeat() + public TopologyRecord asRepeat() { return new TopologyImage(epoch, Kind.Repeat, update); } @@ -391,67 +223,137 @@ public int hashCode() } } - class Accumulator implements AccordJournalValueSerializers.FlyweightImage + class TopologyUpdateSerializer implements UnversionedSerializer { - TopologyImage read, write; - - public Accumulator() - { - } + public static final TopologyUpdateSerializer instance = new TopologyUpdateSerializer(); @Override - public void reset(JournalKey key) + public void serialize(Journal.TopologyUpdate from, DataOutputPlus out) throws IOException { - read = write = null; + out.writeUnsignedVInt32(from.commandStores.size()); + for (Map.Entry e : from.commandStores.entrySet()) + { + out.writeUnsignedVInt32(e.getKey()); + rangesForEpoch.serialize(e.getValue(), out); + } + TopologySerializers.compactTopology.serialize(from.global, out); } - public TopologyImage read() + @Override + public Journal.TopologyUpdate deserialize(DataInputPlus in) throws IOException { - return read; + int commandStoresSize = in.readUnsignedVInt32(); + Int2ObjectHashMap commandStores = new Int2ObjectHashMap<>(); + for (int j = 0; j < commandStoresSize; j++) + { + int commandStoreId = in.readUnsignedVInt32(); + CommandStores.RangesForEpoch rfe = rangesForEpoch.deserialize(in); + commandStores.put(commandStoreId, rfe); + } + Topology global = TopologySerializers.compactTopology.deserialize(in); + return new Journal.TopologyUpdate(commandStores, global); } - public void read(AccordTopologyUpdate update) + @Override + public long serializedSize(Journal.TopologyUpdate from) { - if (Objects.requireNonNull(update.kind()) == Kind.New) - read = new TopologyImage(update.epoch(), Kind.Image, update.getUpdate()); - else - read = (TopologyImage) update; - write = read; - } + long size = TypeSizes.sizeofUnsignedVInt(from.commandStores.size()); + for (Map.Entry e : from.commandStores.entrySet()) + { + size += TypeSizes.sizeofUnsignedVInt(e.getKey()); + size += rangesForEpoch.serializedSize(e.getValue()); + } - public void write(TopologyImage image) - { - Invariants.require(write == read); - this.write = image; + size += TopologySerializers.compactTopology.serializedSize(from.global); + return size; } } - class FlyweightSerializer implements AccordJournalValueSerializers.FlyweightSerializer + class Serializer implements UnversionedSerializer { - public FlyweightSerializer() {} + public static Serializer instance = new Serializer(); @Override - public Accumulator mergerFor() + public void serialize(TopologyRecord t, DataOutputPlus out) throws IOException { - return new Accumulator(); - } + out.writeUnsignedVInt(t.epoch()); + out.writeUnsignedVInt32(t.kind().ordinal()); + switch (t.kind()) + { + case New: + { + TopologyUpdateSerializer.instance.serialize(((NewTopology) t).update, out); + break; + } + case Repeat: + case Image: + TopologyImage image = (TopologyImage) t; + out.writeBoolean(image.update != null); + if (image.update != null) + TopologyUpdateSerializer.instance.serialize(image.update, out); + out.writeByte(0); // defunct enum byte - @Override - public void serialize(JournalKey key, AccordTopologyUpdate from, DataOutputPlus out, Version version) throws IOException - { - Serializer.instance.serialize(from, out); + KeySerializers.ranges.serialize(image.closed, out); + KeySerializers.ranges.serialize(image.retired, out); + break; + default: + throw new UnhandledEnum(t.kind()); + } } @Override - public void reserialize(JournalKey key, Accumulator from, DataOutputPlus out, Version version) throws IOException + public TopologyRecord deserialize(DataInputPlus in) throws IOException { - serialize(key, from.write, out, version); + long epoch = in.readUnsignedVInt(); + Kind kind = Kind.values()[in.readUnsignedVInt32()]; + + switch (kind) + { + case New: + return new NewTopology(TopologyUpdateSerializer.instance.deserialize(in)); + case Repeat: + case Image: + Journal.TopologyUpdate update = null; + if (in.readBoolean()) + update = TopologyUpdateSerializer.instance.deserialize(in); + + TopologyImage image = new TopologyImage(epoch, kind, update); + in.readByte(); // defunct enum byte + + image.closed = KeySerializers.ranges.deserialize(in); + image.retired = KeySerializers.ranges.deserialize(in); + return image; + default: + throw new UnhandledEnum(kind); + } } @Override - public void deserialize(JournalKey key, Accumulator into, DataInputPlus in, Version version) throws IOException + public long serializedSize(TopologyRecord t) { - into.read(Serializer.instance.deserialize(in)); + long size = TypeSizes.sizeofUnsignedVInt(t.epoch()); + size += TypeSizes.sizeofUnsignedVInt(t.kind().ordinal()); + + switch (t.kind()) + { + case New: + size += TopologyUpdateSerializer.instance.serializedSize(((NewTopology) t).update); + break; + case Image: + case Repeat: + TopologyImage image = (TopologyImage) t; + size += TypeSizes.BOOL_SIZE; + if (image.update != null) + size += TopologyUpdateSerializer.instance.serializedSize(image.update); + + size += TypeSizes.BYTE_SIZE; + size += KeySerializers.ranges.serializedSize(image.closed); + size += KeySerializers.ranges.serializedSize(image.retired); + break; + default: + throw new UnhandledEnum(t.kind()); + } + return size; } } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/accord/repair/AccordRepair.java b/src/java/org/apache/cassandra/service/accord/repair/AccordRepair.java index 0cd088691d2b..6c691cce0ede 100644 --- a/src/java/org/apache/cassandra/service/accord/repair/AccordRepair.java +++ b/src/java/org/apache/cassandra/service/accord/repair/AccordRepair.java @@ -39,13 +39,13 @@ import org.apache.cassandra.repair.SharedContext; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.accord.AccordEndpointMapper; import org.apache.cassandra.service.accord.AccordService; -import org.apache.cassandra.service.accord.AccordTopology; import org.apache.cassandra.service.accord.IAccordService; import org.apache.cassandra.service.accord.RequestBookkeeping; import org.apache.cassandra.service.accord.TimeOnlyRequestBookkeeping.LatencyRequestBookkeeping; import org.apache.cassandra.service.accord.TokenRange; +import org.apache.cassandra.service.accord.topology.AccordEndpointMapper; +import org.apache.cassandra.service.accord.topology.AccordTopology; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.utils.Pair; diff --git a/src/java/org/apache/cassandra/service/accord/AccordSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/AccordSerializers.java similarity index 97% rename from src/java/org/apache/cassandra/service/accord/AccordSerializers.java rename to src/java/org/apache/cassandra/service/accord/serializers/AccordSerializers.java index 7818d3550b72..402e28ac2507 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/AccordSerializers.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.serializers; import java.io.IOException; import java.nio.ByteBuffer; @@ -45,8 +45,6 @@ import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.accord.serializers.IVersionedSerializer; -import org.apache.cassandra.service.accord.serializers.Version; import static org.apache.cassandra.db.TypeSizes.sizeof; import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt; diff --git a/src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java index d8e9db930799..ddcaec9071b4 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java @@ -21,128 +21,240 @@ import java.io.IOException; import java.util.NavigableMap; import java.util.TreeMap; +import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; import java.util.function.IntFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.api.LocalListeners.TxnListener; import accord.api.RoutingKey; +import accord.impl.cfr.IdEntry; +import accord.impl.cfr.IdMultiEntry; +import accord.impl.cfr.IdSingleEntry; +import accord.impl.progresslog.TxnState; +import accord.local.CommandStores; import accord.local.DurableBefore; +import accord.local.MaxConflicts; +import accord.local.MaxDecidedRX; import accord.local.RedundantBefore; +import accord.local.RejectBefore; import accord.primitives.Range; import accord.primitives.Ranges; +import accord.primitives.SaveStatus; import accord.primitives.Timestamp; import accord.primitives.TxnId; +import accord.utils.BTreeReducingRangeMap; import accord.utils.Invariants; import accord.utils.ReducingRangeMap; +import accord.utils.VIntCoding; +import accord.utils.btree.ReducingBTree; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.io.UnversionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.utils.CollectionSerializers; -import org.apache.cassandra.utils.NullableSerializer; +import org.apache.cassandra.utils.NoSpamLogger; +import static accord.utils.Invariants.illegalState; import static org.apache.cassandra.service.accord.serializers.CommandSerializers.ExecuteAtSerializer.deserializeNullable; import static org.apache.cassandra.service.accord.serializers.CommandSerializers.ExecuteAtSerializer.serializeNullable; import static org.apache.cassandra.service.accord.serializers.CommandSerializers.ExecuteAtSerializer.serializedNullableSize; public class CommandStoreSerializers { + private static final Logger logger = LoggerFactory.getLogger(CommandStoreSerializers.class); + private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES); + private static final int REDUCING_BTREE_MODE = 0; + private static final int REDUCING_ARRAY_MODE = 1; + private static final int REDUCING_MODE_BIT = 1; + private static final int REDUCING_RESERVED_FLAG_BITS = 3; + + public static final UnversionedSerializer durableBefore = new DurableBeforeSerializer(); + public static final UnversionedSerializer maxConflicts = new MaxConflictsSerializer(); + public static final UnversionedSerializer maxDecidedRX = new ReducingRangeMapSerializer<>(new DecidedRXSerializer(), MaxDecidedRX.DecidedRX[]::new, MaxDecidedRX.SerializerSupport::create, MaxDecidedRX.EMPTY); + public static final UnversionedSerializer redundantBeforeShortBounds = new RedundantBeforeShortBoundsSerializer(); + public static final UnversionedSerializer redundantBefore = new ReducingRangeMapSerializer<>(redundantBeforeShortBounds, RedundantBefore.Bounds[]::new, RedundantBefore.SerializerSupport::create, RedundantBefore.EMPTY); + public static final UnversionedSerializer rejectBefore = new ReducingRangeMapSerializer<>(CommandSerializers.timestamp, Timestamp[]::new, RejectBefore.SerializerSupport::create, RejectBefore.EMPTY); + public static final UnversionedSerializer> bootstrapBeganAt = new TimestampToRangesMapSerializer<>(CommandSerializers.txnId); + public static final UnversionedSerializer> safeToRead = new TimestampToRangesMapSerializer<>(CommandSerializers.timestamp); + public static final UnversionedSerializer txnListener = new TxnListenerSerializer(); + public static final UnversionedSerializer progressLogState = new ProgressLogStateSerializer(); + public static final UnversionedSerializer rangeIndexIdEntry = new RangeIndexIdEntrySerializer(); + public static final UnversionedSerializer rangesForEpoch = new RangesForEpochSerializer(); + private CommandStoreSerializers() {} - public static class ReducingRangeMapSerializer> implements UnversionedSerializer + // TODO (expected): use flags to switch to bitset encoding for nulls + private static abstract class AbstractReducingRangeMapSerializer> implements UnversionedSerializer { - final UnversionedSerializer valueSerializer; - final IntFunction newValueArray; - final BiFunction constructor; + final IntFunction newValueArray; + final BiFunction constructor; + final Map empty; - public ReducingRangeMapSerializer(UnversionedSerializer valueSerializer, IntFunction newValueArray, BiFunction constructor) + public AbstractReducingRangeMapSerializer(IntFunction newValueArray, BiFunction constructor, Map empty) { - this.valueSerializer = valueSerializer; this.newValueArray = newValueArray; this.constructor = constructor; + this.empty = empty; + } + + protected abstract int flags(Map map); + protected abstract UnversionedSerializer valueSerializer(int flags); + + private int safeFlags(Map map) + { + int flags = flags(map); + Invariants.require((flags & ((1 << REDUCING_RESERVED_FLAG_BITS) - 1)) == 0); + return flags | REDUCING_ARRAY_MODE; } @Override - public void serialize(R map, DataOutputPlus out) throws IOException + public void serialize(Map map, DataOutputPlus out) throws IOException { - out.writeBoolean(true); + int flags = safeFlags(map); int mapSize = map.size(); + out.writeUnsignedVInt32(flags); out.writeUnsignedVInt32(mapSize); + if (mapSize == 0) + return; + + UnversionedSerializer valueSerializer = valueSerializer(flags); + for (int i=0; i 0) - KeySerializers.routingKey.serialize(map.startAt(mapSize), out); + KeySerializers.routingKey.serialize(map.startAt(mapSize), out); } @Override - public R deserialize(DataInputPlus in) throws IOException + public Map deserialize(DataInputPlus in) throws IOException { - in.readBoolean(); + int flags = in.readUnsignedVInt32(); int mapSize = in.readUnsignedVInt32(); + + if (mapSize == 0) + return empty; + RoutingKey[] keys = new RoutingKey[mapSize + 1]; - T[] values = newValueArray.apply(mapSize); + V[] values = newValueArray.apply(mapSize); + UnversionedSerializer valueSerializer = valueSerializer(flags); for (int i=0; i 0) - keys[mapSize] = KeySerializers.routingKey.deserialize(in); + keys[mapSize] = KeySerializers.routingKey.deserialize(in); + return constructor.apply(keys, values); } @Override - public long serializedSize(R map) + public long serializedSize(Map map) { - long size = TypeSizes.BOOL_SIZE; + int flags = safeFlags(map); int mapSize = map.size(); + + long size = 0; + size += TypeSizes.sizeofUnsignedVInt(flags); size += TypeSizes.sizeofUnsignedVInt(mapSize); + + if (mapSize == 0) + return size; + + UnversionedSerializer valueSerializer = valueSerializer(flags); for (int i=0; i 0) - size += KeySerializers.routingKey.serializedSize(map.startAt(mapSize)); - + size += KeySerializers.routingKey.serializedSize(map.startAt(mapSize)); return size; } } - public static UnversionedSerializer durableBefore = new ReducingRangeMapSerializer<>(NullableSerializer.wrap(new UnversionedSerializer() + static class ReducingRangeMapSerializer> extends AbstractReducingRangeMapSerializer implements UnversionedSerializer { + final UnversionedSerializer defaultValueSerializer; + + public ReducingRangeMapSerializer(UnversionedSerializer defaultValueSerializer, IntFunction newValueArray, BiFunction constructor, Map empty) + { + super(newValueArray, constructor, empty); + this.defaultValueSerializer = defaultValueSerializer; + } + @Override - public void serialize(DurableBefore.Entry t, DataOutputPlus out) throws IOException + protected int flags(Map map) { - CommandSerializers.txnId.serialize(t.quorumBefore, out); - CommandSerializers.txnId.serialize(t.universalBefore, out); + return 0; } @Override - public DurableBefore.Entry deserialize(DataInputPlus in) throws IOException + protected UnversionedSerializer valueSerializer(int flags) { - TxnId quorumBefore = CommandSerializers.txnId.deserialize(in); - TxnId universalBefore = CommandSerializers.txnId.deserialize(in); - return new DurableBefore.Entry(quorumBefore, universalBefore); + return defaultValueSerializer; } + } + + private static final class DecidedRXSerializer implements UnversionedSerializer + { + private DecidedRXSerializer() {} @Override - public long serializedSize(DurableBefore.Entry t) + public void serialize(MaxDecidedRX.DecidedRX t, DataOutputPlus out) throws IOException { - return CommandSerializers.txnId.serializedSize(t.quorumBefore) - + CommandSerializers.txnId.serializedSize(t.universalBefore); + if (t == null) + { + CommandSerializers.txnId.serialize(null, out); + } + else + { + CommandSerializers.txnId.serialize(t.any, out); + CommandSerializers.txnId.serialize(t.hlcBound, out); + } } - }), DurableBefore.Entry[]::new, DurableBefore.SerializerSupport::create); - public static final UnversionedSerializer redundantBeforeEntry = new UnversionedSerializer<>() + @Override + public MaxDecidedRX.DecidedRX deserialize(DataInputPlus in) throws IOException + { + TxnId any = CommandSerializers.txnId.deserialize(in); + if (any == null) + return null; + TxnId hlcBound = CommandSerializers.txnId.deserialize(in); + return new MaxDecidedRX.DecidedRX(any, hlcBound); + } + + @Override + public long serializedSize(MaxDecidedRX.DecidedRX t) + { + if (t == null) + return CommandSerializers.txnId.serializedSize(null); + + return CommandSerializers.txnId.serializedSize(t.any) + + CommandSerializers.txnId.serializedSize(t.hlcBound); + } + } + + private static class RedundantBeforeShortBoundsSerializer implements UnversionedSerializer { + private RedundantBeforeShortBoundsSerializer() {} + @Override public void serialize(RedundantBefore.Bounds b, DataOutputPlus out) throws IOException { + // was previously wrapped in NullableSerializer; inlined logic here so we can convert to flags in future and save bytes + if (b == null) + { + out.writeByte(0); + return; + } + out.writeByte(1); + KeySerializers.range.serialize(b.range, out); Invariants.require(b.startEpoch <= b.endEpoch); out.writeUnsignedVInt(b.startEpoch); @@ -164,13 +276,16 @@ public void serialize(RedundantBefore.Bounds b, DataOutputPlus out) throws IOExc private short cast(long v) { if ((v & ~0xFFFF) != 0) - throw new IllegalStateException("Cannot serialize RedundantStatus larger than 0xFFFF. Requires serialization version bump."); + throw new IllegalStateException("Cannot serialize RedundantStatus larger than 0xFFFF. Requires serialization changes."); return (short)v; } @Override public RedundantBefore.Bounds deserialize(DataInputPlus in) throws IOException { + if (in.readByte() == 0) + return null; + Range range = KeySerializers.range.deserialize(in); long startEpoch = in.readUnsignedVInt(); long endEpoch = in.readUnsignedVInt(); @@ -192,7 +307,10 @@ public RedundantBefore.Bounds deserialize(DataInputPlus in) throws IOException @Override public long serializedSize(RedundantBefore.Bounds b) { - long size = KeySerializers.range.serializedSize(b.range); + if (b == null) + return 1; + + long size = 1 + KeySerializers.range.serializedSize(b.range); size += TypeSizes.sizeofUnsignedVInt(b.startEpoch); size += TypeSizes.sizeofUnsignedVInt(b.endEpoch == Long.MAX_VALUE ? 0 : 1 + b.endEpoch - b.startEpoch); size += serializedNullableSize(b.staleUntilAtLeast); @@ -204,14 +322,13 @@ public long serializedSize(RedundantBefore.Bounds b) size += 2L * 2 * b.bounds.length; return size; } - }; - public static UnversionedSerializer redundantBefore = new ReducingRangeMapSerializer<>(NullableSerializer.wrap(redundantBeforeEntry), RedundantBefore.Bounds[]::new, RedundantBefore.SerializerSupport::create); + } - private static class TimestampToRangesSerializer implements UnversionedSerializer> + private static class TimestampToRangesMapSerializer implements UnversionedSerializer> { private final UnversionedSerializer timestampSerializer; - public TimestampToRangesSerializer(UnversionedSerializer timestampSerializer) + public TimestampToRangesMapSerializer(UnversionedSerializer timestampSerializer) { this.timestampSerializer = timestampSerializer; } @@ -236,6 +353,502 @@ public long serializedSize(NavigableMap map) } } - public static final UnversionedSerializer> bootstrapBeganAt = new TimestampToRangesSerializer<>(CommandSerializers.txnId); - public static final UnversionedSerializer> safeToRead = new TimestampToRangesSerializer<>(CommandSerializers.timestamp); + private static abstract class BTreeReducingRangeMapSerializer, Map extends BTreeReducingRangeMap> implements UnversionedSerializer + { + private static final int RESERVED_MAP_MASK = 0x3; + + private static final int DISCONTIGUOUS = 1; + private static final int NEW_PREFIX = 2; + + public BTreeReducingRangeMapSerializer() + { + } + + abstract Map empty(); + abstract BTreeReducingRangeMap.Builder builder(); + abstract void serializeWithoutRange(E e, DataOutputPlus out) throws IOException; + abstract long serializedSizeWithoutRange(E e); + abstract E deserialize(RoutingKey start, RoutingKey end, DataInputPlus in, int mapFlags) throws IOException; + abstract E deserializeArrayModeWithoutRange(DataInputPlus in) throws IOException; + + protected int mapFlags() { return 0; } + + @Override + public void serialize(Map map, DataOutputPlus out) throws IOException + { + // for upgrading non-tree structures + int mapFlags = mapFlags(); + Invariants.require((mapFlags & RESERVED_MAP_MASK) == 0); + mapFlags |= REDUCING_BTREE_MODE; + int mapSize = map.size(); + out.writeUnsignedVInt32(mapFlags); + out.writeUnsignedVInt32(mapSize); + + if (mapSize == 0) + return; + + E prev = null; + int fixedLength = 0; + for (E e : map) + { + int flags = 0; + if (prev == null) + { + flags = NEW_PREFIX | DISCONTIGUOUS; + } + else + { + int c = prev.end().compareTo(e.start()); + if (c > 0) + throw illegalState("Not well-formed: %s overlaps %s in %s", prev, e, map); + + if (c < 0) + { + flags = DISCONTIGUOUS; + if (!prev.prefix().equals(e.prefix())) + flags |= NEW_PREFIX; + } + out.writeByte(flags); + } + + if ((flags & DISCONTIGUOUS) != 0) + { + if ((flags & NEW_PREFIX) != 0) + { + KeySerializers.routingKey.serializePrefix(e.prefix(), out); + fixedLength = KeySerializers.routingKey.fixedKeyLengthForPrefix(e.prefix()); + } + if (fixedLength < 0) + out.writeUnsignedVInt32(KeySerializers.routingKey.serializedSizeWithoutPrefixOrLength(e.start())); + KeySerializers.routingKey.serializeWithoutPrefixOrLength(e.start(), out); + } + if (fixedLength < 0) + out.writeUnsignedVInt32(KeySerializers.routingKey.serializedSizeWithoutPrefixOrLength(e.end())); + KeySerializers.routingKey.serializeWithoutPrefixOrLength(e.end(), out); + serializeWithoutRange(e, out); + prev = e; + } + } + + @Override + public Map deserialize(DataInputPlus in) throws IOException + { + int mapFlags = in.readUnsignedVInt32(); + int mapSize = in.readUnsignedVInt32(); + + if (mapSize == 0) + return empty(); + + try (BTreeReducingRangeMap.Builder builder = builder()) + { + if ((mapFlags & REDUCING_MODE_BIT) == REDUCING_BTREE_MODE) + { + Object prefix = null; + RoutingKey prevEnd = null; + E prev = null; + int fixedLength = 0; + while (mapSize-- > 0) + { + int flags; + if (prefix == null) flags = NEW_PREFIX | DISCONTIGUOUS; + else flags = in.readByte(); + + RoutingKey start; + if ((flags & DISCONTIGUOUS) == 0) + { + start = prevEnd; + } + else + { + if ((flags & NEW_PREFIX) != 0) + { + prefix = KeySerializers.routingKey.deserializePrefix(in); + fixedLength = KeySerializers.routingKey.fixedKeyLengthForPrefix(in); + } + int length = fixedLength >= 0 ? fixedLength : in.readUnsignedVInt32(); + start = KeySerializers.routingKey.deserializeWithPrefix(prefix, length, in); + } + + int length = fixedLength >= 0 ? fixedLength : in.readUnsignedVInt32(); + RoutingKey end = KeySerializers.routingKey.deserializeWithPrefix(prefix, length, in); + E cur = deserialize(start, end, in, mapFlags); + if ((flags & DISCONTIGUOUS) != 0) + { + if (prev != null && prev.end().compareTo(start) > 0) + { + if (prev.end().compareTo(end) > 0) + { + noSpamLogger.warn("BTreeReducingRangeMap not well-formed: {} not before {}; skipping", prev, cur); + prevEnd = end; + continue; + } + else + { + E newCur = cur.with(prev.end(), end); + noSpamLogger.warn("BTreeReducingRangeMap not well-formed: {} not before {}; appending {}", prev, cur, newCur); + cur = newCur; + } + } + } + builder.append(cur); + prevEnd = end; + prev = cur; + } + } + else + { + // read linear format for upgrading from non-tree versions of collections + E prev = null; + RoutingKey prevStart = null; + while (mapSize-- > 0) + { + RoutingKey prevEnd = KeySerializers.routingKey.deserialize(in); + if (prev != null) + builder.append(prev.with(prevStart, prevEnd)); + prev = deserializeArrayModeWithoutRange(in); + prevStart = prevEnd; + } + RoutingKey prevEnd = KeySerializers.routingKey.deserialize(in); + if (prev != null) + builder.append(prev.with(prevStart, prevEnd)); + + } + return builder.build(); + } + + } + + @Override + public long serializedSize(Map map) + { + // for upgrading non-tree structures + // noinspection UnnecessaryLocalVariable + int mapFlags = REDUCING_BTREE_MODE; + int mapSize = map.size(); + + long size = TypeSizes.sizeofUnsignedVInt(mapFlags); + size += TypeSizes.sizeofUnsignedVInt(mapSize); + + if (mapSize == 0) + return size; + + E prev = null; + int fixedLength = 0; + for (E e : map) + { + int flags = 0; + if (prev == null) + { + fixedLength = KeySerializers.routingKey.fixedKeyLengthForPrefix(e.prefix()); + flags = NEW_PREFIX | DISCONTIGUOUS; + } + else + { + if (!prev.end().equals(e.start())) + { + flags = DISCONTIGUOUS; + if (!prev.prefix().equals(e.prefix())) + flags |= NEW_PREFIX; + } + size += 1; + } + + if ((flags & DISCONTIGUOUS) != 0) + { + if ((flags & NEW_PREFIX) != 0) + { + size += KeySerializers.routingKey.serializedSizeOfPrefix(e.prefix()); + fixedLength = KeySerializers.routingKey.fixedKeyLengthForPrefix(e.prefix()); + } + if (fixedLength < 0) + size += VIntCoding.sizeOfUnsignedVInt(KeySerializers.routingKey.serializedSizeWithoutPrefixOrLength(e.start())); + size += KeySerializers.routingKey.serializedSizeWithoutPrefixOrLength(e.start()); + } + if (fixedLength < 0) + size += VIntCoding.sizeOfUnsignedVInt(KeySerializers.routingKey.serializedSizeWithoutPrefixOrLength(e.start())); + size += KeySerializers.routingKey.serializedSizeWithoutPrefixOrLength(e.end()); + size += serializedSizeWithoutRange(e); + prev = e; + } + + return size; + } + } + + private static final class MaxConflictsSerializer extends BTreeReducingRangeMapSerializer + { + // use top bits of a single byte vint, to leave room for base impl to fill other way + private static final int SEPARATE_WRITES = 0x40; + + private MaxConflictsSerializer() {} + + @Override + protected int mapFlags() + { + return SEPARATE_WRITES; + } + + @Override + MaxConflicts empty() + { + return MaxConflicts.EMPTY; + } + + @Override + BTreeReducingRangeMap.Builder builder() + { + return new MaxConflicts.Builder(); + } + + @Override + void serializeWithoutRange(MaxConflicts.Entry entry, DataOutputPlus out) throws IOException + { + CommandSerializers.timestamp.serialize(entry.any, out); + CommandSerializers.timestamp.serialize(entry.write, out); + } + + @Override + long serializedSizeWithoutRange(MaxConflicts.Entry entry) + { + return CommandSerializers.timestamp.serializedSize(entry.any) + + CommandSerializers.timestamp.serializedSize(entry.write); + } + + @Override + MaxConflicts.Entry deserialize(RoutingKey start, RoutingKey end, DataInputPlus in, int mapFlags) throws IOException + { + Timestamp all = CommandSerializers.timestamp.deserialize(in); + Timestamp writes = all; + if ((mapFlags & SEPARATE_WRITES) != 0) + writes = CommandSerializers.timestamp.deserialize(in); + return new MaxConflicts.Entry(start, end, all, writes); + } + + @Override + MaxConflicts.Entry deserializeArrayModeWithoutRange(DataInputPlus in) throws IOException + { + Timestamp all = CommandSerializers.timestamp.deserialize(in); + return new MaxConflicts.Entry(all, all); + } + } + + private static final class DurableBeforeSerializer extends BTreeReducingRangeMapSerializer + { + private DurableBeforeSerializer() {} + + @Override + DurableBefore empty() + { + return DurableBefore.EMPTY; + } + + @Override + DurableBefore.Builder builder() + { + return new DurableBefore.Builder(); + } + + @Override + void serializeWithoutRange(DurableBefore.Entry entry, DataOutputPlus out) throws IOException + { + CommandSerializers.txnId.serialize(entry.quorum, out); + CommandSerializers.txnId.serialize(entry.universal, out); + } + + @Override + long serializedSizeWithoutRange(DurableBefore.Entry entry) + { + return CommandSerializers.txnId.serializedSize(entry.quorum) + + CommandSerializers.txnId.serializedSize(entry.universal); + } + + @Override + DurableBefore.Entry deserialize(RoutingKey start, RoutingKey end, DataInputPlus in, int mapFlags) throws IOException + { + TxnId quorum = CommandSerializers.txnId.deserialize(in); + TxnId universal = CommandSerializers.txnId.deserialize(in); + return new DurableBefore.Entry(start, end, quorum, universal); + } + + @Override + DurableBefore.Entry deserializeArrayModeWithoutRange(DataInputPlus in) throws IOException + { + if (!in.readBoolean()) + return null; + TxnId quorum = CommandSerializers.txnId.deserialize(in); + TxnId universal = CommandSerializers.txnId.deserialize(in); + return DurableBefore.Entry.constructWithoutRange(quorum, universal); + } + } + + private static final class TxnListenerSerializer implements UnversionedSerializer + { + private TxnListenerSerializer() {} + + @Override + public void serialize(TxnListener t, DataOutputPlus out) throws IOException + { + if (t == null) + { + CommandSerializers.txnId.serialize(null, out); + } + else + { + CommandSerializers.txnId.serialize(t.waiter, out); + CommandSerializers.txnId.serialize(t.waitingOn, out); + CommandSerializers.saveStatus.serialize(t.awaitingStatus, out); + } + } + + @Override + public TxnListener deserialize(DataInputPlus in) throws IOException + { + TxnId waiter = CommandSerializers.txnId.deserialize(in); + if (waiter == null) + return null; + TxnId waitingOn = CommandSerializers.txnId.deserialize(in); + SaveStatus awaitingStatus = CommandSerializers.saveStatus.deserialize(in); + return new TxnListener(waiter, waitingOn, awaitingStatus); + } + + @Override + public long serializedSize(TxnListener t) + { + if (t == null) + return CommandSerializers.txnId.serializedSize(null); + + return CommandSerializers.txnId.serializedSize(t.waiter) + + CommandSerializers.txnId.serializedSize(t.waitingOn) + + CommandSerializers.saveStatus.serializedSize(t.awaitingStatus); + } + } + + private static final class ProgressLogStateSerializer implements UnversionedSerializer + { + private ProgressLogStateSerializer() {} + + @Override + public void serialize(TxnState t, DataOutputPlus out) throws IOException + { + if (t == null) + { + CommandSerializers.txnId.serialize(null, out); + } + else + { + CommandSerializers.txnId.serialize(t.txnId, out); + out.writeLong(t.encodedState()); + } + } + + @Override + public TxnState deserialize(DataInputPlus in) throws IOException + { + TxnId txnId = CommandSerializers.txnId.deserialize(in); + if (txnId == null) + return null; + long encodedState = in.readLong(); + return TxnState.SerializationSupport.create(txnId, encodedState); + } + + @Override + public long serializedSize(TxnState t) + { + if (t == null) + return CommandSerializers.txnId.serializedSize(null); + + return CommandSerializers.txnId.serializedSize(t.txnId) + TypeSizes.LONG_SIZE; + } + } + + private static final class RangeIndexIdEntrySerializer implements UnversionedSerializer + { + private RangeIndexIdEntrySerializer() {} + + @Override + public void serialize(IdEntry t, DataOutputPlus out) throws IOException + { + byte flags = (byte) ((t.getClass() == IdSingleEntry.class) ? 0 : 1); + out.writeByte(flags); + CommandSerializers.txnId.serialize(t, out); + out.writeUnsignedVInt32(t.encoded()); + if (flags == 0) + { + IdSingleEntry e = (IdSingleEntry) t; + KeySerializers.range.serialize(e.range, out); + } + else + { + IdMultiEntry e = (IdMultiEntry) t; + KeySerializers.ranges.serialize(e.ranges, out); + } + } + + @Override + public IdEntry deserialize(DataInputPlus in) throws IOException + { + byte flags = in.readByte(); + TxnId txnId = CommandSerializers.txnId.deserialize(in); + int encoded = in.readUnsignedVInt32(); + if (flags == 0) + { + Range range = KeySerializers.range.deserialize(in); + return IdEntry.SerializerSupport.create(txnId, encoded, range); + } + else + { + Ranges ranges = KeySerializers.ranges.deserialize(in); + return IdEntry.SerializerSupport.create(txnId, encoded, ranges); + } + } + + @Override + public long serializedSize(IdEntry t) + { + return 1 + CommandSerializers.txnId.serializedSize(t) + + (t.getClass() == IdSingleEntry.class ? KeySerializers.range.serializedSize(((IdSingleEntry)t).range) + : KeySerializers.ranges.serializedSize(((IdMultiEntry)t).ranges)); + } + } + + static class RangesForEpochSerializer implements UnversionedSerializer + { + @Override + public void serialize(CommandStores.RangesForEpoch from, DataOutputPlus out) throws IOException + { + out.writeUnsignedVInt32(from.size()); + for (int i = 0; i < from.size(); i++) + { + out.writeLong(from.epochAtIndex(i)); + KeySerializers.ranges.serialize(from.rangesAtIndex(i), out); + } + } + + @Override + public CommandStores.RangesForEpoch deserialize(DataInputPlus in) throws IOException + { + int size = in.readUnsignedVInt32(); + Ranges[] ranges = new Ranges[size]; + long[] epochs = new long[size]; + for (int i = 0; i < ranges.length; i++) + { + epochs[i] = in.readLong(); + ranges[i] = KeySerializers.ranges.deserialize(in); + } + return new CommandStores.RangesForEpoch(epochs, ranges); + } + + @Override + public long serializedSize(CommandStores.RangesForEpoch from) + { + long size = TypeSizes.sizeofUnsignedVInt(from.size()); + for (int i = 0; i < from.size(); i++) + { + size += TypeSizes.LONG_SIZE; + size += KeySerializers.ranges.serializedSize(from.rangesAtIndex(i)); + } + return size; + } + } + } diff --git a/src/java/org/apache/cassandra/service/accord/serializers/EncodeAsVInt32.java b/src/java/org/apache/cassandra/service/accord/serializers/EncodeAsVInt32.java index 81cae8885f2f..9572c323004e 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/EncodeAsVInt32.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/EncodeAsVInt32.java @@ -54,6 +54,12 @@ public static > EncodeAsVInt32 of(Class clazz) return withNulls(Enum::ordinal, i -> values[i]); } + public static > EncodeAsVInt32 withoutNulls(Class clazz) + { + E[] values = clazz.getEnumConstants(); + return withoutNulls(Enum::ordinal, i -> values[i]); + } + static class WithNulls extends EncodeAsVInt32 { private WithNulls(ToIntFunction encode, IntFunction decode) diff --git a/src/java/org/apache/cassandra/service/accord/serializers/GetDurableBeforeSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/GetDurableBeforeSerializers.java index 1c64d281fdb3..ef6b130381e7 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/GetDurableBeforeSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/GetDurableBeforeSerializers.java @@ -28,7 +28,7 @@ public class GetDurableBeforeSerializers { - public static final UnversionedSerializer request = new UnversionedSerializer() + public static final UnversionedSerializer request = new UnversionedSerializer<>() { @Override public void serialize(GetDurableBefore msg, DataOutputPlus out) throws IOException diff --git a/src/java/org/apache/cassandra/service/accord/serializers/KeySerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/KeySerializers.java index 8cf9f8b463d6..8168788f6d8f 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/KeySerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/KeySerializers.java @@ -914,7 +914,7 @@ final int recordCountToLengthCount(int recordCount) @Override final int serializedSizeWithoutPrefix(RoutingKey routable) { - return routingKey.serializedSizeWithoutPrefix(routable); + return routingKey.serializedSizeWithoutPrefixOrLength(routable); } @Override @@ -1067,8 +1067,8 @@ int recordCountToLengthCount(int recordCount) @Override final int serializedSizeWithoutPrefix(Range range) { - return routingKey.serializedSizeWithoutPrefix(range.start()) - + routingKey.serializedSizeWithoutPrefix(range.end()); + return routingKey.serializedSizeWithoutPrefixOrLength(range.start()) + + routingKey.serializedSizeWithoutPrefixOrLength(range.end()); } @Override @@ -1085,9 +1085,9 @@ final void serializeOffsets(Range[] ranges, int startIndex, int endIndex, DataOu for (int i = startIndex; i < endIndex; ++i) { Range r = ranges[i]; - endOffset += routingKey.serializedSizeWithoutPrefix(r.start()); + endOffset += routingKey.serializedSizeWithoutPrefixOrLength(r.start()); out.writeInt(endOffset); - endOffset += routingKey.serializedSizeWithoutPrefix(r.end()); + endOffset += routingKey.serializedSizeWithoutPrefixOrLength(r.end()); out.writeInt(endOffset); } } diff --git a/src/java/org/apache/cassandra/service/accord/serializers/LatestDepsSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/LatestDepsSerializers.java index 2e08adde7d4a..9a20d6afbd0d 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/LatestDepsSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/LatestDepsSerializers.java @@ -22,7 +22,9 @@ import accord.api.RoutingKey; import accord.messages.GetLatestDeps; +import accord.messages.GetLatestDeps.GetLatestDepsNack; import accord.messages.GetLatestDeps.GetLatestDepsOk; +import accord.messages.GetLatestDeps.GetLatestDepsReply; import accord.primitives.Ballot; import accord.primitives.Deps; import accord.primitives.Known; @@ -45,81 +47,21 @@ public class LatestDepsSerializers public void serialize(LatestDeps t, DataOutputPlus out) throws IOException { out.writeUnsignedVInt32(t.size()); - if (t.size() == 0) - return; - - for (int i = 0 ; i < t.size() ; ++i) - { - RoutingKey start = t.startAt(i); - KeySerializers.routingKey.serialize(start, out); - LatestDeps.LatestEntry e = t.valueAt(i); - if (e == null) - { - CommandSerializers.knownDeps.serialize(null, out); - } - else - { - CommandSerializers.knownDeps.serialize(e.known, out); - CommandSerializers.ballot.serialize(e.ballot, out); - DepsSerializers.nullableDeps.serialize(e.coordinatedDeps, out); - DepsSerializers.nullableDeps.serialize(e.localDeps, out); - } - } - KeySerializers.routingKey.serialize(t.startAt(t.size()), out); + serializeWithoutSize(t, out); } @Override public LatestDeps deserialize(DataInputPlus in) throws IOException { int size = in.readUnsignedVInt32(); - if (size == 0) - return LatestDeps.EMPTY; - - RoutingKey[] starts = new RoutingKey[size + 1]; - LatestDeps.LatestEntry[] values = new LatestDeps.LatestEntry[size]; - for (int i = 0 ; i < size ; ++i) - { - starts[i] = KeySerializers.routingKey.deserialize(in); - Known.KnownDeps knownDeps = CommandSerializers.knownDeps.deserialize(in); - if (knownDeps == null) - continue; - - Ballot ballot = CommandSerializers.ballot.deserialize(in); - Deps coordinatedDeps = DepsSerializers.nullableDeps.deserialize(in); - Deps localDeps = DepsSerializers.nullableDeps.deserialize(in); - values[i] = new LatestDeps.LatestEntry(knownDeps, ballot, coordinatedDeps, localDeps); - } - starts[size] = KeySerializers.routingKey.deserialize(in); - - return LatestDeps.SerializerSupport.create(starts, values); + return deserializeWithSize(size, in); } @Override public long serializedSize(LatestDeps t) { - long size = 0; - size += TypeSizes.sizeofUnsignedVInt(t.size()); - if (t.size() == 0) - return size; - for (int i = 0 ; i < t.size() ; ++i) - { - RoutingKey start = t.startAt(i); - size += KeySerializers.routingKey.serializedSize(start); - LatestDeps.LatestEntry e = t.valueAt(i); - if (e == null) - { - size += CommandSerializers.knownDeps.serializedSize(null); - } - else - { - size += CommandSerializers.knownDeps.serializedSize(e.known); - size += CommandSerializers.ballot.serializedSize(e.ballot); - size += DepsSerializers.nullableDeps.serializedSize(e.coordinatedDeps); - size += DepsSerializers.nullableDeps.serializedSize(e.localDeps); - } - } - size += KeySerializers.routingKey.serializedSize(t.startAt(t.size())); - return size; + return TypeSizes.sizeofUnsignedVInt(t.size()) + + serializedSizeWithoutSize(t); } }; @@ -148,24 +90,105 @@ public long serializedBodySize(GetLatestDeps msg, Version version) } }; - public static final UnversionedSerializer reply = new UnversionedSerializer<>() + public static final UnversionedSerializer reply = new UnversionedSerializer<>() { + static final long NACK = 0x80000000L; @Override - public void serialize(GetLatestDepsOk reply, DataOutputPlus out) throws IOException + public void serialize(GetLatestDepsReply reply, DataOutputPlus out) throws IOException { - latestDeps.serialize(reply.deps, out); + if (reply.isOk()) latestDeps.serialize(((GetLatestDepsOk)reply).deps, out); + else out.writeUnsignedVInt(NACK); } @Override - public GetLatestDepsOk deserialize(DataInputPlus in) throws IOException + public GetLatestDepsReply deserialize(DataInputPlus in) throws IOException { - return new GetLatestDepsOk(latestDeps.deserialize(in)); + long size = in.readUnsignedVInt(); + if (size == NACK) + return GetLatestDepsNack.INSTANCE; + return new GetLatestDepsOk(deserializeWithSize(Math.toIntExact(size), in)); } @Override - public long serializedSize(GetLatestDepsOk reply) + public long serializedSize(GetLatestDepsReply reply) { - return latestDeps.serializedSize(reply.deps); + if (reply.isOk()) return latestDeps.serializedSize(((GetLatestDepsOk)reply).deps); + else return TypeSizes.sizeofUnsignedVInt(NACK); } }; -} + + private static void serializeWithoutSize(LatestDeps t, DataOutputPlus out) throws IOException + { + if (t.size() == 0) + return; + + for (int i = 0 ; i < t.size() ; ++i) + { + RoutingKey start = t.startAt(i); + KeySerializers.routingKey.serialize(start, out); + LatestDeps.LatestEntry e = t.valueAt(i); + if (e == null) + { + CommandSerializers.knownDeps.serialize(null, out); + } + else + { + CommandSerializers.knownDeps.serialize(e.known, out); + CommandSerializers.ballot.serialize(e.ballot, out); + DepsSerializers.nullableDeps.serialize(e.coordinatedDeps, out); + DepsSerializers.nullableDeps.serialize(e.localDeps, out); + } + } + KeySerializers.routingKey.serialize(t.startAt(t.size()), out); + } + + public static LatestDeps deserializeWithSize(int size, DataInputPlus in) throws IOException + { + if (size == 0) + return LatestDeps.EMPTY; + + RoutingKey[] starts = new RoutingKey[size + 1]; + LatestDeps.LatestEntry[] values = new LatestDeps.LatestEntry[size]; + for (int i = 0 ; i < size ; ++i) + { + starts[i] = KeySerializers.routingKey.deserialize(in); + Known.KnownDeps knownDeps = CommandSerializers.knownDeps.deserialize(in); + if (knownDeps == null) + continue; + + Ballot ballot = CommandSerializers.ballot.deserialize(in); + Deps coordinatedDeps = DepsSerializers.nullableDeps.deserialize(in); + Deps localDeps = DepsSerializers.nullableDeps.deserialize(in); + values[i] = new LatestDeps.LatestEntry(knownDeps, ballot, coordinatedDeps, localDeps); + } + starts[size] = KeySerializers.routingKey.deserialize(in); + + return LatestDeps.SerializerSupport.create(starts, values); + } + + private static long serializedSizeWithoutSize(LatestDeps t) + { + long size = 0; + if (t.size() == 0) + return size; + + for (int i = 0 ; i < t.size() ; ++i) + { + RoutingKey start = t.startAt(i); + size += KeySerializers.routingKey.serializedSize(start); + LatestDeps.LatestEntry e = t.valueAt(i); + if (e == null) + { + size += CommandSerializers.knownDeps.serializedSize(null); + } + else + { + size += CommandSerializers.knownDeps.serializedSize(e.known); + size += CommandSerializers.ballot.serializedSize(e.ballot); + size += DepsSerializers.nullableDeps.serializedSize(e.coordinatedDeps); + size += DepsSerializers.nullableDeps.serializedSize(e.localDeps); + } + } + size += KeySerializers.routingKey.serializedSize(t.startAt(t.size())); + return size; + }} diff --git a/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java index ab6063819644..56ad82df4086 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java @@ -24,7 +24,6 @@ import javax.annotation.Nullable; import accord.api.Result; -import accord.api.RoutingKey; import accord.messages.BeginRecovery; import accord.messages.BeginRecovery.RecoverNack; import accord.messages.BeginRecovery.RecoverOk; @@ -32,7 +31,6 @@ import accord.primitives.Ballot; import accord.primitives.Deps; import accord.primitives.FullRoute; -import accord.primitives.Known.KnownDeps; import accord.primitives.LatestDeps; import accord.primitives.PartialTxn; import accord.primitives.Participants; @@ -43,7 +41,6 @@ import accord.primitives.Writes; import org.apache.cassandra.db.TypeSizes; -import org.apache.cassandra.io.UnversionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.service.accord.serializers.CommandSerializers.ExecuteAtSerializer; @@ -51,6 +48,7 @@ import org.apache.cassandra.utils.vint.VIntCoding; import static accord.messages.BeginRecovery.RecoverReply.Kind.Ok; +import static org.apache.cassandra.service.accord.serializers.LatestDepsSerializers.latestDeps; public class RecoverySerializers { @@ -204,80 +202,4 @@ public long serializedSize(RecoverReply reply, Version version) + (reply.kind() == Ok ? serializedOkSize((RecoverOk) reply, version) : serializedNackSize((RecoverNack) reply, version)); } }; - - public static final UnversionedSerializer latestDeps = new UnversionedSerializer<>() - { - @Override - public void serialize(LatestDeps t, DataOutputPlus out) throws IOException - { - out.writeUnsignedVInt32(t.size()); - for (int i = 0 ; i < t.size() ; ++i) - { - RoutingKey start = t.startAt(i); - KeySerializers.routingKey.serialize(start, out); - LatestDeps.LatestEntry e = t.valueAt(i); - if (e == null) - { - CommandSerializers.knownDeps.serialize(null, out); - } - else - { - CommandSerializers.knownDeps.serialize(e.known, out); - CommandSerializers.ballot.serialize(e.ballot, out); - DepsSerializers.nullableDeps.serialize(e.coordinatedDeps, out); - DepsSerializers.nullableDeps.serialize(e.localDeps, out); - } - } - KeySerializers.routingKey.serialize(t.startAt(t.size()), out); - } - - @Override - public LatestDeps deserialize(DataInputPlus in) throws IOException - { - int size = in.readUnsignedVInt32(); - RoutingKey[] starts = new RoutingKey[size + 1]; - LatestDeps.LatestEntry[] values = new LatestDeps.LatestEntry[size]; - for (int i = 0 ; i < size ; ++i) - { - starts[i] = KeySerializers.routingKey.deserialize(in); - KnownDeps knownDeps = CommandSerializers.knownDeps.deserialize(in); - if (knownDeps == null) - continue; - - Ballot ballot = CommandSerializers.ballot.deserialize(in); - Deps coordinatedDeps = DepsSerializers.nullableDeps.deserialize(in); - Deps localDeps = DepsSerializers.nullableDeps.deserialize(in); - values[i] = new LatestDeps.LatestEntry(knownDeps, ballot, coordinatedDeps, localDeps); - } - starts[size] = KeySerializers.routingKey.deserialize(in); - - return LatestDeps.SerializerSupport.create(starts, values); - } - - @Override - public long serializedSize(LatestDeps t) - { - long size = 0; - size += TypeSizes.sizeofUnsignedVInt(t.size()); - for (int i = 0 ; i < t.size() ; ++i) - { - RoutingKey start = t.startAt(i); - size += KeySerializers.routingKey.serializedSize(start); - LatestDeps.LatestEntry e = t.valueAt(i); - if (e == null) - { - size += CommandSerializers.knownDeps.serializedSize(null); - } - else - { - size += CommandSerializers.knownDeps.serializedSize(e.known); - size += CommandSerializers.ballot.serializedSize(e.ballot); - size += DepsSerializers.nullableDeps.serializedSize(e.coordinatedDeps); - size += DepsSerializers.nullableDeps.serializedSize(e.localDeps); - } - } - size += KeySerializers.routingKey.serializedSize(t.startAt(t.size())); - return size; - } - }; } diff --git a/src/java/org/apache/cassandra/service/accord/ImmediateAsyncExecutor.java b/src/java/org/apache/cassandra/service/accord/serializers/SimpleReplySerializer.java similarity index 58% rename from src/java/org/apache/cassandra/service/accord/ImmediateAsyncExecutor.java rename to src/java/org/apache/cassandra/service/accord/serializers/SimpleReplySerializer.java index d5c54b0c9b28..b0f1ec3f499b 100644 --- a/src/java/org/apache/cassandra/service/accord/ImmediateAsyncExecutor.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/SimpleReplySerializer.java @@ -16,28 +16,13 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.serializers; -import javax.annotation.Nonnull; +import accord.messages.SimpleReply; -import accord.impl.AbstractAsyncExecutor; +import org.apache.cassandra.io.UnversionedSerializer; -import org.apache.cassandra.service.accord.api.AccordAgent; - -public class ImmediateAsyncExecutor implements AbstractAsyncExecutor +public class SimpleReplySerializer { - public static final ImmediateAsyncExecutor INSTANCE = new ImmediateAsyncExecutor(); - - @Override - public void execute(@Nonnull Runnable command) - { - try - { - command.run(); - } - catch (Throwable t) - { - AccordAgent.handleException(t); - } - } + public static final UnversionedSerializer reply = EncodeAsVInt32.withoutNulls(SimpleReply.class); } diff --git a/src/java/org/apache/cassandra/service/accord/AccordEndpointMapper.java b/src/java/org/apache/cassandra/service/accord/topology/AccordEndpointMapper.java similarity index 97% rename from src/java/org/apache/cassandra/service/accord/AccordEndpointMapper.java rename to src/java/org/apache/cassandra/service/accord/topology/AccordEndpointMapper.java index ea68de5de876..b17a4e410801 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordEndpointMapper.java +++ b/src/java/org/apache/cassandra/service/accord/topology/AccordEndpointMapper.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.topology; import java.util.Map; diff --git a/src/java/org/apache/cassandra/service/accord/AccordFastPath.java b/src/java/org/apache/cassandra/service/accord/topology/AccordFastPath.java similarity index 99% rename from src/java/org/apache/cassandra/service/accord/AccordFastPath.java rename to src/java/org/apache/cassandra/service/accord/topology/AccordFastPath.java index dc3399ad1e1e..da92c96fb7e3 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordFastPath.java +++ b/src/java/org/apache/cassandra/service/accord/topology/AccordFastPath.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.topology; import java.io.IOException; import java.util.Map; diff --git a/src/java/org/apache/cassandra/service/accord/AccordFastPathCoordinator.java b/src/java/org/apache/cassandra/service/accord/topology/AccordFastPathCoordinator.java similarity index 97% rename from src/java/org/apache/cassandra/service/accord/AccordFastPathCoordinator.java rename to src/java/org/apache/cassandra/service/accord/topology/AccordFastPathCoordinator.java index 9b64921a8821..46ecddd4b279 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordFastPathCoordinator.java +++ b/src/java/org/apache/cassandra/service/accord/topology/AccordFastPathCoordinator.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.topology; import java.util.Collections; import java.util.HashMap; @@ -43,8 +43,9 @@ import org.apache.cassandra.gms.IEndpointStateChangeSubscriber; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.service.accord.AccordFastPath.NodeInfo; -import org.apache.cassandra.service.accord.AccordFastPath.Status; +import org.apache.cassandra.service.accord.AccordService; +import org.apache.cassandra.service.accord.topology.AccordFastPath.NodeInfo; +import org.apache.cassandra.service.accord.topology.AccordFastPath.Status; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.Epoch; diff --git a/src/java/org/apache/cassandra/service/accord/AccordStaleReplicas.java b/src/java/org/apache/cassandra/service/accord/topology/AccordStaleReplicas.java similarity index 99% rename from src/java/org/apache/cassandra/service/accord/AccordStaleReplicas.java rename to src/java/org/apache/cassandra/service/accord/topology/AccordStaleReplicas.java index ad0a06a76140..f728662083e2 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordStaleReplicas.java +++ b/src/java/org/apache/cassandra/service/accord/topology/AccordStaleReplicas.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.topology; import java.io.IOException; import java.util.Objects; diff --git a/src/java/org/apache/cassandra/service/accord/AccordSyncPropagator.java b/src/java/org/apache/cassandra/service/accord/topology/AccordSyncPropagator.java similarity index 96% rename from src/java/org/apache/cassandra/service/accord/AccordSyncPropagator.java rename to src/java/org/apache/cassandra/service/accord/topology/AccordSyncPropagator.java index f37e2621e20e..0a396658ff64 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordSyncPropagator.java +++ b/src/java/org/apache/cassandra/service/accord/topology/AccordSyncPropagator.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.topology; import java.io.IOException; import java.util.Collection; @@ -38,6 +38,7 @@ import accord.messages.SimpleReply; import accord.primitives.Ranges; import accord.topology.Topology; +import accord.topology.TopologyManager; import accord.utils.Invariants; import accord.utils.SortedArrays.SortedArrayList; import accord.utils.SortedListSet; @@ -55,6 +56,7 @@ import org.apache.cassandra.net.MessageDelivery; import org.apache.cassandra.net.RequestCallback; import org.apache.cassandra.net.Verb; +import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.service.accord.serializers.KeySerializers; import org.apache.cassandra.service.accord.serializers.TopologySerializers; import org.apache.cassandra.utils.CollectionSerializers; @@ -210,12 +212,12 @@ void setTestListener(TestListener listener) this.listener = listener; } - boolean hasPending() + public synchronized boolean hasPending() { return !pending.isEmpty(); } - synchronized boolean hasPending(long epoch) + public synchronized boolean hasPending(long epoch) { if (pending.isEmpty()) return false; return pending.values().stream().allMatch(n -> { @@ -443,6 +445,15 @@ Notification merge(Notification add) return new Notification(epoch, syncComplete, closed.with(add.closed), retired.with(add.retired), Math.max(add.attempts, this.attempts)); } + public void process(TopologyManager topologyManager) + { + readyToCoordinate.forEach(id -> topologyManager.onReadyToCoordinate(id, epoch)); + if (!closed.isEmpty()) + topologyManager.onEpochClosed(closed, epoch); + if (!retired.isEmpty()) + topologyManager.onEpochRetired(retired, epoch); + } + @Override public String toString() { diff --git a/src/java/org/apache/cassandra/service/accord/AccordTopology.java b/src/java/org/apache/cassandra/service/accord/topology/AccordTopology.java similarity index 97% rename from src/java/org/apache/cassandra/service/accord/AccordTopology.java rename to src/java/org/apache/cassandra/service/accord/topology/AccordTopology.java index f37aef6811d8..c5298c0164ca 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordTopology.java +++ b/src/java/org/apache/cassandra/service/accord/topology/AccordTopology.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.topology; import java.util.ArrayList; import java.util.Collection; @@ -56,8 +56,10 @@ import org.apache.cassandra.schema.ReplicationParams; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.accord.AccordService; +import org.apache.cassandra.service.accord.IAccordService; +import org.apache.cassandra.service.accord.TokenRange; import org.apache.cassandra.service.accord.api.TokenKey; -import org.apache.cassandra.service.accord.fastpath.FastPathStrategy; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.Epoch; @@ -218,22 +220,22 @@ public List> ranges() } } - static TokenRange minRange(TableId table, Token token) + public static TokenRange minRange(TableId table, Token token) { return TokenRange.create(TokenKey.min(table, token.getPartitioner()), new TokenKey(table, token)); } - static TokenRange maxRange(TableId table, Token token) + public static TokenRange maxRange(TableId table, Token token) { return TokenRange.create(new TokenKey(table, token), TokenKey.max(table, token.getPartitioner())); } - static TokenRange fullRange(TableId table, IPartitioner partitioner) + public static TokenRange fullRange(TableId table, IPartitioner partitioner) { return TokenRange.create(TokenKey.min(table, partitioner), TokenKey.max(table, partitioner)); } - static TokenRange range(TableId table, Range range) + public static TokenRange range(TableId table, Range range) { Token minToken = range.left.minValue(); return TokenRange.create(range.left.equals(minToken) ? TokenKey.min(table, minToken.getPartitioner()) : new TokenKey(table, range.left), diff --git a/src/java/org/apache/cassandra/service/accord/AccordTopologyService.java b/src/java/org/apache/cassandra/service/accord/topology/AccordTopologyService.java similarity index 91% rename from src/java/org/apache/cassandra/service/accord/AccordTopologyService.java rename to src/java/org/apache/cassandra/service/accord/topology/AccordTopologyService.java index 85d6f5d2c1c8..d63fab19ae91 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordTopologyService.java +++ b/src/java/org/apache/cassandra/service/accord/topology/AccordTopologyService.java @@ -16,11 +16,10 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.topology; import java.util.HashSet; import java.util.Set; -import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; import javax.annotation.concurrent.GuardedBy; @@ -42,7 +41,6 @@ import org.apache.cassandra.concurrent.ScheduledExecutorPlus; import org.apache.cassandra.concurrent.ScheduledExecutors; -import org.apache.cassandra.concurrent.Shutdownable; import org.apache.cassandra.concurrent.Stage; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.MessageDelivery; @@ -57,7 +55,7 @@ import static org.apache.cassandra.utils.Simulate.With.MONITORS; @Simulate(with=MONITORS) -public class AccordTopologyService implements TopologyService, Shutdownable, TopologyListener +public class AccordTopologyService implements TopologyService, TopologyListener { public static final Logger logger = LoggerFactory.getLogger(AccordTopologyService.class); @@ -100,33 +98,11 @@ public void onStartup(Node node) syncPropagator.onNodesRemoved(removed); } - @Override - public synchronized boolean isTerminated() - { - return state == State.SHUTDOWN; - } - - @Override public synchronized void shutdown() { - if (isTerminated()) - return; state = State.SHUTDOWN; } - @Override - public Object shutdownNow() - { - shutdown(); - return null; - } - - @Override - public boolean awaitTermination(long timeout, TimeUnit units) throws InterruptedException - { - return isTerminated(); - } - @Override public void onReceived(Topology topology) { diff --git a/src/java/org/apache/cassandra/service/accord/EndpointMapping.java b/src/java/org/apache/cassandra/service/accord/topology/EndpointMapping.java similarity index 96% rename from src/java/org/apache/cassandra/service/accord/EndpointMapping.java rename to src/java/org/apache/cassandra/service/accord/topology/EndpointMapping.java index 1bddf0d52ead..855df4b793d6 100644 --- a/src/java/org/apache/cassandra/service/accord/EndpointMapping.java +++ b/src/java/org/apache/cassandra/service/accord/topology/EndpointMapping.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.topology; import java.util.ArrayList; import java.util.List; @@ -45,9 +45,9 @@ import static java.util.concurrent.TimeUnit.MINUTES; -class EndpointMapping implements AccordEndpointMapper +public class EndpointMapping implements AccordEndpointMapper { - static class Updateable implements AccordEndpointMapper + public static class Updateable implements AccordEndpointMapper { private volatile EndpointMapping mapping = EMPTY; @@ -173,7 +173,7 @@ public NodeStatus nodeStatus(Node.Id id) return Double.parseDouble(event.value) == 0.0 ? NodeStatus.UNHEALTHY : NodeStatus.HEALTHY; } - static class Builder + public static class Builder { private final long epoch; private final BiMap mapping = HashBiMap.create(); @@ -207,7 +207,7 @@ public EndpointMapping build() } } - static Builder builder(long epoch) + public static Builder builder(long epoch) { return new Builder(epoch); } diff --git a/src/java/org/apache/cassandra/service/accord/fastpath/FastPathStrategy.java b/src/java/org/apache/cassandra/service/accord/topology/FastPathStrategy.java similarity index 99% rename from src/java/org/apache/cassandra/service/accord/fastpath/FastPathStrategy.java rename to src/java/org/apache/cassandra/service/accord/topology/FastPathStrategy.java index a53e342b5fee..4757785a6472 100644 --- a/src/java/org/apache/cassandra/service/accord/fastpath/FastPathStrategy.java +++ b/src/java/org/apache/cassandra/service/accord/topology/FastPathStrategy.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord.fastpath; +package org.apache.cassandra.service.accord.topology; import java.io.IOException; import java.util.HashMap; diff --git a/src/java/org/apache/cassandra/service/accord/FetchTopologies.java b/src/java/org/apache/cassandra/service/accord/topology/FetchTopologies.java similarity index 98% rename from src/java/org/apache/cassandra/service/accord/FetchTopologies.java rename to src/java/org/apache/cassandra/service/accord/topology/FetchTopologies.java index 80a1098f393a..043b198b47c6 100644 --- a/src/java/org/apache/cassandra/service/accord/FetchTopologies.java +++ b/src/java/org/apache/cassandra/service/accord/topology/FetchTopologies.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.topology; import java.io.IOException; import java.util.ArrayList; @@ -42,6 +42,7 @@ import org.apache.cassandra.net.MessagingUtils; import org.apache.cassandra.net.Verb; import org.apache.cassandra.repair.SharedContext; +import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.service.accord.serializers.TopologySerializers; import org.apache.cassandra.utils.concurrent.Future; diff --git a/src/java/org/apache/cassandra/service/accord/fastpath/InheritKeyspaceFastPathStrategy.java b/src/java/org/apache/cassandra/service/accord/topology/InheritKeyspaceFastPathStrategy.java similarity index 97% rename from src/java/org/apache/cassandra/service/accord/fastpath/InheritKeyspaceFastPathStrategy.java rename to src/java/org/apache/cassandra/service/accord/topology/InheritKeyspaceFastPathStrategy.java index 39fae55d929b..c95879f3efa4 100644 --- a/src/java/org/apache/cassandra/service/accord/fastpath/InheritKeyspaceFastPathStrategy.java +++ b/src/java/org/apache/cassandra/service/accord/topology/InheritKeyspaceFastPathStrategy.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord.fastpath; +package org.apache.cassandra.service.accord.topology; import java.util.Map; import java.util.Set; diff --git a/src/java/org/apache/cassandra/service/accord/fastpath/ParameterizedFastPathStrategy.java b/src/java/org/apache/cassandra/service/accord/topology/ParameterizedFastPathStrategy.java similarity index 99% rename from src/java/org/apache/cassandra/service/accord/fastpath/ParameterizedFastPathStrategy.java rename to src/java/org/apache/cassandra/service/accord/topology/ParameterizedFastPathStrategy.java index 78f6ffd9f709..24e299407d66 100644 --- a/src/java/org/apache/cassandra/service/accord/fastpath/ParameterizedFastPathStrategy.java +++ b/src/java/org/apache/cassandra/service/accord/topology/ParameterizedFastPathStrategy.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord.fastpath; +package org.apache.cassandra.service.accord.topology; import java.io.IOException; import java.util.ArrayList; diff --git a/src/java/org/apache/cassandra/service/accord/fastpath/SimpleFastPathStrategy.java b/src/java/org/apache/cassandra/service/accord/topology/SimpleFastPathStrategy.java similarity index 98% rename from src/java/org/apache/cassandra/service/accord/fastpath/SimpleFastPathStrategy.java rename to src/java/org/apache/cassandra/service/accord/topology/SimpleFastPathStrategy.java index 9540fb84fa42..977d53090223 100644 --- a/src/java/org/apache/cassandra/service/accord/fastpath/SimpleFastPathStrategy.java +++ b/src/java/org/apache/cassandra/service/accord/topology/SimpleFastPathStrategy.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord.fastpath; +package org.apache.cassandra.service.accord.topology; import java.util.Map; import java.util.Set; diff --git a/src/java/org/apache/cassandra/service/accord/WatermarkCollector.java b/src/java/org/apache/cassandra/service/accord/topology/WatermarkCollector.java similarity index 97% rename from src/java/org/apache/cassandra/service/accord/WatermarkCollector.java rename to src/java/org/apache/cassandra/service/accord/topology/WatermarkCollector.java index 88c9acce31a4..3633d10a667c 100644 --- a/src/java/org/apache/cassandra/service/accord/WatermarkCollector.java +++ b/src/java/org/apache/cassandra/service/accord/topology/WatermarkCollector.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.topology; import java.io.IOException; import java.util.ArrayList; @@ -58,6 +58,8 @@ import org.apache.cassandra.net.NoPayload; import org.apache.cassandra.net.Verb; import org.apache.cassandra.repair.SharedContext; +import org.apache.cassandra.service.accord.AccordService; +import org.apache.cassandra.service.accord.TokenRange; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.FBUtilities; @@ -122,7 +124,7 @@ public void doVerb(Message message) }; @VisibleForTesting - static void fetchAndReportWatermarksAsync(TopologyManager topologyManager) + public static void fetchAndReportWatermarksAsync(TopologyManager topologyManager) { SharedContext context = SharedContext.Global.instance; Set peers = new HashSet<>(); diff --git a/src/java/org/apache/cassandra/service/accord/txn/TxnCondition.java b/src/java/org/apache/cassandra/service/accord/txn/TxnCondition.java index 90897d8ea3e5..380566155ff7 100644 --- a/src/java/org/apache/cassandra/service/accord/txn/TxnCondition.java +++ b/src/java/org/apache/cassandra/service/accord/txn/TxnCondition.java @@ -59,7 +59,7 @@ import org.apache.cassandra.utils.ObjectSizes; import static com.google.common.base.Preconditions.checkNotNull; -import static org.apache.cassandra.service.accord.AccordSerializers.clusteringSerializer; +import static org.apache.cassandra.service.accord.serializers.AccordSerializers.clusteringSerializer; import static org.apache.cassandra.service.accord.txn.TxnData.TxnDataNameKind.CAS_READ; import static org.apache.cassandra.service.accord.txn.TxnData.txnDataName; import static org.apache.cassandra.utils.CollectionSerializers.deserializeList; diff --git a/src/java/org/apache/cassandra/service/accord/txn/TxnNamedRead.java b/src/java/org/apache/cassandra/service/accord/txn/TxnNamedRead.java index 5cf61c2d8dbb..29f02207909e 100644 --- a/src/java/org/apache/cassandra/service/accord/txn/TxnNamedRead.java +++ b/src/java/org/apache/cassandra/service/accord/txn/TxnNamedRead.java @@ -301,9 +301,12 @@ public Data call() TxnData result = new TxnData(); if (iterator.hasNext()) { - TxnDataKeyValue value = new TxnDataKeyValue(iterator.next()); - if (value.hasRows() || command.selectsFullPartition()) - result.put(name, value); + try (RowIterator row = iterator.next()) + { + TxnDataKeyValue value = new TxnDataKeyValue(row); + if (value.hasRows() || command.selectsFullPartition()) + result.put(name, value); + } } return result; } diff --git a/src/java/org/apache/cassandra/service/accord/txn/TxnRead.java b/src/java/org/apache/cassandra/service/accord/txn/TxnRead.java index 1f025c713906..bddb2b6fba7a 100644 --- a/src/java/org/apache/cassandra/service/accord/txn/TxnRead.java +++ b/src/java/org/apache/cassandra/service/accord/txn/TxnRead.java @@ -65,8 +65,8 @@ import static accord.primitives.Routables.Slice.Minimal; import static accord.utils.Invariants.require; import static com.google.common.base.Preconditions.checkArgument; -import static org.apache.cassandra.service.accord.AccordSerializers.consistencyLevelSerializer; import static org.apache.cassandra.service.accord.IAccordService.SUPPORTED_READ_CONSISTENCY_LEVELS; +import static org.apache.cassandra.service.accord.serializers.AccordSerializers.consistencyLevelSerializer; import static org.apache.cassandra.service.accord.txn.TxnData.TxnDataNameKind.CAS_READ; import static org.apache.cassandra.service.accord.txn.TxnData.TxnDataNameKind.USER; import static org.apache.cassandra.service.accord.txn.TxnData.txnDataName; diff --git a/src/java/org/apache/cassandra/service/accord/txn/TxnReference.java b/src/java/org/apache/cassandra/service/accord/txn/TxnReference.java index 8b6f7539518f..81548bfeabb5 100644 --- a/src/java/org/apache/cassandra/service/accord/txn/TxnReference.java +++ b/src/java/org/apache/cassandra/service/accord/txn/TxnReference.java @@ -58,7 +58,7 @@ import org.apache.cassandra.utils.ObjectSizes; import static org.apache.cassandra.db.marshal.CollectionType.Kind.SET; -import static org.apache.cassandra.service.accord.AccordSerializers.columnMetadataSerializer; +import static org.apache.cassandra.service.accord.serializers.AccordSerializers.columnMetadataSerializer; public abstract class TxnReference { diff --git a/src/java/org/apache/cassandra/service/accord/txn/TxnReferenceOperation.java b/src/java/org/apache/cassandra/service/accord/txn/TxnReferenceOperation.java index f997008210c8..a07784f2f334 100644 --- a/src/java/org/apache/cassandra/service/accord/txn/TxnReferenceOperation.java +++ b/src/java/org/apache/cassandra/service/accord/txn/TxnReferenceOperation.java @@ -54,12 +54,12 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.accord.AccordSerializers; +import org.apache.cassandra.service.accord.serializers.AccordSerializers; import org.apache.cassandra.service.accord.serializers.TableMetadatas; import org.apache.cassandra.utils.ByteBufferUtil; import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP; -import static org.apache.cassandra.service.accord.AccordSerializers.columnMetadataSerializer; +import static org.apache.cassandra.service.accord.serializers.AccordSerializers.columnMetadataSerializer; public class TxnReferenceOperation { diff --git a/src/java/org/apache/cassandra/service/accord/txn/TxnUpdate.java b/src/java/org/apache/cassandra/service/accord/txn/TxnUpdate.java index 3cf34fbf7ac8..5d57bcf51c18 100644 --- a/src/java/org/apache/cassandra/service/accord/txn/TxnUpdate.java +++ b/src/java/org/apache/cassandra/service/accord/txn/TxnUpdate.java @@ -76,7 +76,7 @@ import static accord.utils.SortedArrays.Search.CEIL; import static com.google.common.base.Preconditions.checkState; import static java.lang.Boolean.FALSE; -import static org.apache.cassandra.service.accord.AccordSerializers.consistencyLevelSerializer; +import static org.apache.cassandra.service.accord.serializers.AccordSerializers.consistencyLevelSerializer; import static org.apache.cassandra.service.accord.txn.TxnUpdate.BlockFragment.NO_BLOCK_FRAGMENTS; import static org.apache.cassandra.service.accord.txn.TxnUpdate.ConditionalBlock.NO_CONDITIONAL_BLOCKS; import static org.apache.cassandra.utils.ArraySerializers.skipArray; diff --git a/src/java/org/apache/cassandra/tcm/ClusterMetadata.java b/src/java/org/apache/cassandra/tcm/ClusterMetadata.java index 4a5f67c1fca2..b99663a65555 100644 --- a/src/java/org/apache/cassandra/tcm/ClusterMetadata.java +++ b/src/java/org/apache/cassandra/tcm/ClusterMetadata.java @@ -62,9 +62,9 @@ import org.apache.cassandra.schema.Keyspaces; import org.apache.cassandra.schema.ReplicationParams; import org.apache.cassandra.schema.TableId; -import org.apache.cassandra.service.accord.AccordFastPath; -import org.apache.cassandra.service.accord.AccordStaleReplicas; -import org.apache.cassandra.service.accord.AccordTopology; +import org.apache.cassandra.service.accord.topology.AccordFastPath; +import org.apache.cassandra.service.accord.topology.AccordStaleReplicas; +import org.apache.cassandra.service.accord.topology.AccordTopology; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; import org.apache.cassandra.service.consensus.migration.TableMigrationState; import org.apache.cassandra.tcm.extensions.ExtensionKey; diff --git a/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java b/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java index e582271a84d9..9c2c4ca0bfe0 100644 --- a/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java +++ b/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java @@ -55,8 +55,8 @@ import org.apache.cassandra.schema.DistributedSchema; import org.apache.cassandra.schema.Keyspaces; import org.apache.cassandra.schema.ReplicationParams; -import org.apache.cassandra.service.accord.AccordFastPath; -import org.apache.cassandra.service.accord.AccordStaleReplicas; +import org.apache.cassandra.service.accord.topology.AccordFastPath; +import org.apache.cassandra.service.accord.topology.AccordStaleReplicas; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; import org.apache.cassandra.tcm.listeners.SchemaListener; import org.apache.cassandra.tcm.log.Entry; diff --git a/src/java/org/apache/cassandra/tcm/StubClusterMetadataService.java b/src/java/org/apache/cassandra/tcm/StubClusterMetadataService.java index 6036554e0d00..a7956cd21aa6 100644 --- a/src/java/org/apache/cassandra/tcm/StubClusterMetadataService.java +++ b/src/java/org/apache/cassandra/tcm/StubClusterMetadataService.java @@ -28,8 +28,8 @@ import org.apache.cassandra.schema.DistributedSchema; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.Keyspaces; -import org.apache.cassandra.service.accord.AccordFastPath; -import org.apache.cassandra.service.accord.AccordStaleReplicas; +import org.apache.cassandra.service.accord.topology.AccordFastPath; +import org.apache.cassandra.service.accord.topology.AccordStaleReplicas; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; import org.apache.cassandra.tcm.Commit.Replicator; import org.apache.cassandra.tcm.log.Entry; diff --git a/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java b/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java index 5c46403ddc6c..4ba870837a34 100644 --- a/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java +++ b/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java @@ -55,8 +55,8 @@ import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.SchemaKeyspace; import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.service.accord.AccordFastPath; -import org.apache.cassandra.service.accord.AccordStaleReplicas; +import org.apache.cassandra.service.accord.topology.AccordFastPath; +import org.apache.cassandra.service.accord.topology.AccordStaleReplicas; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Epoch; diff --git a/src/java/org/apache/cassandra/tcm/sequences/CancelCMSReconfiguration.java b/src/java/org/apache/cassandra/tcm/sequences/CancelCMSReconfiguration.java index 5ce36e09a814..4139110236a7 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/CancelCMSReconfiguration.java +++ b/src/java/org/apache/cassandra/tcm/sequences/CancelCMSReconfiguration.java @@ -33,7 +33,7 @@ import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.ReplicationParams; import org.apache.cassandra.schema.SchemaConstants; -import org.apache.cassandra.service.accord.fastpath.FastPathStrategy; +import org.apache.cassandra.service.accord.topology.FastPathStrategy; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Transformation; import org.apache.cassandra.tcm.membership.Directory; diff --git a/src/java/org/apache/cassandra/tcm/transformations/AccordMarkHardRemoved.java b/src/java/org/apache/cassandra/tcm/transformations/AccordMarkHardRemoved.java index 4c39abc14672..c0b82300c359 100644 --- a/src/java/org/apache/cassandra/tcm/transformations/AccordMarkHardRemoved.java +++ b/src/java/org/apache/cassandra/tcm/transformations/AccordMarkHardRemoved.java @@ -31,7 +31,7 @@ import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.service.accord.AccordTopology; +import org.apache.cassandra.service.accord.topology.AccordTopology; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.MultiStepOperation; import org.apache.cassandra.tcm.Transformation; diff --git a/src/java/org/apache/cassandra/tcm/transformations/AccordMarkRejoining.java b/src/java/org/apache/cassandra/tcm/transformations/AccordMarkRejoining.java index 516a03dee8fd..b0001cbe22ef 100644 --- a/src/java/org/apache/cassandra/tcm/transformations/AccordMarkRejoining.java +++ b/src/java/org/apache/cassandra/tcm/transformations/AccordMarkRejoining.java @@ -30,7 +30,7 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.service.accord.AccordTopology; +import org.apache.cassandra.service.accord.topology.AccordTopology; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Transformation; import org.apache.cassandra.tcm.membership.NodeId; diff --git a/src/java/org/apache/cassandra/tcm/transformations/AccordMarkStale.java b/src/java/org/apache/cassandra/tcm/transformations/AccordMarkStale.java index be4385f4262d..1e3aef581781 100644 --- a/src/java/org/apache/cassandra/tcm/transformations/AccordMarkStale.java +++ b/src/java/org/apache/cassandra/tcm/transformations/AccordMarkStale.java @@ -34,7 +34,7 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.SchemaConstants; -import org.apache.cassandra.service.accord.AccordTopology; +import org.apache.cassandra.service.accord.topology.AccordTopology; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Transformation; import org.apache.cassandra.tcm.membership.NodeId; diff --git a/src/java/org/apache/cassandra/tcm/transformations/ReconfigureAccordFastPath.java b/src/java/org/apache/cassandra/tcm/transformations/ReconfigureAccordFastPath.java index 23e587d2f7a3..6bdabb0aac44 100644 --- a/src/java/org/apache/cassandra/tcm/transformations/ReconfigureAccordFastPath.java +++ b/src/java/org/apache/cassandra/tcm/transformations/ReconfigureAccordFastPath.java @@ -27,8 +27,8 @@ import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.service.accord.AccordFastPath; import org.apache.cassandra.service.accord.serializers.TopologySerializers; +import org.apache.cassandra.service.accord.topology.AccordFastPath; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Transformation; import org.apache.cassandra.tcm.sequences.LockedRanges; diff --git a/src/java/org/apache/cassandra/tcm/transformations/cms/PrepareCMSReconfiguration.java b/src/java/org/apache/cassandra/tcm/transformations/cms/PrepareCMSReconfiguration.java index 18c7a23471f2..cfb7db66dda4 100644 --- a/src/java/org/apache/cassandra/tcm/transformations/cms/PrepareCMSReconfiguration.java +++ b/src/java/org/apache/cassandra/tcm/transformations/cms/PrepareCMSReconfiguration.java @@ -45,7 +45,7 @@ import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.ReplicationParams; import org.apache.cassandra.schema.SchemaConstants; -import org.apache.cassandra.service.accord.fastpath.FastPathStrategy; +import org.apache.cassandra.service.accord.topology.FastPathStrategy; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Transformation; import org.apache.cassandra.tcm.membership.NodeId; diff --git a/src/java/org/apache/cassandra/tools/StandaloneJournalUtil.java b/src/java/org/apache/cassandra/tools/StandaloneJournalUtil.java index 99814b651f5a..9998395d051b 100644 --- a/src/java/org/apache/cassandra/tools/StandaloneJournalUtil.java +++ b/src/java/org/apache/cassandra/tools/StandaloneJournalUtil.java @@ -49,9 +49,10 @@ import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.Tables; -import org.apache.cassandra.service.accord.AccordJournal; import org.apache.cassandra.service.accord.AccordKeyspace; import org.apache.cassandra.service.accord.JournalKey; +import org.apache.cassandra.service.accord.journal.AccordJournal; +import org.apache.cassandra.service.accord.journal.CommandChanges; import org.apache.cassandra.service.accord.serializers.Version; import org.apache.cassandra.tcm.ClusterMetadataService; @@ -274,7 +275,7 @@ public void run() Map cache = new HashMap<>(); journal.start(null); - journal.forEach(key -> processKey(cache, journal, key, txnId, sinceTimestamp, untilTimestamp, skipAllErrors, skipExceptionTypes), false); + journal.forEach(key -> processKey(cache, journal, key, txnId, sinceTimestamp, untilTimestamp, skipAllErrors, skipExceptionTypes), false, 0); } private void processKey(Map redundantBeforeCache, AccordJournal journal, JournalKey key, Timestamp txnId, Timestamp minTimestamp, Timestamp maxTimestamp, boolean skipAllErrors, Set skipExceptionTypes) @@ -301,7 +302,7 @@ private void processKey(Map redundantBeforeCache, Acco output.out.println("Individual entries:"); journal.forEachEntry(key, (in, userVersion) -> { - AccordJournal.Builder builder = new AccordJournal.Builder(key.id, ALL); + CommandChanges builder = new CommandChanges(key.id, ALL); builder.deserializeNext(in, userVersion); output.out.println(String.format("\t%s", builder.toString("\n\t\t"))); counter.getAndIncrement(); @@ -309,7 +310,7 @@ private void processKey(Map redundantBeforeCache, Acco if (construct) { - AccordJournal.Builder builder = new AccordJournal.Builder(key.id, ALL); + CommandChanges builder = new CommandChanges(key.id, ALL); journal.forEachEntry(key, builder::deserializeNext); output.out.println("Reconstructed\n\t\t" + builder.construct(redundantBeforeCache.computeIfAbsent(key.commandStoreId, k -> journal.loadRedundantBefore(key.commandStoreId)))); } diff --git a/src/java/org/apache/cassandra/utils/Crc.java b/src/java/org/apache/cassandra/utils/Crc.java index f1a31584f364..c9c43f6b5682 100644 --- a/src/java/org/apache/cassandra/utils/Crc.java +++ b/src/java/org/apache/cassandra/utils/Crc.java @@ -37,11 +37,15 @@ protected CRC32 initialValue() private static final byte[] initialBytes = new byte[] { (byte) 0xFA, (byte) 0x2D, (byte) 0x55, (byte) 0xCA }; - public static final class InvalidCrc extends IOException + public static class InvalidCrc extends IOException { + public final int read; + public final int computed; public InvalidCrc(int read, int computed) { super(String.format("Read %d, Computed %d", read, computed)); + this.read = read; + this.computed = computed; } } diff --git a/src/java/org/apache/cassandra/utils/ExecutorUtils.java b/src/java/org/apache/cassandra/utils/ExecutorUtils.java index 30a67fbe3cc4..28dffd3b0ce5 100644 --- a/src/java/org/apache/cassandra/utils/ExecutorUtils.java +++ b/src/java/org/apache/cassandra/utils/ExecutorUtils.java @@ -182,7 +182,8 @@ public static void awaitTerminationUntil(long deadline, Collection executors) } else if (executor instanceof Shutdownable) { - if (wait <= 0 || !((Shutdownable)executor).awaitTermination(wait, NANOSECONDS)) + Shutdownable shutdownable = (Shutdownable) executor; + if (wait <= 0 ? !shutdownable.isTerminated() : !shutdownable.awaitTermination(wait, NANOSECONDS)) throw new TimeoutException(executor + " did not terminate on time"); } else if (executor instanceof Thread) diff --git a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java index 738b4853ac2c..b11dafbc7294 100644 --- a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java +++ b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java @@ -19,6 +19,7 @@ import java.io.FileNotFoundException; import java.net.SocketException; +import java.nio.channels.ClosedByInterruptException; import java.nio.file.FileSystemException; import java.util.ArrayList; import java.util.List; @@ -158,7 +159,7 @@ else if (t instanceof UnrecoverableIllegalStateException) throw (UncheckedInterruptedException)t; if (DatabaseDescriptor.getDiskFailurePolicy() == Config.DiskFailurePolicy.die) - if (t instanceof FSError || t instanceof CorruptSSTableException) + if ((t instanceof FSError || t instanceof CorruptSSTableException) && !(t.getCause() instanceof ClosedByInterruptException)) isUnstable = true; // Check for file handle exhaustion diff --git a/test/conf/cassandra.yaml b/test/conf/cassandra.yaml index 07200f2d16d0..236d89e95e57 100644 --- a/test/conf/cassandra.yaml +++ b/test/conf/cassandra.yaml @@ -125,3 +125,8 @@ memtable: class_name: TrieMemtable # Note: keep the memtable configuration at the end of the file, so that the default mapping can be changed without # duplicating the whole section above. +accord.shutdown_grace_period: 120s +accord.command_store_shard_count: 2 +accord.queue_shard_count: 1 + + diff --git a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java index fd87c86b6c7b..7e51a4f67498 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java @@ -909,7 +909,7 @@ public void schemaChangeIgnoringStoppedInstances(String query) schemaChange(query, true); } - private void schemaChange(String query, boolean ignoreStoppedInstances) + public void schemaChange(String query, boolean ignoreStoppedInstances) { I instance = ignoreStoppedInstances ? getFirstRunningInstance() : get(1); schemaChange(query, ignoreStoppedInstances, instance); diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java index 6f18007c6705..c5498ccaf0b7 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java @@ -1004,6 +1004,16 @@ public Future shutdown(boolean runOnExitThreads, boolean shutdownMessaging throw e; } }; + error = parallelRun(error, executor, + // If an index build completes as shutting down, setIndexBuild may trigger + // a CFS.forceBlockingFlush + () -> SecondaryIndexManager.shutdownAndWait(1L, MINUTES) + ); + + error = parallelRun(error, executor, () -> { + if (AccordService.isSetupOrStarting()) + AccordService.unsafeInstance().shutdownAndWait(1L, MINUTES); + }); error = parallelRun(error, executor, shutdownBatchlogAndHints, @@ -1014,7 +1024,6 @@ public Future shutdown(boolean runOnExitThreads, boolean shutdownMessaging () -> StreamReceiveTask.shutdownAndWait(1L, MINUTES), () -> StreamTransferTask.shutdownAndWait(1L, MINUTES), () -> StreamManager.instance.stop(), - () -> SecondaryIndexManager.shutdownAndWait(1L, MINUTES), () -> IndexSummaryManager.instance.shutdownAndWait(1L, MINUTES), () -> ColumnFamilyStore.shutdownExecutorsAndWait(1L, MINUTES), () -> BufferPools.shutdownLocalCleaner(1L, MINUTES), @@ -1047,10 +1056,11 @@ public Future shutdown(boolean runOnExitThreads, boolean shutdownMessaging () -> SharedExecutorPool.SHARED.shutdownAndWait(1L, MINUTES) ); - error = parallelRun(error, executor, () -> { - if (AccordService.isSetupOrStarting()) - AccordService.unsafeInstance().shutdownAndWait(1L, MINUTES); - }); + // ScheduledExecutors shuts down after MessagingService, as MessagingService may issue tasks to it and + // before CommitLog, as any thread calling executeInternal could wait indefinitely + // on commitlog allocator (e.g. SSTableReader tidier on non-periodic calling + // SystemKeyspace.clearSSTableReadMeter) + error = parallelRun(error, executor, () -> ScheduledExecutors.shutdownNowAndWait(1L, MINUTES)); // CommitLog must shut down after Stage, or threads from the latter may attempt to use the former. // (ex. A Mutation stage thread may attempt to add a mutation to the CommitLog.) diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java index 1548e1e51240..506407cb13e8 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java @@ -74,7 +74,7 @@ protected IInvokableInstance failedAndResumeBootstrapAndJoinNode(Cluster cluster IInvokableInstance newInstance = cluster.bootstrap(config); newInstance.startup(cluster); spinUntilTrue(() -> cluster.stream().anyMatch(instance -> instance.callOnInstance(() -> StreamListener.listener.hasFailedStream))); - try { newInstance.shutdown(false).get(5L, TimeUnit.MINUTES); } + try { newInstance.shutdown(false).get(3L, TimeUnit.MINUTES); } catch (InterruptedException | ExecutionException | TimeoutException e) { throw new RuntimeException(e); } cluster.get(1, 2).forEach(instance -> instance.runOnInstance(() -> StreamListener.listener.failStream = false)); newInstance.startup(cluster); diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordCommandStoreTryExecuteListeningTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordCommandStoreTryExecuteListeningTest.java index 8639e87db11f..1f853fd5f57e 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordCommandStoreTryExecuteListeningTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordCommandStoreTryExecuteListeningTest.java @@ -18,6 +18,14 @@ package org.apache.cassandra.distributed.test.accord; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.stream.Stream; + import org.assertj.core.api.Assertions; import org.junit.Test; @@ -31,6 +39,7 @@ import accord.primitives.Deps; import accord.primitives.FullRoute; import accord.primitives.KeyDeps; +import accord.primitives.Range; import accord.primitives.RangeDeps; import accord.primitives.Ranges; import accord.primitives.Routable; @@ -97,12 +106,12 @@ public void testTryExecuteListening() throws Throwable PartitionKey key = pk(1, "ks", "tbl"); AccordCommandStore commandStore = (AccordCommandStore) node.commandStores().unsafeForKey(key.toUnseekable()); - Command txn1a = executed(node, SaveStatus.Applied); - Command txn1b = executed(node, SaveStatus.PreApplied); - Command txn2a = executed(node, SaveStatus.PreApplied, txn1a.txnId()); - Command txn2b = executed(node, SaveStatus.PreApplied, txn1b.txnId()); - Command txn3 = executed(node, SaveStatus.PreApplied, txn1a.txnId(), txn1b.txnId(), txn2b.txnId()); - Command txn4 = executed(node, SaveStatus.PreApplied, txn1a.txnId(), txn1b.txnId(), txn3.txnId()); + Command txn1a = executed(node, SaveStatus.Applied, 1); + Command txn1b = executed(node, SaveStatus.PreApplied, 2); + Command txn2a = executed(node, SaveStatus.PreApplied, 1, txn1a.txnId()); + Command txn2b = executed(node, SaveStatus.PreApplied, 2, txn1b.txnId()); + Command txn3 = executed(node, SaveStatus.PreApplied,1, txn1a.txnId(), 2, txn1b.txnId(), txn2b.txnId()); + Command txn4 = executed(node, SaveStatus.PreApplied, 1, txn1a.txnId(), txn3.txnId(), 2, txn1b.txnId(), txn3.txnId()); Command[] commands = new Command[] { txn1a, txn1b, txn2a, txn2b, txn3, txn4 }; AccordService.getBlocking(commandStore.chain((Empty)() -> "Test", safeStore -> { @@ -113,7 +122,7 @@ public void testTryExecuteListening() throws Throwable commandStore.unsafeGetListeners().register(txn3.txnId(), SaveStatus.Applied, txn4.txnId()); })); - AccordService.getBlocking(commandStore.operatorTryToExecuteListeningTxns()); + AccordService.getBlocking(commandStore.tryToExecuteListeningTxns(true)); for (Command command : commands) { @@ -124,26 +133,70 @@ public void testTryExecuteListening() throws Throwable } } - private static Command executed(Node node, SaveStatus saveStatus, TxnId ... dependencies) + private static Command executed(Node node, SaveStatus saveStatus, Object ... inputs) { - PartitionKey key = pk(1, "ks", "tbl"); + int depCount; + Map> depsByInputKey = new TreeMap<>(); + TxnId[] txnIds; + { + PartitionKey k = null; + for (Object input : inputs) + { + if (input instanceof Integer) + { + k = keyN((Integer) input, node); + depsByInputKey.put(k, new ArrayList<>()); + } + else depsByInputKey.get(k).add((TxnId)input); + } + txnIds = depsByInputKey.values().stream().flatMap(Collection::stream).distinct().sorted().toArray(TxnId[]::new); + depCount = depsByInputKey.values().stream().mapToInt(Collection::size).sum(); + } + PartitionKey[] keys = depsByInputKey.keySet().toArray(PartitionKey[]::new); + Range[] ranges = Stream.of(keys).map(PartitionKey::asRange).toArray(Range[]::new); + + PartitionKey key = keys[0]; + AccordCommandStore commandStore = (AccordCommandStore) node.commandStores().unsafeForKey(key.toUnseekable()); + Txn txn = node.agent().emptySystemTxn(Txn.Kind.ExclusiveSyncPoint, Routable.Domain.Range); TxnId txnId = node.nextTxnId(txn); FullRoute route; - try { route = node.computeRoute(txnId, Ranges.of(key.asRange())); } + try { route = node.computeRoute(txnId, Ranges.of(ranges)); } catch (TopologyException e) { throw new RuntimeException(e); } - AccordCommandStore commandStore = (AccordCommandStore) node.commandStores().unsafeForKey(key.toUnseekable()); - int[] rangesToTxnIds = new int[dependencies.length + 1]; - rangesToTxnIds[0] = rangesToTxnIds.length; - for (int i = 1; i < rangesToTxnIds.length ; ++i) - rangesToTxnIds[i] = i - 1; - Deps deps = new Deps(KeyDeps.NONE, RangeDeps.SerializerSupport.create(new accord.primitives.Range[] { key.asRange() }, dependencies, rangesToTxnIds, null)); + int[] rangesToTxnIds = new int[depCount + ranges.length]; + { + int offset = ranges.length; + for (int i = 0 ; i < ranges.length ; ++i) + { + for (TxnId dep : depsByInputKey.get(keys[i])) + rangesToTxnIds[offset++] = Arrays.binarySearch(txnIds, dep); + rangesToTxnIds[i] = offset; + } + } + Deps deps = new Deps(KeyDeps.NONE, RangeDeps.SerializerSupport.create(ranges, txnIds, rangesToTxnIds, null)); Command.WaitingOn waitingOn; { - LargeBitSet waitingOnBits = new LargeBitSet(dependencies.length); - waitingOnBits.setRange(0, dependencies.length); - waitingOn = new Command.WaitingOn(RoutingKeys.EMPTY, deps.rangeDeps, new ImmutableBitSet(waitingOnBits), new ImmutableBitSet(dependencies.length)); + LargeBitSet waitingOnBits = new LargeBitSet(txnIds.length); + waitingOnBits.setRange(0, txnIds.length); + waitingOn = new Command.WaitingOn(RoutingKeys.EMPTY, deps.rangeDeps, new ImmutableBitSet(waitingOnBits), new ImmutableBitSet(txnIds.length)); } return Command.Executed.executed(txnId, saveStatus, Status.Durability.NotDurable, StoreParticipants.execute(commandStore.unsafeGetRangesForEpoch(), route, txnId, txnId.epoch()), Ballot.ZERO, txnId, txn.intersecting(route, true), deps.intersecting(route), Ballot.ZERO, waitingOn, null, ResultSerializers.APPLIED); } + private static PartitionKey keyN(int n, Node node) + { + PartitionKey first = pk(1, "ks", "tbl"); + if (n == 1) + return first; + + AccordCommandStore commandStore = (AccordCommandStore) node.commandStores().unsafeForKey(first.toUnseekable()); + + int i = 2; + while (true) + { + PartitionKey next = pk(i, "ks", "tbl"); + if (commandStore.unsafeGetRangesForEpoch().all().contains(next) && --n == 0) + return next; + } + } + } diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordJournalIntegrationTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordJournalIntegrationTest.java index c1bd8d6b4805..f8e2f82d2d09 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordJournalIntegrationTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordJournalIntegrationTest.java @@ -47,7 +47,7 @@ public void saveLoadSanityCheck() throws Throwable { try (WithProperties wp = new WithProperties().set(CassandraRelevantProperties.DTEST_ACCORD_JOURNAL_SANITY_CHECK_ENABLED, "true"); Cluster cluster = init(Cluster.build(1) - .withConfig(config -> config.set("accord.catchup_on_start", "false")) + .withConfig(config -> config.set("accord.catchup_on_start", "DISABLED")) .withoutVNodes() .start())) { @@ -96,7 +96,7 @@ public void memtableStateReloadingTest() throws Throwable { try (Cluster cluster = Cluster.build(1) .withoutVNodes() - .withConfig(config -> config.set("accord.catchup_on_start", "false")) + .withConfig(config -> config.set("accord.catchup_on_start", "DISABLED")) .start()) { cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};"); @@ -128,7 +128,7 @@ public void restartWithEpochChanges() throws IOException .withoutVNodes() .withConfig(c -> { c.with(GOSSIP).with(NETWORK); - c.set("accord.catchup_on_start", "false"); + c.set("accord.catchup_on_start", "DISABLED"); }) .start()) { diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordLoadTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordLoadTest.java index e8786075357d..3fe95b507047 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordLoadTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordLoadTest.java @@ -54,6 +54,7 @@ import org.apache.cassandra.distributed.shared.DistributedTestBase; import org.apache.cassandra.net.Verb; import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.service.accord.AccordExecutor; import org.apache.cassandra.service.accord.AccordKeyspace; import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.utils.EstimatedHistogram; @@ -61,7 +62,6 @@ import static java.lang.System.currentTimeMillis; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.NANOSECONDS; -import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.cassandra.db.ColumnFamilyStore.FlushReason.UNIT_TESTS; public class AccordLoadTest extends AccordTestBase @@ -78,6 +78,7 @@ public static void setUp() throws IOException .set("accord.shard_durability_target_splits", "8") .set("accord.shard_durability_max_splits", "16") .set("accord.shard_durability_cycle", "1m") + .set("accord.catchup_on_start_fail_latency", "2m") // .set("accord.ephemeral_read_enabled", "true") ), 3); } @@ -105,21 +106,22 @@ public boolean matches(int i, int i1, IMessage iMessage) ICoordinator coordinator = cluster.coordinator(1); final int repairInterval = Integer.MAX_VALUE; - final int compactionInterval = 20_000; + final int compactionInterval = Integer.MAX_VALUE; // final int flushInterval = 50_000; - final int journalFlushInterval = 2_000; - final int cfkFlushInterval = 10_000; - final int dataFlushInterval = 10_000; - final int compactionPeriodSeconds = 0; - int restartInterval = 30_000; + final int journalFlushInterval = Integer.MAX_VALUE; + final int cfkFlushInterval = Integer.MAX_VALUE; + final int dataFlushInterval = Integer.MAX_VALUE; +// final int compactionPeriodSeconds = 0; +// int restartInterval = 30_000; + int restartInterval = Integer.MAX_VALUE; final int restartDecay = 2; // final int restartInterval = Integer.MAX_VALUE; final int batchSizeLimit = 200; final long batchTime = TimeUnit.SECONDS.toNanos(10); - final int concurrency = 100; - final int ratePerSecond = 1000; + final int concurrency = 200; + final int ratePerSecond = 500; // final int keyCount = 10_000; - final int keyCount = 10; + final int keyCount = 10_000; final float readChance = 0.33f; long nextRepairAt = repairInterval; long nextCompactionAt = compactionInterval; @@ -130,12 +132,13 @@ public boolean matches(int i, int i1, IMessage iMessage) final ExecutorService restartExecutor = Executors.newSingleThreadExecutor(); final BitSet initialised = new BitSet(); + java.util.concurrent.Future restarting = null; cluster.get(1).nodetoolResult("cms", "reconfigure", "3").asserts().success(); - cluster.forEach(i -> i.runOnInstance(() -> { - if (compactionPeriodSeconds > 0) - ((AccordService) AccordService.instance()).journal().compactor().updateCompactionPeriod(1, SECONDS); +// cluster.forEach(i -> i.runOnInstance(() -> { +// if (compactionPeriodSeconds > 0) +// ((AccordService) AccordService.instance()).journal().compactor().updateCompactionPeriod(1, SECONDS); // ((AccordSpec.JournalSpec)((AccordService) AccordService.instance()).journal().configuration()).segmentSize = 128 << 10; - })); +// })); Random random = new Random(); final Semaphore inFlight = new Semaphore(concurrency); @@ -153,29 +156,44 @@ public boolean matches(int i, int i1, IMessage iMessage) try { long commandStart = System.nanoTime(); - int k = random.nextInt(keyCount); + int k1 = random.nextInt(keyCount); + int k2 = random.nextInt(keyCount); if (random.nextFloat() < readChance) { coordinator.executeWithResult((success, fail) -> { inFlight.release(); if (fail == null) histogram.add(NANOSECONDS.toMicros(System.nanoTime() - commandStart)); - }, "SELECT * FROM " + qualifiedAccordTableName + " WHERE k = ?;", ConsistencyLevel.SERIAL, k); + }, "BEGIN TRANSACTION\n" + + "SELECT * FROM " + qualifiedAccordTableName + " WHERE k IN ?;\n" + + "COMMIT TRANSACTION;", ConsistencyLevel.SERIAL, + List.of(k1, k2) +// List.of(k1) + ); } - else if (initialised.get(k)) + else if (initialised.get(k1) && initialised.get(k2)) { coordinator.executeWithResult((success, fail) -> { inFlight.release(); if (fail == null) histogram.add(NANOSECONDS.toMicros(System.nanoTime() - commandStart)); - }, "UPDATE " + qualifiedAccordTableName + " SET v += 1 WHERE k = ? IF EXISTS;", ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM, k); + }, "BEGIN TRANSACTION\n" + + "UPDATE " + qualifiedAccordTableName + " SET v += 1 WHERE k = ?;\n" + + "UPDATE " + qualifiedAccordTableName + " SET v += 1 WHERE k = ?;\n" + + "COMMIT TRANSACTION;", ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM, k1, k2); } else { - initialised.set(k); + initialised.set(k1); + initialised.set(k2); coordinator.executeWithResult((success, fail) -> { inFlight.release(); if (fail == null) histogram.add(NANOSECONDS.toMicros(System.nanoTime() - commandStart)); // else exceptions.add(fail); - }, "UPDATE " + qualifiedAccordTableName + " SET v = 0 WHERE k = ? IF NOT EXISTS;", ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM, k); + }, "UPDATE " + qualifiedAccordTableName + " SET v = 0 WHERE k = ? IF NOT EXISTS;", ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM, k1); + coordinator.executeWithResult((success, fail) -> { + inFlight.release(); + if (fail == null) histogram.add(NANOSECONDS.toMicros(System.nanoTime() - commandStart)); + // else exceptions.add(fail); + }, "UPDATE " + qualifiedAccordTableName + " SET v = 0 WHERE k = ? IF NOT EXISTS;", ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM, k2); } } catch (RejectedExecutionException e) @@ -223,10 +241,13 @@ else if (initialised.get(k)) cluster.forEach(i -> { try { - i.runOnInstance(() -> { - if (AccordService.started()) - ((AccordService) AccordService.instance()).journal().closeCurrentSegmentForTestingIfNonEmpty(); - }); + if (!i.isShutdown()) + { + i.runOnInstance(() -> { + if (AccordService.started()) + ((AccordService) AccordService.instance()).journal().closeCurrentSegmentForTestingIfNonEmpty(); + }); + } } catch (Throwable t) { @@ -274,29 +295,40 @@ else if (initialised.get(k)) if ((nextRestartAt -= batchSize) <= 0) { - nextRestartAt += restartInterval; - restartInterval = Math.max(restartInterval, restartInterval * restartDecay); - int nodeIdx = 1 + random.nextInt(cluster.size()); - restartExecutor.submit(() -> { - System.out.printf("restarting node %d...\n", nodeIdx); - try - { - cluster.get(nodeIdx).shutdown().get(); - cluster.get(nodeIdx).startup(); - while (!cluster.get(nodeIdx).callOnInstance(() -> AccordService.started())) - Thread.sleep(1000); - return null; - } - catch (InterruptedException | ExecutionException e) - { - throw new RuntimeException(e); - } - }); + if (restarting == null || restarting.isDone()) + { + if (restarting != null) + restarting.get(); + + nextRestartAt += restartInterval; + int nodeIdx = 1 + random.nextInt(cluster.size()); + restarting = restartExecutor.submit(() -> { + System.out.printf("restarting node %d...\n", nodeIdx); + try + { + cluster.get(nodeIdx).shutdown().get(); + cluster.get(nodeIdx).startup(); + return null; + } + catch (InterruptedException | ExecutionException e) + { + throw new RuntimeException(e); + } + }); + if (nodeIdx == coordinator.instance().config().num()) + coordinator = cluster.coordinator((nodeIdx % cluster.size()) + 1); + } } final Date date = new Date(); System.out.printf("%tT rate: %.2f/s (%d total)\n", date, (((float)batchSizeLimit * 1000) / NANOSECONDS.toMillis(System.nanoTime() - batchStart)), batchSize); System.out.printf("%tT percentiles: %d %d %d %d\n", date, histogram.percentile(.25)/1000, histogram.percentile(.5)/1000, histogram.percentile(.75)/1000, histogram.percentile(1)/1000); + cluster.forEach(() -> { + String waiting = ""; + for (AccordExecutor executor : AccordService.instance().executors()) + waiting += executor.unsafeWaitingToRunCount() + " "; + System.out.println(waiting); + }); class VerbCount { diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordMetricsTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordMetricsTest.java index 897c0d0e6d06..c51a32363a8c 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordMetricsTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordMetricsTest.java @@ -257,7 +257,7 @@ public void testTimeoutMetrics() assertReplicaMetrics(0, "rw", 0, 0, 0); assertReplicaMetrics(1, "rw", 0, 0, 0); - assertCoordinatorMetrics(0, "ro", 0, 0, 0, 0, 1); + assertCoordinatorMetrics(0, "ro", 0, 0, 0, 0, 0); assertCoordinatorMetrics(1, "ro", 0, 0, 0, 0, 0); assertReplicaMetrics(0, "ro", 0, 0, 0); assertReplicaMetrics(1, "ro", 0, 0, 0); diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordProgressLogTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordProgressLogTest.java index 3f4163ddb22a..a32d895794cc 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordProgressLogTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordProgressLogTest.java @@ -93,6 +93,8 @@ public void testFetchTimeWindow() throws Throwable cluster.schemaChange("CREATE TABLE ks.tbl (k int, c int, v int, primary key (k, c)) WITH " + TransactionalMode.full.asCqlParam()); String query = "BEGIN TRANSACTION\n" + " INSERT INTO ks.tbl (k, c) VALUES (0, 0);\n" + + " INSERT INTO ks.tbl (k, c) VALUES (1, 1);\n" + + " INSERT INTO ks.tbl (k, c) VALUES (2, 2);\n" + "COMMIT TRANSACTION"; IMessageFilters.Filter dropApply = cluster.filters().outbound().from(1).verbs(Verb.ACCORD_APPLY_REQ.id).drop(); @@ -121,7 +123,7 @@ public void testFetchTimeWindow() throws Throwable logger.info("Awaited at {}", fetchStartedAt.get()); long timeDeltaMillis = TimeUnit.NANOSECONDS.toMillis(fetchStartedAt.get() - coordinationStartedAt); Assert.assertTrue("Fetch started in " + timeDeltaMillis + "ms", timeDeltaMillis >= 100); - Assert.assertTrue("Fetch started in " + timeDeltaMillis + "ms", timeDeltaMillis <= 2000); + Assert.assertTrue("Fetch started in " + timeDeltaMillis + "ms", timeDeltaMillis <= 4000); } } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordSimpleFastPathTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordSimpleFastPathTest.java index 909096370dec..633756735bad 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordSimpleFastPathTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordSimpleFastPathTest.java @@ -39,8 +39,8 @@ import org.apache.cassandra.distributed.test.TestBaseImpl; import org.apache.cassandra.gms.FailureDetector; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.service.accord.AccordFastPath; import org.apache.cassandra.service.accord.AccordService; +import org.apache.cassandra.service.accord.topology.AccordFastPath; import org.apache.cassandra.service.consensus.TransactionalMode; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/journal/AccordJournalReplayTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/journal/AccordJournalReplayTest.java index 4c9f77f628a7..2c9535078918 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/journal/AccordJournalReplayTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/journal/AccordJournalReplayTest.java @@ -97,7 +97,8 @@ public void replayCommandWithOnlyDurableSyncPointDependency() throws Throwable .set("accord.shard_durability_target_splits", "1") .set("accord.retry_syncpoint", "1s*attempts") .set("accord.retry_durability", "1s*attempts") - .set("accord.catchup_on_start", "false") + .set("accord.journal.replay_save_point", "NO") + .set("accord.catchup_on_start", "DISABLED") .with(NETWORK, GOSSIP)) .start()) { diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/journal/JournalAccessRouteIndexOnStartupRaceTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/journal/JournalAccessRouteIndexOnStartupRaceTest.java index f55495f0eb94..04f53770c555 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/journal/JournalAccessRouteIndexOnStartupRaceTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/journal/JournalAccessRouteIndexOnStartupRaceTest.java @@ -43,9 +43,9 @@ import org.apache.cassandra.index.accord.RouteJournalIndex; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.RetryStrategy; -import org.apache.cassandra.service.accord.AccordJournalTable; import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.service.accord.TokenRange; +import org.apache.cassandra.service.accord.journal.RangeSearchManager; import org.apache.cassandra.service.consensus.TransactionalMode; import org.apache.cassandra.utils.Shared; import org.apache.cassandra.utils.concurrent.CountDownLatch; @@ -66,7 +66,7 @@ public class JournalAccessRouteIndexOnStartupRaceTest extends TestBaseImpl public void test() throws IOException { try (Cluster cluster = Cluster.build(1) - .withConfig(config -> config.set("accord.catchup_on_start", "false")) + .withConfig(config -> config.set("accord.catchup_on_start", "DISABLED")) .withInstanceInitializer(BBHelper::install).start()) { IInvokableInstance node = cluster.get(1); @@ -136,7 +136,7 @@ public static void install(ClassLoader cl, int id) .make() .load(cl, ClassLoadingStrategy.Default.INJECTION); - new ByteBuddy().rebase(AccordJournalTable.class) + new ByteBuddy().rebase(RangeSearchManager.class) .method(named("maybeWait")) .intercept(MethodDelegation.to(BBHelper.class)) .make() diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/journal/StatefulJournalRestartTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/journal/StatefulJournalRestartTest.java index d8f1c10b14b9..bd80842f02ad 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/journal/StatefulJournalRestartTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/journal/StatefulJournalRestartTest.java @@ -73,7 +73,7 @@ * at accord.utils.Invariants.createIllegalState(Invariants.java:77) * at accord.utils.Invariants.illegalState(Invariants.java:82) * at accord.utils.Invariants.require(Invariants.java:272) - * at org.apache.cassandra.service.accord.AccordJournal.replay(AccordJournal.java:452) + * at org.apache.cassandra.service.accord.journal.AccordJournal.replay(AccordJournal.java:452) * at org.apache.cassandra.service.accord.AccordService.replayJournal(AccordService.java:246) * at org.apache.cassandra.service.accord.AccordService.startup(AccordService.java:235) * at org.apache.cassandra.distributed.impl.Instance.partialStartup(Instance.java:878) diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataTestHelper.java b/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataTestHelper.java index 1598c5ca8974..bb01c624b657 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataTestHelper.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataTestHelper.java @@ -59,8 +59,8 @@ import org.apache.cassandra.schema.SchemaTestUtil; import org.apache.cassandra.schema.SchemaTransformation; import org.apache.cassandra.service.ClientState; -import org.apache.cassandra.service.accord.AccordFastPath; -import org.apache.cassandra.service.accord.AccordStaleReplicas; +import org.apache.cassandra.service.accord.topology.AccordFastPath; +import org.apache.cassandra.service.accord.topology.AccordStaleReplicas; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; import org.apache.cassandra.tcm.AtomicLongBackedProcessor; import org.apache.cassandra.tcm.ClusterMetadata; diff --git a/test/distributed/org/apache/cassandra/fuzz/topology/AccordBootstrapTest.java b/test/distributed/org/apache/cassandra/fuzz/topology/AccordBootstrapTest.java index 4ddcd03d45a7..2d3474f36de7 100644 --- a/test/distributed/org/apache/cassandra/fuzz/topology/AccordBootstrapTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/topology/AccordBootstrapTest.java @@ -66,6 +66,10 @@ public void bootstrapFuzzTest() throws Throwable .set("write_request_timeout", "2s") .set("request_timeout", "5s") .set("concurrent_accord_operations", 2) + .set("accord.shard_durability_target_splits", "1") + .set("accord.shard_durability_max_splits", "4") + .set("accord.catchup_on_start_fail_latency", "60s") + .set("accord.shutdown_grace_period", "60s") .set("progress_barrier_min_consistency_level", "QUORUM") .set("progress_barrier_default_consistency_level", "QUORUM") .set("metadata_snapshot_frequency", 5)) diff --git a/test/distributed/org/apache/cassandra/fuzz/topology/AccordBounceTest.java b/test/distributed/org/apache/cassandra/fuzz/topology/AccordBounceTest.java index 69d12bf9ee31..b0ce42908ec6 100644 --- a/test/distributed/org/apache/cassandra/fuzz/topology/AccordBounceTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/topology/AccordBounceTest.java @@ -118,7 +118,7 @@ public void commandStoresBounceTest() throws Throwable try (Cluster cluster = init(builder().withNodes(1).start())) { withRandom(rng -> { - Generator schemaGen = SchemaGenerators.trivialSchema(KEYSPACE, new Supplier() { + Generator schemaGen = SchemaGenerators.trivialSchema(KEYSPACE, new Supplier<>() { int i = 0; @Override public String get() @@ -144,16 +144,24 @@ public String get() .build(schema, hb, cluster))); } - Runnable writeAndValidate = () -> { + Runnable write = () -> { for (HistoryBuilder hb : historyBuilders) for (int pk = 0; pk < 10; pk++) for (int i = 0; i < 10; i++) hb.insert(pk); + }; + + Runnable validate = () -> { for (HistoryBuilder hb : historyBuilders) for (int pk = 0; pk < 10; pk++) hb.selectPartition(pk); }; + Runnable writeAndValidate = () -> { + write.run(); + validate.run(); + }; + // Command Stores should not be lost on bounce Map> before = cluster.get(1).callOnInstance(() -> { Map> m = new HashMap<>(); @@ -170,6 +178,7 @@ public String get() writeAndValidate.run(); ClusterUtils.stopUnchecked(cluster.get(1)); cluster.get(1).startup(); + validate.run(); SchemaSpec schema = schemaGen.generate(rng); cluster.schemaChange(schema.compile()); diff --git a/test/distributed/org/apache/cassandra/fuzz/topology/AccordHardCatchupTest.java b/test/distributed/org/apache/cassandra/fuzz/topology/AccordHardCatchupTest.java new file mode 100644 index 000000000000..746d3b5bb7f8 --- /dev/null +++ b/test/distributed/org/apache/cassandra/fuzz/topology/AccordHardCatchupTest.java @@ -0,0 +1,139 @@ +/* + * 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.fuzz.topology; + +import java.util.HashSet; + +import org.junit.Test; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.TokenSupplier; +import org.apache.cassandra.distributed.shared.ClusterUtils; +import org.apache.cassandra.distributed.shared.NetworkTopology; +import org.apache.cassandra.distributed.test.log.FuzzTestBase; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.dsl.HistoryBuilderHelper; +import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; +import org.apache.cassandra.harry.execution.InJvmDTestVisitExecutor; +import org.apache.cassandra.harry.execution.QueryBuildingVisitExecutor; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.Generators.TrackingGenerator; +import org.apache.cassandra.harry.gen.SchemaGenerators; +import org.apache.cassandra.service.consensus.TransactionalMode; + +import static org.apache.cassandra.distributed.shared.ClusterUtils.waitForCMSToQuiesce; +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; + +public class AccordHardCatchupTest extends FuzzTestBase +{ + private static final int WRITES = 10; + private static final int POPULATION = 1000; + + @Test + public void hardCatchupFuzzTest() throws Throwable + { + CassandraRelevantProperties.SYSTEM_TRACES_DEFAULT_RF.setInt(3); + Cluster.Builder builder = builder(); + try (Cluster cluster = builder.withNodes(3) + .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(100)) + .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(100, "dc0", "rack0")) + .withConfig((config) -> config.with(Feature.NETWORK, Feature.GOSSIP)) + .start()) + { + IInvokableInstance cmsInstance = cluster.get(1); + waitForCMSToQuiesce(cluster, cmsInstance); + + HashSet downInstances = new HashSet<>(); + withRandom(rng -> { + Generator schemaGen = SchemaGenerators.trivialSchema(KEYSPACE, () -> "bootstrap_fuzz", POPULATION, + SchemaSpec.optionsBuilder() + .addWriteTimestamps(false) + .withTransactionalMode(TransactionalMode.full) + ); + + SchemaSpec schema = schemaGen.generate(rng); + TrackingGenerator pkGen = Generators.tracking(Generators.int32(0, Math.min(schema.valueGenerators.pkPopulation(), POPULATION))); + Generator ckGen = Generators.int32(0, Math.min(schema.valueGenerators.ckPopulation(), POPULATION)); + HistoryBuilder history = new ReplayingHistoryBuilder(schema.valueGenerators, + hb -> InJvmDTestVisitExecutor.builder() + .consistencyLevel(ConsistencyLevel.QUORUM) + .wrapQueries(QueryBuildingVisitExecutor.WrapQueries.TRANSACTION) + .pageSizeSelector(p -> InJvmDTestVisitExecutor.PageSizeSelector.NO_PAGING) + .nodeSelector(lts -> { + while (true) + { + int pick = rng.nextInt(1, cluster.size() + 1); + if (!downInstances.contains(pick)) + return pick; + + } + }) + .build(schema, hb, cluster)); + + Runnable writeAndValidate = () -> { + for (int i = 0; i < WRITES; i++) + HistoryBuilderHelper.insertRandomData(schema, pkGen, ckGen, rng, history); + + for (int pk : pkGen.generated()) + history.selectPartition(pk); + }; + + history.customThrowing(() -> { + cluster.schemaChange(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3};", KEYSPACE)); + cluster.schemaChange(schema.compile()); + waitForCMSToQuiesce(cluster, cmsInstance); + }, "Setup"); + Thread.sleep(1000); + writeAndValidate.run(); + + history.customThrowing(() -> { + downInstances.add(2); + ClusterUtils.stopUnchecked(cluster.get(2)); + cluster.get(1).logs().watchFor("/127.0.0.2:.* is now DOWN"); + }, "Shut down node 2"); + + writeAndValidate.run(); + + history.customThrowing(() -> { + cluster.get(2).config().set("accord.catchup_on_start", "HARD"); + cluster.get(2).startup(); + cluster.get(2).logs().watchFor(".*Catchup.*"); + cluster.get(1).logs().watchFor("/127.0.0.2:.* is now UP"); + downInstances.remove(2); + }, "Start down node 2"); + + writeAndValidate.run(); + + history.customThrowing(() -> { + downInstances.add(1); + ClusterUtils.stopUnchecked(cluster.get(1)); + cluster.get(2).logs().watchFor("/127.0.0.1:.* is now DOWN"); + }, "Shut down node 1"); + + writeAndValidate.run(); + }); + } + } +} diff --git a/test/distributed/org/apache/cassandra/fuzz/topology/AccordRebootstrapTest.java b/test/distributed/org/apache/cassandra/fuzz/topology/AccordRebootstrapTest.java new file mode 100644 index 000000000000..d5a8f79c63ae --- /dev/null +++ b/test/distributed/org/apache/cassandra/fuzz/topology/AccordRebootstrapTest.java @@ -0,0 +1,169 @@ +/* + * 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.fuzz.topology; + +import java.nio.file.Path; +import java.util.HashSet; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.IntPredicate; + +import org.junit.Test; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.TokenSupplier; +import org.apache.cassandra.distributed.shared.ClusterUtils; +import org.apache.cassandra.distributed.shared.NetworkTopology; +import org.apache.cassandra.distributed.test.log.FuzzTestBase; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.dsl.HistoryBuilderHelper; +import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; +import org.apache.cassandra.harry.execution.InJvmDTestVisitExecutor; +import org.apache.cassandra.harry.execution.QueryBuildingVisitExecutor; +import org.apache.cassandra.harry.gen.EntropySource; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.Generators.TrackingGenerator; +import org.apache.cassandra.harry.gen.SchemaGenerators; +import org.apache.cassandra.harry.util.ThrowingRunnable; +import org.apache.cassandra.io.util.PathUtils; +import org.apache.cassandra.service.consensus.TransactionalMode; + +import static org.apache.cassandra.distributed.shared.ClusterUtils.waitForCMSToQuiesce; +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; + +public class AccordRebootstrapTest extends FuzzTestBase +{ + private static final int WRITES = 10; + private static final int POPULATION = 1000; + + @Test + public void rebootstrapFuzzTest() throws Throwable + { + CassandraRelevantProperties.SYSTEM_TRACES_DEFAULT_RF.setInt(3); + Cluster.Builder builder = builder(); + try (Cluster cluster = builder.withNodes(3) + .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(100)) + .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(100, "dc0", "rack0")) + .withConfig((config) -> config.with(Feature.NETWORK, Feature.GOSSIP)) + .start()) + { + IInvokableInstance cmsInstance = cluster.get(1); + waitForCMSToQuiesce(cluster, cmsInstance); + + HashSet downInstances = new HashSet<>(); + AtomicInteger nextId = new AtomicInteger(); + withRandom(rng -> { + Generator schemaGen = SchemaGenerators.trivialSchema(KEYSPACE, () -> "bootstrap_fuzz" + (nextId.incrementAndGet()), POPULATION, + SchemaSpec.optionsBuilder() + .addWriteTimestamps(false) + .withTransactionalMode(TransactionalMode.full) + ); + + History history1 = createNewSchemaWithWriteAndValidate(schemaGen, rng, cluster, downInstances::contains); + history1.writeAndValidate(); + + history1.run(() -> { + downInstances.add(2); + ClusterUtils.stopUnchecked(cluster.get(2)); + cluster.get(1).logs().watchFor("/127.0.0.2:.* is now DOWN"); + }, "Shut down node 2"); + + history1.writeAndValidate(); + History history2 = createNewSchemaWithWriteAndValidate(schemaGen, rng, cluster, downInstances::contains); + history2.writeAndValidate(); + History history3 = createNewSchemaWithWriteAndValidate(schemaGen, rng, cluster, downInstances::contains); + history3.writeAndValidate(); + + history1.run(() -> { + cluster.get(2).config().set("accord.journal.stop_marker_failure_policy", "REBOOTSTRAP"); + Path journalDir = Path.of(cluster.get(2).config().get("accord.journal_directory").toString()); + Path stopMarker = journalDir.resolve("stopped"); + PathUtils.delete(stopMarker); + cluster.get(2).startup(); + cluster.get(2).logs().watchFor(".*Rebootstrapping.*"); + cluster.get(1).logs().watchFor("/127.0.0.2:.* is now UP"); + downInstances.remove(2); + }, "Start down node 2"); + + history1.writeAndValidate(); + history2.writeAndValidate(); + history3.writeAndValidate(); + }); + } + } + + interface History + { + void writeAndValidate(); + void run(ThrowingRunnable run, String tag); + } + + private History createNewSchemaWithWriteAndValidate(Generator schemaGen, EntropySource rng, Cluster cluster, IntPredicate downInstances) throws InterruptedException + { + IInvokableInstance cmsInstance = cluster.get(1); + SchemaSpec schema = schemaGen.generate(rng); + TrackingGenerator pkGen = Generators.tracking(Generators.int32(0, Math.min(schema.valueGenerators.pkPopulation(), POPULATION))); + Generator ckGen = Generators.int32(0, Math.min(schema.valueGenerators.ckPopulation(), POPULATION)); + HistoryBuilder history = new ReplayingHistoryBuilder(schema.valueGenerators, + hb -> InJvmDTestVisitExecutor.builder() + .consistencyLevel(ConsistencyLevel.QUORUM) + .wrapQueries(QueryBuildingVisitExecutor.WrapQueries.TRANSACTION) + .pageSizeSelector(p -> InJvmDTestVisitExecutor.PageSizeSelector.NO_PAGING) + .nodeSelector(lts -> { + while (true) + { + int pick = rng.nextInt(1, cluster.size() + 1); + if (!downInstances.test(pick)) + return pick; + } + }) + .build(schema, hb, cluster)); + + history.customThrowing(() -> { + cluster.schemaChange(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3};", KEYSPACE), true); + cluster.schemaChange(schema.compile(), true); + waitForCMSToQuiesce(cluster, cmsInstance); + }, "Setup"); + Thread.sleep(1000); + + return new History() + { + @Override + public void writeAndValidate() + { + for (int i = 0; i < WRITES; i++) + HistoryBuilderHelper.insertRandomData(schema, pkGen, ckGen, rng, history); + + for (int pk : pkGen.generated()) + history.selectPartition(pk); + } + + @Override + public void run(ThrowingRunnable run, String tag) + { + history.customThrowing(run, tag); + } + }; + } +} diff --git a/test/distributed/org/apache/cassandra/fuzz/topology/JournalGCTest.java b/test/distributed/org/apache/cassandra/fuzz/topology/JournalGCTest.java index d70192f13677..670286fdfe6f 100644 --- a/test/distributed/org/apache/cassandra/fuzz/topology/JournalGCTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/topology/JournalGCTest.java @@ -114,7 +114,7 @@ public void journalGCTest() throws Throwable ((AccordService) AccordService.instance()).journal().forEach((v) -> { if (v.type == JournalKey.Type.COMMAND_DIFF && (a.get() == null || v.id.compareTo(a.get()) > 0)) a.set(v.id); - }, false); + }, false, 0); return a.get() == null ? "" : a.get().toString(); }); @@ -124,7 +124,7 @@ public void journalGCTest() throws Throwable ((AccordService) AccordService.instance()).journal().forEach((v) -> { if (v.type == JournalKey.Type.COMMAND_DIFF && v.id.compareTo(maxId) <= 0) a.incrementAndGet(); - }, false); + }, false, 0); return a.get(); }, maximumId); diff --git a/test/distributed/org/apache/cassandra/fuzz/topology/TopologyMixupTestBase.java b/test/distributed/org/apache/cassandra/fuzz/topology/TopologyMixupTestBase.java index 9fa798bb6397..481392149d30 100644 --- a/test/distributed/org/apache/cassandra/fuzz/topology/TopologyMixupTestBase.java +++ b/test/distributed/org/apache/cassandra/fuzz/topology/TopologyMixupTestBase.java @@ -380,7 +380,7 @@ public void applyUnit(State state) protected void preCheck(Property.StatefulBuilder statefulBuilder) { - + statefulBuilder.withSeed(1L); } protected void destroyState(State state, @Nullable Throwable cause) throws Throwable @@ -524,6 +524,7 @@ public State(RandomSource rs, BiFunction schemaSpecGen .withTokenSupplier(topologyHistory) .withConfig(c -> { c.with(Feature.values()) + .set("accord.shutdown_grace_period", "30s") .set("write_request_timeout", "10s") .set("read_request_timeout", "10s") .set("range_request_timeout", "20s") @@ -725,10 +726,14 @@ public String toString() } private String epochHistory = null; - + private boolean closing; @Override public void close() throws Exception { + if (closing) + return; + + closing = true; var cmsNodesUp = Sets.intersection(asSet(cmsGroup), asSet(topologyHistory.up())); int cmsNode = Iterables.getFirst(cmsNodesUp, null); try diff --git a/test/distributed/org/apache/cassandra/service/accord/AccordJournalCompactionTest.java b/test/distributed/org/apache/cassandra/service/accord/AccordJournalCompactionTest.java index 3290a0e7a1ac..a5d5e2c4313c 100644 --- a/test/distributed/org/apache/cassandra/service/accord/AccordJournalCompactionTest.java +++ b/test/distributed/org/apache/cassandra/service/accord/AccordJournalCompactionTest.java @@ -50,11 +50,12 @@ import org.apache.cassandra.journal.TestParams; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.accord.journal.AccordJournal; import org.apache.cassandra.utils.AccordGenerators; import static accord.api.Journal.FieldUpdates; import static accord.local.CommandStores.RangesForEpoch; -import static org.apache.cassandra.service.accord.AccordJournalValueSerializers.DurableBeforeAccumulator; +import static org.apache.cassandra.service.accord.journal.MergeSerializers.DurableBeforeMerger; public class AccordJournalCompactionTest @@ -87,7 +88,7 @@ public void segmentMergeTest() throws InterruptedException cfs.disableAutoCompaction(); RedundantBefore redundantBeforeAccumulator = RedundantBefore.EMPTY; - DurableBeforeAccumulator durableBeforeAccumulator = new DurableBeforeAccumulator(); + DurableBeforeMerger durableBeforeMerger = new DurableBeforeMerger(); NavigableMap safeToReadAtAccumulator = ImmutableSortedMap.of(Timestamp.NONE, Ranges.EMPTY); NavigableMap bootstrapBeganAtAccumulator = ImmutableSortedMap.of(TxnId.NONE, Ranges.EMPTY); RangesForEpoch rangesForEpochAccumulator = null; @@ -113,7 +114,6 @@ public boolean enableCompaction() try { journal.start(null); - journal.unsafeSetStarted(); Timestamp timestamp = Timestamp.NONE; RandomSource rs = new DefaultRandom(1); @@ -134,7 +134,7 @@ public boolean enableCompaction() journal.saveStoreState(1, updates, null); redundantBeforeAccumulator = updates.newRedundantBefore; - durableBeforeAccumulator.update(addDurableBefore); + durableBeforeMerger.update(addDurableBefore); if (updates.newBootstrapBeganAt != null) bootstrapBeganAtAccumulator = updates.newBootstrapBeganAt; if (updates.newSafeToRead != null) @@ -149,14 +149,14 @@ public boolean enableCompaction() } // Assert.assertEquals(redundantBeforeAccumulator.get(), journal.loadRedundantBefore(1)); - Assert.assertEquals(durableBeforeAccumulator.get(), journal.durableBeforePersister().load()); + Assert.assertEquals(durableBeforeMerger.get(), journal.durableBeforePersister().load()); Assert.assertEquals(bootstrapBeganAtAccumulator, journal.loadBootstrapBeganAt(1)); Assert.assertEquals(safeToReadAtAccumulator, journal.loadSafeToRead(1)); Assert.assertEquals(rangesForEpochAccumulator, journal.loadRangesForEpoch(1)); } finally { - journal.shutdown(); + journal.stop(); } } diff --git a/test/distributed/org/apache/cassandra/service/accord/BurnTestKeySerializers.java b/test/distributed/org/apache/cassandra/service/accord/BurnTestKeySerializers.java index 50936d902cba..18a83782a0c6 100644 --- a/test/distributed/org/apache/cassandra/service/accord/BurnTestKeySerializers.java +++ b/test/distributed/org/apache/cassandra/service/accord/BurnTestKeySerializers.java @@ -107,7 +107,7 @@ public int serializedSizeOfPrefix(Object prefix) } @Override - public int serializedSizeWithoutPrefix(PrefixedIntHashKey key) + public int serializedSizeWithoutPrefixOrLength(PrefixedIntHashKey key) { return 8; } @@ -181,7 +181,7 @@ public int serializedSizeOfPrefix(Object prefix) } @Override - public int serializedSizeWithoutPrefix(PrefixedIntHashKey.Hash key) + public int serializedSizeWithoutPrefixOrLength(PrefixedIntHashKey.Hash key) { return 4; } diff --git a/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java b/test/distributed/org/apache/cassandra/service/accord/journal/AccordJournalBurnTest.java similarity index 91% rename from test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java rename to test/distributed/org/apache/cassandra/service/accord/journal/AccordJournalBurnTest.java index aba9a8c69eef..b7b8df50cb47 100644 --- a/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java +++ b/test/distributed/org/apache/cassandra/service/accord/journal/AccordJournalBurnTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.journal; import java.io.IOException; import java.nio.file.Files; @@ -56,6 +56,7 @@ import accord.utils.RandomSource; import org.apache.cassandra.ServerTestUtils; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Directories; import org.apache.cassandra.db.Keyspace; @@ -79,6 +80,10 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.Tables; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.accord.AccordKeyspace; +import org.apache.cassandra.service.accord.BurnTestKeySerializers; +import org.apache.cassandra.service.accord.IAccordService; +import org.apache.cassandra.service.accord.JournalKey; import org.apache.cassandra.service.accord.serializers.CommandSerializers; import org.apache.cassandra.service.accord.serializers.DepsSerializers; import org.apache.cassandra.service.accord.serializers.KeySerializers; @@ -89,6 +94,7 @@ import org.apache.cassandra.utils.CloseableIterator; import static accord.impl.PrefixedIntHashKey.ranges; +import static org.apache.cassandra.config.AccordSpec.RangeIndexMode.journal_sai; import static org.apache.cassandra.utils.TimeUUID.Generator.nextTimeUUID; public class AccordJournalBurnTest extends BurnTestBase @@ -199,11 +205,12 @@ public int segmentSize() } }, directory, cfs) { + Node node; @Override public void start(Node node) { + this.node = node; super.start(node); - unsafeSetStarted(); } @Override @@ -234,16 +241,20 @@ public void saveTopology(TopologyUpdate topologyUpdate, Runnable onFlush) @Override protected SegmentCompactor compactor(ColumnFamilyStore cfs, Version userVersion) { - return new NemesisAccordSegmentCompactor<>(userVersion, cfs, randomSource.fork()) + if (rangeSearch == null) + { + Invariants.require(DatabaseDescriptor.getAccord().range_index_mode != journal_sai); + return new NemesisSegmentCompactor<>(userVersion, cfs, randomSource.fork()); + } + + return new NemesisSegmentCompactor<>(userVersion, cfs, randomSource.fork()) { @Nullable @Override public Collection> compact(Collection> staticSegments) { - if (journalTable == null) - throw new IllegalStateException("Unsafe access to AccordJournal during ; journalTable was touched before it was published"); Collection> result = super.compact(staticSegments); - journalTable.safeNotify(index -> index.remove(staticSegments)); + rangeSearch.safeNotify(index -> index.remove(staticSegments)); return result; } }; @@ -262,10 +273,10 @@ private CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, public void purge(CommandStores commandStores, EpochSupplier minEpoch) { ++counter; - this.journal.closeCurrentSegmentForTestingIfNonEmpty(); - this.journal.runCompactorForTesting(); + this.segments.closeCurrentSegmentForTestingIfNonEmpty(); + this.segments.runCompactorForTesting(); - Set orig = cfs.getLiveSSTables(); + Set orig = table.getLiveSSTables(); List all = new ArrayList<>(orig); if (all.size() <= 1) return; @@ -290,18 +301,18 @@ public void purge(CommandStores commandStores, EpochSupplier minEpoch) TreeMap before = read(commandStores); Collection newSStables; - try (LifecycleTransaction txn = cfs.getTracker().tryModify(selected, OperationType.COMPACTION); - CompactionController controller = new CompactionController(cfs, selected, 0); + try (LifecycleTransaction txn = table.getTracker().tryModify(selected, OperationType.COMPACTION); + CompactionController controller = new CompactionController(table, selected, 0); CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, scanners, controller, 0, nextTimeUUID(), ActiveCompactionsTracker.NOOP, null, - () -> getCompactionInfo(node, cfs.getTableId()), + () -> getCompactionInfo(node, table.getTableId()), () -> Version.V1)) { - try (CompactionAwareWriter writer = getCompactionAwareWriter(cfs, cfs.getDirectories(), txn, selected)) + try (CompactionAwareWriter writer = getCompactionAwareWriter(table, table.getDirectories(), txn, selected)) { while (ci.hasNext()) writer.append(ci.next()); @@ -328,13 +339,13 @@ public void purge(CommandStores commandStores, EpochSupplier minEpoch) Invariants.require(null != before.get(e.getKey())); Invariants.require(false); } - Invariants.require(!orig.equals(cfs.getLiveSSTables())); + Invariants.require(!orig.equals(table.getLiveSSTables())); } private TreeMap read(CommandStores commandStores) { TreeMap result = new TreeMap<>(JournalKey.SUPPORT::compare); - try (CloseableIterator> iter = journalTable.keyIterator(null, null, false)) + try (CloseableIterator> iter = keyIterator(null, null, false, 0)) { JournalKey prev = null; while (iter.hasNext()) @@ -356,11 +367,11 @@ private TreeMap read(CommandStores commandStores) } @Override - public boolean replay(CommandStores commandStores) + public boolean replay(CommandStores commandStores, Object param) { // Make sure to replay _only_ static segments this.closeCurrentSegmentForTestingIfNonEmpty(); - return super.replay(commandStores); + return super.replay(commandStores, param); } @Override diff --git a/test/distributed/org/apache/cassandra/service/accord/NemesisAccordSegmentCompactor.java b/test/distributed/org/apache/cassandra/service/accord/journal/NemesisSegmentCompactor.java similarity index 93% rename from test/distributed/org/apache/cassandra/service/accord/NemesisAccordSegmentCompactor.java rename to test/distributed/org/apache/cassandra/service/accord/journal/NemesisSegmentCompactor.java index a8c11397372d..e58556222ebf 100644 --- a/test/distributed/org/apache/cassandra/service/accord/NemesisAccordSegmentCompactor.java +++ b/test/distributed/org/apache/cassandra/service/accord/journal/NemesisSegmentCompactor.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.journal; import java.util.Arrays; import java.util.HashSet; @@ -36,13 +36,13 @@ * * For testing purposes only. */ -public class NemesisAccordSegmentCompactor extends AbstractAccordSegmentCompactor +public class NemesisSegmentCompactor extends AbstractSegmentCompactor { private final RandomSource randomSource; private final SSTableTxnWriter[] writers; private final Set written = new HashSet<>(); - public NemesisAccordSegmentCompactor(Version userVersion, ColumnFamilyStore cfs, RandomSource randomSource) + public NemesisSegmentCompactor(Version userVersion, ColumnFamilyStore cfs, RandomSource randomSource) { super(userVersion, cfs); this.randomSource = randomSource; diff --git a/test/simulator/test/org/apache/cassandra/simulator/test/AccordJournalSimulationTest.java b/test/simulator/test/org/apache/cassandra/simulator/test/AccordJournalSimulationTest.java index 377c63031a13..9ac555f40e70 100644 --- a/test/simulator/test/org/apache/cassandra/simulator/test/AccordJournalSimulationTest.java +++ b/test/simulator/test/org/apache/cassandra/simulator/test/AccordJournalSimulationTest.java @@ -90,7 +90,7 @@ public void simpleRWTest() public static void check() { - State.journal.start(); + State.journal.start(0L); try { final int count = 100; @@ -118,7 +118,7 @@ public static void check() } finally { - State.journal.shutdown(); + State.journal.stop(); if (!State.thrown.isEmpty()) { diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java index cdeb89d1cdc7..799f59993cf0 100644 --- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java +++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java @@ -78,6 +78,7 @@ public class DatabaseDescriptorRefTest "org.apache.cassandra.auth.INetworkAuthorizer", "org.apache.cassandra.auth.IRoleManager", "org.apache.cassandra.config.AccordSpec", + "org.apache.cassandra.config.AccordSpec$CatchupMode", "org.apache.cassandra.config.AccordSpec$FetchRetrySpec", "org.apache.cassandra.config.AccordSpec$JournalSpec", "org.apache.cassandra.config.AccordSpec$MinEpochRetrySpec", @@ -85,7 +86,11 @@ public class DatabaseDescriptorRefTest "org.apache.cassandra.config.AccordSpec$QueueShardModel", "org.apache.cassandra.config.AccordSpec$QueueSubmissionModel", "org.apache.cassandra.config.AccordSpec$RangeIndexMode", + "org.apache.cassandra.config.AccordSpec$RebootstrapMode", "org.apache.cassandra.config.AccordSpec$TransactionalRangeMigration", + "org.apache.cassandra.config.AccordSpec$JournalSpec$ReplayMode", + "org.apache.cassandra.config.AccordSpec$JournalSpec$ReplaySavePoint", + "org.apache.cassandra.config.AccordSpec$JournalSpec$StopMarkerFailurePolicy", "org.apache.cassandra.config.CassandraRelevantProperties", "org.apache.cassandra.config.CassandraRelevantProperties$PropertyConverter", "org.apache.cassandra.config.Config", @@ -297,8 +302,8 @@ public class DatabaseDescriptorRefTest "org.apache.cassandra.io.util.SpinningDiskOptimizationStrategy", "org.apache.cassandra.journal.Params", "org.apache.cassandra.journal.Params$FailurePolicy", + "org.apache.cassandra.journal.Params$RecoverableCrcFailurePolicy", "org.apache.cassandra.journal.Params$FlushMode", - "org.apache.cassandra.journal.Params$ReplayMode", "org.apache.cassandra.locator.Endpoint", "org.apache.cassandra.locator.IEndpointSnitch", "org.apache.cassandra.locator.InetAddressAndPort", diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java index 00881fedcf7f..c8b636c94e28 100644 --- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java +++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java @@ -32,7 +32,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Consumer; +import java.util.function.BiConsumer; import java.util.function.Supplier; import com.google.common.annotations.VisibleForTesting; @@ -936,7 +936,7 @@ public boolean shouldSwitch(FlushReason reason) } @Override - public > T ensureFlushListener(Object key, Supplier onDurablyFlushed) + public > T ensureFlushListener(Object key, Supplier onDurablyFlushed) { return null; } diff --git a/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java b/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java index 4d81d1baabfa..0b32311e867e 100644 --- a/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java +++ b/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java @@ -56,7 +56,7 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.Tables; import org.apache.cassandra.schema.Types; -import org.apache.cassandra.service.accord.fastpath.FastPathStrategy; +import org.apache.cassandra.service.accord.topology.FastPathStrategy; import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.service.snapshot.SnapshotOptions; diff --git a/test/unit/org/apache/cassandra/index/accord/RouteIndexTest.java b/test/unit/org/apache/cassandra/index/accord/RouteIndexTest.java index bf610ad76586..4a2c3af99995 100644 --- a/test/unit/org/apache/cassandra/index/accord/RouteIndexTest.java +++ b/test/unit/org/apache/cassandra/index/accord/RouteIndexTest.java @@ -27,6 +27,8 @@ import java.util.Objects; import java.util.Set; import java.util.TreeSet; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -85,7 +87,6 @@ import org.apache.cassandra.dht.Murmur3Partitioner.LongToken; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.TableId; -import org.apache.cassandra.service.accord.AccordJournal; import org.apache.cassandra.service.accord.AccordKeyspace; import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.service.accord.AccordTestUtils; @@ -93,6 +94,7 @@ import org.apache.cassandra.service.accord.IAccordService.AccordCompactionInfo; import org.apache.cassandra.service.accord.TokenRange; import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.service.accord.journal.AccordJournal; import org.apache.cassandra.service.consensus.TransactionalMode; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.membership.NodeId; @@ -106,6 +108,7 @@ import static accord.local.RedundantStatus.SomeStatus.NONE; import static accord.utils.Property.commands; import static accord.utils.Property.stateful; +import static org.apache.cassandra.config.AccordSpec.JournalSpec.StopMarkerFailurePolicy.UNSAFE_STARTUP; import static org.apache.cassandra.config.AccordSpec.RangeIndexMode.journal_sai; import static org.apache.cassandra.config.DatabaseDescriptor.getPartitioner; import static org.apache.cassandra.schema.SchemaConstants.ACCORD_KEYSPACE_NAME; @@ -131,7 +134,8 @@ public static void setUpClass() DatabaseDescriptor.daemonInitialization(); DatabaseDescriptor.setAccordTransactionsEnabled(true); // disable journal compaction so the test can control when it happens - DatabaseDescriptor.getAccord().enable_journal_compaction = false; + DatabaseDescriptor.getAccord().journal.enable_compaction = false; + DatabaseDescriptor.getAccord().journal.stopMarkerFailurePolicy = UNSAFE_STARTUP; DatabaseDescriptor.getAccord().range_index_mode = journal_sai; DatabaseDescriptor.setIncrementalBackupsEnabled(false); DatabaseDescriptor.setAutoSnapshot(false); @@ -384,7 +388,7 @@ public String toString() return "RangeSearch{" + "storeId=" + storeId + ", range=" + range + - ", minDecidedId=" + decidedRX + + ", minDecided=" + decidedRX + '}'; } } @@ -610,15 +614,14 @@ public String toString() } @Override - public void close() + public void close() throws InterruptedException, TimeoutException { - accordService.shutdown(); + accordService.shutdownAndWait(2L, TimeUnit.MINUTES); } - private void restartAccord() + private void restartAccord() throws InterruptedException, TimeoutException { - accordService.journal().closeCurrentSegmentForTestingIfNonEmpty(); - accordService.shutdown(); + accordService.shutdownAndWait(2L, TimeUnit.MINUTES); accordService = startAccord(); } } diff --git a/test/unit/org/apache/cassandra/io/Serializers.java b/test/unit/org/apache/cassandra/io/Serializers.java index 8a1f58a2dc34..1fbef3b58af9 100644 --- a/test/unit/org/apache/cassandra/io/Serializers.java +++ b/test/unit/org/apache/cassandra/io/Serializers.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.function.BiConsumer; +import java.util.function.BiPredicate; import org.assertj.core.api.Assertions; @@ -37,12 +38,26 @@ public class Serializers public static void testSerde(DataOutputBuffer output, AsymmetricUnversionedSerializer serializer, T input) throws IOException { - testSerde(output, serializer, input, (actual, expected) -> Assertions.assertThat(actual) - .describedAs("The deserialized output does not match the serialized input; difference %s", new LazyToString(() -> ReflectionUtils.recursiveEquals(actual, input).toString())) - .isEqualTo(expected)); + testSerde(output, serializer, input, (expected, actual) -> { + Assertions.assertThat(expected) + .describedAs("The deserialized output does not match the serialized input; difference %s", new LazyToString(() -> ReflectionUtils.recursiveEquals(actual, input).toString())) + .isEqualTo(actual); + }); } public static void testSerde(DataOutputBuffer output, AsymmetricUnversionedSerializer serializer, T input, BiConsumer testEqual) throws IOException + { + testSerde(output, serializer, input, serializer, testEqual); + } + + public static void testSerde(DataOutputBuffer output, AsymmetricUnversionedSerializer serializer, T1 input, AsymmetricUnversionedSerializer deserializer, BiPredicate equals) throws IOException + { + testSerde(output, serializer, input, deserializer, (expected, actual) -> { + Assertions.assertThat(equals.test(expected, actual)).isTrue(); + }); + } + + public static void testSerde(DataOutputBuffer output, AsymmetricUnversionedSerializer serializer, T1 input, AsymmetricUnversionedSerializer deserializer, BiConsumer testEqual) throws IOException { output.clear(); long expectedSize = serializer.serializedSize(input); @@ -50,12 +65,12 @@ public static void testSerde(DataOutputBuffer output, AsymmetricUnversionedS Assertions.assertThat(output.getLength()).describedAs("The serialized size and bytes written do not match").isEqualTo(expectedSize); ByteBuffer buffer = output.unsafeGetBufferAndFlip(); DataInputBuffer in = new DataInputBuffer(buffer, false); - T read = serializer.deserialize(in); - testEqual.accept(read, input); + T2 read = deserializer.deserialize(in); + testEqual.accept(input, read); Assertions.assertThat(buffer.remaining()).describedAs("deserialize did not consume all the serialized input").isEqualTo(0); buffer.flip(); buffer.mark(); - serializer.skip(in); + deserializer.skip(in); Assertions.assertThat(buffer.remaining()).describedAs("skip did not consume all the serialized input").isEqualTo(0); boolean testByteBufferMethods; try @@ -70,8 +85,8 @@ public static void testSerde(DataOutputBuffer output, AsymmetricUnversionedS if (testByteBufferMethods) { ByteBuffer serialized2 = serializer.serialize(input); - T read2 = serializer.deserialize(serialized2); - Assertions.assertThat(read2).describedAs("The deserialized output does not match the serialized input; difference %s", new LazyToString(() -> ReflectionUtils.recursiveEquals(read2, input).toString())).isEqualTo(input); + T2 read2 = deserializer.deserialize(serialized2); + testEqual.accept(input, read2); } } diff --git a/test/unit/org/apache/cassandra/journal/JournalTest.java b/test/unit/org/apache/cassandra/journal/JournalTest.java index 62342777a74a..22eea40abd7a 100644 --- a/test/unit/org/apache/cassandra/journal/JournalTest.java +++ b/test/unit/org/apache/cassandra/journal/JournalTest.java @@ -52,7 +52,7 @@ public void testSimpleReadWrite() throws IOException Journal journal = new Journal<>("TestJournal", directory, TestParams.INSTANCE, TimeUUIDKeySupport.INSTANCE, LongSerializer.INSTANCE, SegmentCompactor.noop(), new OpOrder()); - journal.start(); + journal.start(0L); TimeUUID id1 = nextTimeUUID(); TimeUUID id2 = nextTimeUUID(); @@ -69,17 +69,17 @@ public void testSimpleReadWrite() throws IOException assertEquals(3L, (long) journal.readLast(id3)); assertEquals(4L, (long) journal.readLast(id4)); - journal.shutdown(); + journal.stop(); journal = new Journal<>("TestJournal", directory, TestParams.INSTANCE, TimeUUIDKeySupport.INSTANCE, LongSerializer.INSTANCE, SegmentCompactor.noop(), new OpOrder()); - journal.start(); + journal.start(0L); assertEquals(1L, (long) journal.readLast(id1)); assertEquals(2L, (long) journal.readLast(id2)); assertEquals(3L, (long) journal.readLast(id3)); assertEquals(4L, (long) journal.readLast(id4)); - journal.shutdown(); + journal.stop(); } static class LongSerializer implements ValueSerializer diff --git a/test/unit/org/apache/cassandra/journal/SegmentTest.java b/test/unit/org/apache/cassandra/journal/SegmentTest.java index 240922a01e30..353905211f8e 100644 --- a/test/unit/org/apache/cassandra/journal/SegmentTest.java +++ b/test/unit/org/apache/cassandra/journal/SegmentTest.java @@ -32,6 +32,7 @@ import org.apache.cassandra.utils.concurrent.OpOrder; import static org.apache.cassandra.harry.checker.TestHelper.withRandom; +import static org.apache.cassandra.journal.Params.RecoverableCrcFailurePolicy.FAIL; import static org.apache.cassandra.utils.TimeUUID.Generator.nextTimeUUID; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -113,7 +114,7 @@ public void testReadClosedSegmentByID() throws IOException activeSegment.close(null); - StaticSegment staticSegment = StaticSegment.open(descriptor, TimeUUIDKeySupport.INSTANCE); + StaticSegment staticSegment = StaticSegment.open(descriptor, TimeUUIDKeySupport.INSTANCE, FAIL); // read all 4 entries by id and compare with originals EntrySerializer.EntryHolder holder = new EntrySerializer.EntryHolder<>(); @@ -167,7 +168,7 @@ public void testReadClosedSegmentSequentially() throws IOException tidier.await.issue(); activeSegment.close(null); - StaticSegment.SequentialReader reader = StaticSegment.sequentialReader(descriptor, TimeUUIDKeySupport.INSTANCE, activeSegment.metadata.fsyncLimit()); + StaticSegment.SequentialReader reader = StaticSegment.sequentialReader(descriptor, TimeUUIDKeySupport.INSTANCE, activeSegment.metadata.fsyncLimit(), FAIL); // read all 4 entries sequentially and compare with originals assertTrue(reader.advance()); diff --git a/test/unit/org/apache/cassandra/journal/TestParams.java b/test/unit/org/apache/cassandra/journal/TestParams.java index d464f00fade2..4002f251e4e9 100644 --- a/test/unit/org/apache/cassandra/journal/TestParams.java +++ b/test/unit/org/apache/cassandra/journal/TestParams.java @@ -44,15 +44,15 @@ public FailurePolicy failurePolicy() } @Override - public FlushMode flushMode() + public RecoverableCrcFailurePolicy crcFailureOnRebuildPolicy() { - return FlushMode.GROUP; + return RecoverableCrcFailurePolicy.FAIL; } @Override - public ReplayMode replayMode() + public FlushMode flushMode() { - return null; + return FlushMode.GROUP; } @Override diff --git a/test/unit/org/apache/cassandra/locator/MetaStrategyTest.java b/test/unit/org/apache/cassandra/locator/MetaStrategyTest.java index 36cc9d154702..5e3cdd4c42ba 100644 --- a/test/unit/org/apache/cassandra/locator/MetaStrategyTest.java +++ b/test/unit/org/apache/cassandra/locator/MetaStrategyTest.java @@ -33,8 +33,8 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.schema.DistributedSchema; -import org.apache.cassandra.service.accord.AccordFastPath; -import org.apache.cassandra.service.accord.AccordStaleReplicas; +import org.apache.cassandra.service.accord.topology.AccordFastPath; +import org.apache.cassandra.service.accord.topology.AccordStaleReplicas; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Epoch; diff --git a/test/unit/org/apache/cassandra/schema/FastPathSchemaTest.java b/test/unit/org/apache/cassandra/schema/FastPathSchemaTest.java index e4c603079821..03f0732c4b13 100644 --- a/test/unit/org/apache/cassandra/schema/FastPathSchemaTest.java +++ b/test/unit/org/apache/cassandra/schema/FastPathSchemaTest.java @@ -30,8 +30,8 @@ import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.service.accord.fastpath.FastPathStrategy; -import org.apache.cassandra.service.accord.fastpath.ParameterizedFastPathStrategy; +import org.apache.cassandra.service.accord.topology.FastPathStrategy; +import org.apache.cassandra.service.accord.topology.ParameterizedFastPathStrategy; import static java.lang.String.format; diff --git a/test/unit/org/apache/cassandra/service/accord/AccordCacheTest.java b/test/unit/org/apache/cassandra/service/accord/AccordCacheTest.java index 4c514ec5bab3..5cb919a66e30 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordCacheTest.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordCacheTest.java @@ -93,13 +93,13 @@ public Throwable failure() } @Override - public void invalidate() + public void markUnsafe() { invalidated = true; } @Override - public boolean invalidated() + public boolean isUnsafe() { return invalidated; } diff --git a/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java b/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java index bd8cc9f2aec6..a53ab74e19d7 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java @@ -67,6 +67,7 @@ import static org.apache.cassandra.service.accord.AccordTestUtils.fullRange; import static org.apache.cassandra.service.accord.AccordTestUtils.timestamp; import static org.apache.cassandra.service.accord.AccordTestUtils.txnId; +import static org.apache.cassandra.service.accord.SimulatedAccordCommandStoreTestBase.emptyNode; public class AccordCommandTest { @@ -107,6 +108,7 @@ public void basicCycleTest() throws Throwable Route route = fullRoute.overlapping(fullRange(txn)); PartialTxn partialTxn = txn.intersecting(route, true); PreAccept preAccept = PreAccept.SerializerSupport.create(txnId, route, 1, 1, 1, partialTxn, null, false, fullRoute); + preAccept.unsafeSetNode(emptyNode); // Check preaccept getBlocking(commandStore.execute(preAccept, safeStore -> { @@ -199,9 +201,11 @@ public void computeDeps() throws Throwable Route route = fullRoute.overlapping(fullRange(txn)); PartialTxn partialTxn = txn.intersecting(route, true); PreAccept preAccept1 = PreAccept.SerializerSupport.create(txnId1, route, 1, 1, 1, partialTxn, null, false, fullRoute); + preAccept1.unsafeSetNode(emptyNode); getBlocking(commandStore.execute(preAccept1, safeStore -> { persistDiff(commandStore, safeStore, txnId1, route, () -> { + preAccept1.unsafeSetNode(emptyNode); preAccept1.apply(safeStore); }); })); @@ -209,6 +213,7 @@ public void computeDeps() throws Throwable // second preaccept should identify txnId1 as a dependency TxnId txnId2 = txnId(1, clock.incrementAndGet(), 1); PreAccept preAccept2 = PreAccept.SerializerSupport.create(txnId2, route, 1, 1, 1, partialTxn, null, false, fullRoute); + preAccept2.unsafeSetNode(emptyNode); getBlocking(commandStore.execute(preAccept2, safeStore -> { persistDiff(commandStore, safeStore, txnId2, route, () -> { PreAccept.PreAcceptReply reply = preAccept2.apply(safeStore); diff --git a/test/unit/org/apache/cassandra/service/accord/AccordJournalOrderTest.java b/test/unit/org/apache/cassandra/service/accord/AccordJournalOrderTest.java index a4b8c75955b2..77a47084c5cb 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordJournalOrderTest.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordJournalOrderTest.java @@ -51,6 +51,8 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.service.accord.journal.AccordJournal; +import org.apache.cassandra.service.accord.journal.CommandChanges; import org.apache.cassandra.service.consensus.TransactionalMode; import org.apache.cassandra.utils.StorageCompatibilityMode; @@ -97,7 +99,7 @@ public void simpleKeyTest() Runnable check = () -> { for (JournalKey key : res.keySet()) { - AccordJournal.Builder diffs = accordJournal.load(key.commandStoreId, key.id); + CommandChanges diffs = accordJournal.load(key.commandStoreId, key.id); Assert.assertEquals(String.format("%d != %d for key %s", diffs.count(), res.get(key).intValue(), key), diffs.count(), res.get(key).intValue()); } diff --git a/test/unit/org/apache/cassandra/service/accord/AccordMessageSinkTest.java b/test/unit/org/apache/cassandra/service/accord/AccordMessageSinkTest.java index c9a774fa18e9..98972fda1dc8 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordMessageSinkTest.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordMessageSinkTest.java @@ -53,6 +53,7 @@ import org.apache.cassandra.net.MessageDelivery; import org.apache.cassandra.service.accord.AccordFetchCoordinator.AccordFetchRequest; import org.apache.cassandra.service.accord.api.AccordTimeService; +import org.apache.cassandra.service.accord.topology.AccordEndpointMapper; import org.apache.cassandra.tcm.ClusterMetadataService; public class AccordMessageSinkTest diff --git a/test/unit/org/apache/cassandra/service/accord/AccordSerializersTest.java b/test/unit/org/apache/cassandra/service/accord/AccordSerializersTest.java index 44908a0fa115..e52d27b26ad9 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordSerializersTest.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordSerializersTest.java @@ -22,6 +22,7 @@ import org.apache.cassandra.io.Serializers; import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.service.accord.serializers.AccordSerializers; import org.apache.cassandra.utils.CassandraGenerators; import org.apache.cassandra.utils.Generators; diff --git a/test/unit/org/apache/cassandra/service/accord/AccordStaleReplicasTest.java b/test/unit/org/apache/cassandra/service/accord/AccordStaleReplicasTest.java index 63f3f3766022..116e1877ee5a 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordStaleReplicasTest.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordStaleReplicasTest.java @@ -31,6 +31,7 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.service.accord.topology.AccordStaleReplicas; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.serialization.AsymmetricMetadataSerializers; import org.apache.cassandra.tcm.serialization.Version; diff --git a/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java b/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java index 84c4a57d612b..bae8c0085c69 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java @@ -103,6 +103,7 @@ import org.apache.cassandra.service.accord.AccordCacheEntry.LoadExecutor; import org.apache.cassandra.service.accord.api.AccordAgent; import org.apache.cassandra.service.accord.api.PartitionKey; +import org.apache.cassandra.service.accord.journal.AccordJournal; import org.apache.cassandra.service.accord.serializers.TableMetadatas; import org.apache.cassandra.service.accord.serializers.TableMetadatasAndKeys; import org.apache.cassandra.service.accord.txn.TxnData; diff --git a/test/unit/org/apache/cassandra/service/accord/AccordTopologyTest.java b/test/unit/org/apache/cassandra/service/accord/AccordTopologyTest.java index 596d99da2736..1b19f0fd42a6 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordTopologyTest.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordTopologyTest.java @@ -44,6 +44,8 @@ import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.Tables; +import org.apache.cassandra.service.accord.topology.AccordFastPath; +import org.apache.cassandra.service.accord.topology.AccordTopology; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.membership.Location; diff --git a/test/unit/org/apache/cassandra/service/accord/CommandChangeTest.java b/test/unit/org/apache/cassandra/service/accord/CommandChangeTest.java index 312a67f3c457..4fc35b49c879 100644 --- a/test/unit/org/apache/cassandra/service/accord/CommandChangeTest.java +++ b/test/unit/org/apache/cassandra/service/accord/CommandChangeTest.java @@ -42,6 +42,8 @@ import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.accord.journal.CommandChangeWriter; +import org.apache.cassandra.service.accord.journal.CommandChanges; import org.apache.cassandra.service.accord.serializers.Version; import org.apache.cassandra.service.consensus.TransactionalMode; import org.apache.cassandra.utils.AccordGenerators; @@ -109,8 +111,8 @@ public void serde() out.clear(); Command orig = cmdBuilder.build(saveStatus); - AccordJournal.Writer.make(null, orig).write(out, version); - AccordJournal.Builder builder = new AccordJournal.Builder(orig.txnId(), Load.ALL); + CommandChangeWriter.make(null, orig).write(out, version); + CommandChanges builder = new CommandChanges(orig.txnId(), Load.ALL); builder.deserializeNext(new DataInputBuffer(out.unsafeGetBufferAndFlip(), false), version); // We are not persisting the result, so force it for strict equality builder.forceResult(orig.result()); diff --git a/test/unit/org/apache/cassandra/service/accord/DurableBeforeIntegrationTest.java b/test/unit/org/apache/cassandra/service/accord/DurableBeforeIntegrationTest.java new file mode 100644 index 000000000000..b0374e5d8a5f --- /dev/null +++ b/test/unit/org/apache/cassandra/service/accord/DurableBeforeIntegrationTest.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.accord; + +import java.io.IOException; + +import org.junit.Assert; +import org.junit.Test; + +import accord.api.RoutingKey; +import accord.local.AbstractDurableBeforeTest; +import accord.local.DurableBefore; +import accord.primitives.Ranges; +import accord.utils.Gen; +import accord.utils.RandomSource; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.io.Serializers; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.service.accord.serializers.CommandStoreSerializers; + +import static org.apache.cassandra.service.accord.serializers.CommandStoreSerializersTest.durableBeforeLinear; + +public class DurableBeforeIntegrationTest extends AbstractDurableBeforeTest +{ + static + { + DatabaseDescriptor.clientInitialization(); + DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); + } + + @Override + protected RoutingKey key(int prefix, int hash) + { + return new TokenKey(TableId.fromLong(1 + prefix), new Murmur3Partitioner.LongToken(hash)); + } + + @Test + @Override + public void test() + { + super.test(); + } + + @Override + protected void assertEquals(Object a, Object b) + { + Assert.assertEquals(a, b); + } + + @Override + protected void assertTrue(boolean isTrue) + { + Assert.assertTrue(isTrue); + } + + @Override + protected void check(RandomSource rs, DurableBefore tree, DurableBeforeLinear linear, Gen genRanges) + { + super.check(rs, tree, linear, genRanges); + if (rs.decide(0.1f)) + testSer(tree, linear); + } + + static void testSer(DurableBefore tree, DurableBeforeLinear linear) + { + try (DataOutputBuffer buffer = new DataOutputBuffer()) + { + Serializers.testSerde(buffer, CommandStoreSerializers.durableBefore, tree); + Serializers.testSerde(buffer, durableBeforeLinear, linear, CommandStoreSerializers.durableBefore, DurableBeforeLinear::isEqualTo); + } + catch (IOException e) + { + throw new AssertionError(e); + } + } +} diff --git a/test/unit/org/apache/cassandra/service/accord/EndpointMappingTest.java b/test/unit/org/apache/cassandra/service/accord/EndpointMappingTest.java index 5a54dec82737..2ba17f547261 100644 --- a/test/unit/org/apache/cassandra/service/accord/EndpointMappingTest.java +++ b/test/unit/org/apache/cassandra/service/accord/EndpointMappingTest.java @@ -24,6 +24,7 @@ import accord.local.Node; +import org.apache.cassandra.service.accord.topology.EndpointMapping; import org.apache.cassandra.utils.CassandraGenerators; import static org.quicktheories.QuickTheory.qt; diff --git a/test/unit/org/apache/cassandra/service/accord/EpochSyncTest.java b/test/unit/org/apache/cassandra/service/accord/EpochSyncTest.java index c2070a76f507..6b92f1b9800a 100644 --- a/test/unit/org/apache/cassandra/service/accord/EpochSyncTest.java +++ b/test/unit/org/apache/cassandra/service/accord/EpochSyncTest.java @@ -94,6 +94,10 @@ import org.apache.cassandra.schema.TableParams; import org.apache.cassandra.schema.Tables; import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.service.accord.topology.AccordEndpointMapper; +import org.apache.cassandra.service.accord.topology.AccordSyncPropagator; +import org.apache.cassandra.service.accord.topology.AccordTopology; +import org.apache.cassandra.service.accord.topology.AccordTopologyService; import org.apache.cassandra.service.consensus.TransactionalMode; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; diff --git a/test/unit/org/apache/cassandra/service/accord/FetchTopologiesTest.java b/test/unit/org/apache/cassandra/service/accord/FetchTopologiesTest.java index bf94bede9058..72d7dfaaf96c 100644 --- a/test/unit/org/apache/cassandra/service/accord/FetchTopologiesTest.java +++ b/test/unit/org/apache/cassandra/service/accord/FetchTopologiesTest.java @@ -23,6 +23,7 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.io.Serializers; import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.service.accord.topology.FetchTopologies; import org.apache.cassandra.utils.AccordGenerators; import static accord.utils.Property.qt; diff --git a/test/unit/org/apache/cassandra/service/accord/MaxConflictsIntegrationTest.java b/test/unit/org/apache/cassandra/service/accord/MaxConflictsIntegrationTest.java new file mode 100644 index 000000000000..439eefe8b198 --- /dev/null +++ b/test/unit/org/apache/cassandra/service/accord/MaxConflictsIntegrationTest.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.accord; + +import java.io.IOException; + +import org.junit.Assert; +import org.junit.Test; + +import accord.api.RoutingKey; +import accord.local.AbstractMaxConflictsTest; +import accord.local.MaxConflicts; +import accord.local.MaxConflictsTest; +import accord.utils.RandomSource; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.io.Serializers; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.service.accord.serializers.CommandStoreSerializers; + +public class MaxConflictsIntegrationTest extends AbstractMaxConflictsTest +{ + static + { + DatabaseDescriptor.clientInitialization(); + DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); + } + + @Override + protected void assertTrue(boolean isTrue) + { + Assert.assertTrue(isTrue); + } + + @Test + @Override + public void test() + { + super.test(); + } + + @Override + protected RoutingKey key(int prefix, int hash) + { + return new TokenKey(TableId.fromLong(1 + prefix), new Murmur3Partitioner.LongToken(hash)); + } + + @Override + protected void check(RandomSource rs, MaxConflicts prev, MaxConflicts next) + { + super.check(rs, prev, next); + if (rs.decide(0.1f)) + testSer(next); + } + + static void testSer(MaxConflicts tree) + { + try (DataOutputBuffer buffer = new DataOutputBuffer()) + { + Serializers.testSerde(buffer, CommandStoreSerializers.maxConflicts, tree); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + } +} diff --git a/src/java/org/apache/cassandra/service/accord/RangeTreeRangeAccessor.java b/test/unit/org/apache/cassandra/service/accord/RangeTreeRangeAccessor.java similarity index 100% rename from src/java/org/apache/cassandra/service/accord/RangeTreeRangeAccessor.java rename to test/unit/org/apache/cassandra/service/accord/RangeTreeRangeAccessor.java diff --git a/test/unit/org/apache/cassandra/service/accord/RouteInMemoryIndexTest.java b/test/unit/org/apache/cassandra/service/accord/RouteInMemoryIndexTest.java index 3744a6c7c20b..98dbac8e104b 100644 --- a/test/unit/org/apache/cassandra/service/accord/RouteInMemoryIndexTest.java +++ b/test/unit/org/apache/cassandra/service/accord/RouteInMemoryIndexTest.java @@ -49,6 +49,8 @@ import org.apache.cassandra.index.accord.TxnRange; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.service.accord.journal.RangeSearcher; +import org.apache.cassandra.service.accord.journal.SegmentRangeSearcher; import org.apache.cassandra.utils.CloseableIterator; import static accord.utils.Property.commands; @@ -133,7 +135,7 @@ private static TxnRange nextTxnRange(RandomSource rs, State state) private static class State { - private final JournalSegmentRangeSearcher index = new JournalSegmentRangeSearcher<>(); + private final SegmentRangeSearcher index = new SegmentRangeSearcher<>(); private final Model model = new Model(); private final float unfiltered; private final float minDecidedIdNull; @@ -279,17 +281,17 @@ void update(long segment, TxnId txnId, TokenRange range) segments.computeIfAbsent(segment, i -> new Segment()).add(range, txnId); } - public JournalRangeSearcher.Result search(TokenRange range, TxnId minTxnId, TxnId maxTxnId, @Nullable DecidedRX decidedRX) + public RangeSearcher.Result search(TokenRange range, TxnId minTxnId, TxnId maxTxnId, @Nullable DecidedRX decidedRX) { return search(vrange -> range.compareIntersecting(vrange) == 0, minTxnId, maxTxnId, decidedRX); } - public JournalRangeSearcher.Result search(TokenKey key, TxnId minTxnId, TxnId maxTxnId, @Nullable DecidedRX decidedRX) + public RangeSearcher.Result search(TokenKey key, TxnId minTxnId, TxnId maxTxnId, @Nullable DecidedRX decidedRX) { return search(range -> range.contains(key), minTxnId, maxTxnId, decidedRX); } - public JournalRangeSearcher.Result search(Predicate test, TxnId minTxnId, TxnId maxTxnId, @Nullable DecidedRX decidedRX) + public RangeSearcher.Result search(Predicate test, TxnId minTxnId, TxnId maxTxnId, @Nullable DecidedRX decidedRX) { TreeSet result = new TreeSet<>(); for (var segment: segments.values()) @@ -302,7 +304,7 @@ public JournalRangeSearcher.Result search(Predicate test, TxnId minT result.add(value.txnId); } } - return new JournalRangeSearcher.DefaultResult(minTxnId, maxTxnId, decidedRX, CloseableIterator.wrap(result.iterator())); + return new RangeSearcher.DefaultResult(minTxnId, maxTxnId, decidedRX, CloseableIterator.wrap(result.iterator())); } void remove(long segment) diff --git a/test/unit/org/apache/cassandra/service/accord/SimpleAccordEndpointMapper.java b/test/unit/org/apache/cassandra/service/accord/SimpleAccordEndpointMapper.java index 707017da128f..02a423964c40 100644 --- a/test/unit/org/apache/cassandra/service/accord/SimpleAccordEndpointMapper.java +++ b/test/unit/org/apache/cassandra/service/accord/SimpleAccordEndpointMapper.java @@ -26,6 +26,7 @@ import accord.local.Node; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.service.accord.topology.AccordEndpointMapper; import org.apache.cassandra.utils.ByteBufferUtil; public enum SimpleAccordEndpointMapper implements AccordEndpointMapper diff --git a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java index 91668c5312a0..8bb62c3a6752 100644 --- a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java +++ b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java @@ -94,6 +94,9 @@ import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.service.accord.journal.RangeSearcher; +import org.apache.cassandra.service.accord.journal.SegmentRangeSearcher; +import org.apache.cassandra.service.accord.topology.AccordTopology; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.Generators; @@ -103,6 +106,7 @@ import static org.apache.cassandra.db.ColumnFamilyStore.FlushReason.UNIT_TESTS; import static org.apache.cassandra.schema.SchemaConstants.ACCORD_KEYSPACE_NAME; import static org.apache.cassandra.service.accord.AccordService.getBlocking; +import static org.apache.cassandra.service.accord.SimulatedAccordCommandStoreTestBase.emptyNode; import static org.apache.cassandra.utils.AccordGenerators.fromQT; public class SimulatedAccordCommandStore implements AutoCloseable @@ -457,6 +461,7 @@ public Pair> enqueuePreAccept(Txn txn, TxnId txnId = nextTxnId(txn.kind(), txn.keys().domain()); PreAccept preAccept = new PreAccept(nodeId, topologies, txnId, txn, null, false, route); return Pair.create(txnId, processAsync(preAccept, safe -> { + preAccept.unsafeSetNode(emptyNode); var reply = preAccept.apply(safe); Assertions.assertThat(reply.isOk()).isTrue(); return (PreAccept.PreAcceptOk) reply; @@ -496,9 +501,9 @@ public void close() throws Exception commandStore.shutdown(); } - private static class DefaultJournal extends InMemoryJournal implements JournalRangeSearcher.Supplier + private static class DefaultJournal extends InMemoryJournal implements RangeSearcher.Supplier { - private final JournalSegmentRangeSearcher index = new JournalSegmentRangeSearcher<>(); + private final SegmentRangeSearcher index = new SegmentRangeSearcher<>(); private DefaultJournal(Node.Id id, RandomSource rs) { super(id, rs); @@ -524,7 +529,7 @@ public void purge(CommandStores commandStores, EpochSupplier epochSupplier) } @Override - public JournalRangeSearcher rangeSearcher() + public RangeSearcher rangeSearcher() { return index; } diff --git a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStoreTestBase.java b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStoreTestBase.java index 726987898a2e..cd39bacfe556 100644 --- a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStoreTestBase.java +++ b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStoreTestBase.java @@ -34,8 +34,11 @@ import org.junit.Before; import org.junit.BeforeClass; +import accord.Utils; import accord.api.RoutingKey; import accord.impl.SizeOfIntersectionSorter; +import accord.impl.TestAgent; +import accord.impl.mock.MockCluster; import accord.local.Node; import accord.messages.BeginRecovery; import accord.messages.PreAccept; @@ -55,6 +58,7 @@ import accord.primitives.TxnId; import accord.primitives.Unseekables; import accord.topology.Topologies; +import accord.topology.Topology; import accord.utils.Gen; import accord.utils.Gens; import accord.utils.Invariants; @@ -73,6 +77,7 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.accord.api.PartitionKey; import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.service.accord.topology.AccordTopology; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.RTree; @@ -94,8 +99,7 @@ public abstract class SimulatedAccordCommandStoreTestBase extends CQLTester CassandraRelevantProperties.SAI_TEST_DISABLE_TIMEOUT.setBoolean(true); } - protected enum DepsMessage - {PreAccept, BeginRecovery, PreAcceptThenBeginRecovery} + protected enum DepsMessage {PreAccept, BeginRecovery, PreAcceptThenBeginRecovery} protected static final Gen> mixedDomainGen = Gens.enums().allMixedDistribution(Routable.Domain.class); protected static final Gen mixedTokenGen = top -> { @@ -114,6 +118,7 @@ protected enum DepsMessage protected static TableMetadata intTbl, reverseTokenTbl; protected static Node.Id nodeId; + protected static final Node emptyNode = Utils.createNode(Node.Id.NONE, Topology.EMPTY, null, new MockCluster.Clock(0), new TestAgent()); @BeforeClass public static void setUpClass() @@ -282,6 +287,7 @@ protected static Pair> assertBeginRecoveryAfterPreAcceptAs PreAccept preAccept = new PreAccept(nodeId, new Topologies.Single(SizeOfIntersectionSorter.SUPPLIER, instance.topology), txnId, txn, null, false, route); var preAcceptAsync = instance.processAsync(preAccept, safe -> { + preAccept.unsafeSetNode(emptyNode); var reply = preAccept.apply(safe); Assertions.assertThat(reply.isOk()).isTrue(); PreAccept.PreAcceptOk success = (PreAccept.PreAcceptOk) reply; diff --git a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordTaskTest.java b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordTaskTest.java index c824516b9b77..e32144b77226 100644 --- a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordTaskTest.java +++ b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordTaskTest.java @@ -123,6 +123,7 @@ private static void test(RandomSource rs, int numSamples, TableMetadata tbl, Gen @Override public PreAcceptReply applyInternal(SafeCommandStore safeStore) { + unsafeSetNode(emptyNode); PreAcceptReply result = super.applyInternal(safeStore); if (action == Action.FAILURE) throw new SimulatedFault("PreAccept failed for keys " + keys()); diff --git a/test/unit/org/apache/cassandra/service/accord/WatermarkCollectorTest.java b/test/unit/org/apache/cassandra/service/accord/WatermarkCollectorTest.java index 59210bbe967f..4b9d45847909 100644 --- a/test/unit/org/apache/cassandra/service/accord/WatermarkCollectorTest.java +++ b/test/unit/org/apache/cassandra/service/accord/WatermarkCollectorTest.java @@ -36,6 +36,7 @@ import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.io.Serializers; import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.service.accord.topology.WatermarkCollector; import org.apache.cassandra.utils.AccordGenerators; import static accord.utils.Property.qt; diff --git a/test/unit/org/apache/cassandra/service/accord/journal/AccordTopologyUpdateTest.java b/test/unit/org/apache/cassandra/service/accord/journal/TopologyRecordTest.java similarity index 58% rename from test/unit/org/apache/cassandra/service/accord/journal/AccordTopologyUpdateTest.java rename to test/unit/org/apache/cassandra/service/accord/journal/TopologyRecordTest.java index 21b7e7bc3aed..199e7f20db87 100644 --- a/test/unit/org/apache/cassandra/service/accord/journal/AccordTopologyUpdateTest.java +++ b/test/unit/org/apache/cassandra/service/accord/journal/TopologyRecordTest.java @@ -40,11 +40,10 @@ import org.apache.cassandra.utils.AccordGenerators; import static accord.utils.Property.qt; +import static org.apache.cassandra.service.accord.serializers.CommandStoreSerializersTest.rangesForEpochGen; -public class AccordTopologyUpdateTest +public class TopologyRecordTest { - private static final long[] EPOCHS = new long[0]; - private static final Ranges[] RANGES = new Ranges[0]; private static final TableId TBL1 = TableId.fromRaw(0, 0); static @@ -58,23 +57,13 @@ public void before() DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); } - @Test - public void rangesForEpoch() - { - @SuppressWarnings({ "resource", "IOResourceOpenedButNotSafelyClosed" }) DataOutputBuffer output = new DataOutputBuffer(); - qt().forAll(rangesForEpochGen()).check(expected -> { - maybeUpdatePartitioner(expected); - Serializers.testSerde(output, AccordTopologyUpdate.RangesForEpochSerializer.instance, expected); - }); - } - @Test public void topologyUpdate() { @SuppressWarnings({ "resource", "IOResourceOpenedButNotSafelyClosed" }) DataOutputBuffer output = new DataOutputBuffer(); qt().forAll(topologyUpdateGen()).check(expected -> { maybeUpdatePartitioner(expected); - Serializers.testSerde(output, AccordTopologyUpdate.TopologyUpdateSerializer.instance, expected); + Serializers.testSerde(output, TopologyRecord.TopologyUpdateSerializer.instance, expected); }); } @@ -84,35 +73,10 @@ public void accordTopologyUpdate() @SuppressWarnings({ "resource", "IOResourceOpenedButNotSafelyClosed" }) DataOutputBuffer output = new DataOutputBuffer(); qt().forAll(accordTopologyUpdateGen()).check(expected -> { maybeUpdatePartitioner(expected); - Serializers.testSerde(output, AccordTopologyUpdate.Serializer.instance, expected); + Serializers.testSerde(output, TopologyRecord.Serializer.instance, expected); }); } - private static Gen rangesForEpochGen() - { - return AccordGenerators.partitioner().flatMap(p -> rangesForEpochGen(AccordGenerators.rangesSplitOrArbitrary(p))); - } - - private static Gen rangesForEpochGen(Gen rangesGen) - { - Gen.IntGen sizeGen = Gens.ints().between(0, 10); - Gen.LongGen epochGen = AccordGens.epochs(); - return rs -> { - int size = sizeGen.nextInt(rs); - if (size == 0) - return new CommandStores.RangesForEpoch(EPOCHS, RANGES); - long epoch = epochGen.nextLong(rs); - long[] epochs = new long[size]; - Ranges[] ranges = new Ranges[size]; - for (int i = 0; i < size; i++) - { - epochs[i] = epoch++; - ranges[i] = rangesGen.next(rs); - } - return new CommandStores.RangesForEpoch(epochs, ranges); - }; - } - private static Gen topologyUpdateGen() { Gen partitionerGen = AccordGenerators.partitioner(); @@ -132,18 +96,18 @@ private static Gen topologyUpdateGen() }; } - private static Gen accordTopologyUpdateGen() + private static Gen accordTopologyUpdateGen() { Gen.LongGen epochGen = AccordGens.epochs(); Gen topologyUpdateGen = topologyUpdateGen(); - Gen kindGen = Gens.enums().all(AccordTopologyUpdate.Kind.class); + Gen kindGen = Gens.enums().all(TopologyRecord.Kind.class); return rs -> { - AccordTopologyUpdate.Kind kind = kindGen.next(rs); + TopologyRecord.Kind kind = kindGen.next(rs); switch (kind) { - case New: return new AccordTopologyUpdate.NewTopology(topologyUpdateGen.next(rs)); - case Image: return new AccordTopologyUpdate.TopologyImage(epochGen.nextLong(rs), AccordTopologyUpdate.Kind.Image, topologyUpdateGen.next(rs)); - case Repeat: return new AccordTopologyUpdate.TopologyImage(epochGen.nextLong(rs), AccordTopologyUpdate.Kind.Repeat); + case New: return new TopologyRecord.NewTopology(topologyUpdateGen.next(rs)); + case Image: return new TopologyRecord.TopologyImage(epochGen.nextLong(rs), TopologyRecord.Kind.Image, topologyUpdateGen.next(rs)); + case Repeat: return new TopologyRecord.TopologyImage(epochGen.nextLong(rs), TopologyRecord.Kind.Repeat); default: throw new AssertionError("Unknown kind: " + kind); } }; @@ -154,23 +118,10 @@ private static void maybeUpdatePartitioner(Journal.TopologyUpdate expected) AccordGenerators.maybeUpdatePartitioner(expected.global.ranges()); } - private static void maybeUpdatePartitioner(AccordTopologyUpdate expected) + private static void maybeUpdatePartitioner(TopologyRecord expected) { Journal.TopologyUpdate update = expected.getUpdate(); if (update != null) maybeUpdatePartitioner(expected.getUpdate()); } - - private void maybeUpdatePartitioner(CommandStores.RangesForEpoch expected) - { - if (expected.size() > 0) - { - for (int i = 0; i < expected.size(); i++) - { - Ranges ranges = expected.rangesAtIndex(i); - if (AccordGenerators.maybeUpdatePartitioner(ranges)) - return; - } - } - } } \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/service/accord/repair/RequiredResponseTrackerTest.java b/test/unit/org/apache/cassandra/service/accord/repair/RequiredResponseTrackerTest.java index 14a28939e00b..c8570a1a6fd3 100644 --- a/test/unit/org/apache/cassandra/service/accord/repair/RequiredResponseTrackerTest.java +++ b/test/unit/org/apache/cassandra/service/accord/repair/RequiredResponseTrackerTest.java @@ -45,7 +45,7 @@ import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.Tables; -import org.apache.cassandra.service.accord.AccordTopology; +import org.apache.cassandra.service.accord.topology.AccordTopology; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.membership.Location; diff --git a/test/unit/org/apache/cassandra/service/accord/serializers/CommandStoreSerializersTest.java b/test/unit/org/apache/cassandra/service/accord/serializers/CommandStoreSerializersTest.java index 4151ca53263e..146c3abc565c 100644 --- a/test/unit/org/apache/cassandra/service/accord/serializers/CommandStoreSerializersTest.java +++ b/test/unit/org/apache/cassandra/service/accord/serializers/CommandStoreSerializersTest.java @@ -18,21 +18,38 @@ package org.apache.cassandra.service.accord.serializers; +import java.io.IOException; + import org.junit.Test; +import accord.local.AbstractDurableBeforeTest.DurableBeforeLinear; +import accord.local.CommandStores; +import accord.local.DurableBefore; import accord.local.RedundantBefore; +import accord.primitives.Ranges; +import accord.primitives.TxnId; +import accord.utils.AccordGens; +import accord.utils.Gen; import accord.utils.Gens; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.io.Serializers; +import org.apache.cassandra.io.UnversionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.service.accord.serializers.CommandStoreSerializers.ReducingRangeMapSerializer; import org.apache.cassandra.utils.AccordGenerators; +import org.apache.cassandra.utils.NullableSerializer; import static accord.utils.Property.qt; public class CommandStoreSerializersTest { + private static final long[] EPOCHS = new long[0]; + private static final Ranges[] RANGES = new Ranges[0]; + static { DatabaseDescriptor.clientInitialization(); @@ -46,7 +63,7 @@ public void redundantBeforeEntry() qt().forAll(Gens.random(), AccordGenerators.partitioner()).check((rs, partitioner) -> { DatabaseDescriptor.setPartitionerUnsafe(partitioner); RedundantBefore.Bounds entry = AccordGenerators.redundantBeforeEntry(partitioner).next(rs); - Serializers.testSerde(buffer, CommandStoreSerializers.redundantBeforeEntry, entry); + Serializers.testSerde(buffer, CommandStoreSerializers.redundantBeforeShortBounds, entry); }); } @@ -62,4 +79,93 @@ public void redundantBefore() }); } + @Test + public void durableBefore() + { + DataOutputBuffer buffer = new DataOutputBuffer(); + qt().forAll(Gens.random(), AccordGenerators.partitioner()).check((rs, partitioner) -> { + DatabaseDescriptor.setPartitionerUnsafe(partitioner); + // serializer doesn't support the empty set, so filter out + DurableBefore durableBefore = AccordGenerators.durableBeforeGen(partitioner).next(rs); + Serializers.testSerde(buffer, CommandStoreSerializers.durableBefore, durableBefore); + Serializers.testSerde(buffer, durableBeforeLinear, DurableBeforeLinear.from(durableBefore), CommandStoreSerializers.durableBefore, DurableBeforeLinear::isEqualTo); + }); + } + + @Test + public void rangesForEpoch() + { + @SuppressWarnings({ "resource", "IOResourceOpenedButNotSafelyClosed" }) DataOutputBuffer output = new DataOutputBuffer(); + qt().forAll(rangesForEpochGen()).check(expected -> { + maybeUpdatePartitioner(expected); + Serializers.testSerde(output, CommandStoreSerializers.rangesForEpoch, expected); + }); + } + + public static Gen rangesForEpochGen() + { + return AccordGenerators.partitioner().flatMap(p -> rangesForEpochGen(AccordGenerators.rangesSplitOrArbitrary(p))); + } + + public static Gen rangesForEpochGen(Gen rangesGen) + { + Gen.IntGen sizeGen = Gens.ints().between(0, 10); + Gen.LongGen epochGen = AccordGens.epochs(); + return rs -> { + int size = sizeGen.nextInt(rs); + if (size == 0) + return new CommandStores.RangesForEpoch(EPOCHS, RANGES); + long epoch = epochGen.nextLong(rs); + long[] epochs = new long[size]; + Ranges[] ranges = new Ranges[size]; + for (int i = 0; i < size; i++) + { + epochs[i] = epoch++; + ranges[i] = rangesGen.next(rs); + } + return new CommandStores.RangesForEpoch(epochs, ranges); + }; + } + + private void maybeUpdatePartitioner(CommandStores.RangesForEpoch expected) + { + if (expected.size() > 0) + { + for (int i = 0; i < expected.size(); i++) + { + Ranges ranges = expected.rangesAtIndex(i); + if (AccordGenerators.maybeUpdatePartitioner(ranges)) + return; + } + } + } + + static final UnversionedSerializer durableBeforeEntry = new NonTreeDurableBeforeEntrySerializer(); + public static final UnversionedSerializer durableBeforeLinear = new ReducingRangeMapSerializer<>(NullableSerializer.wrap(durableBeforeEntry), DurableBefore.Entry[]::new, (i1, i2) -> { throw new UnsupportedOperationException(); }, DurableBeforeLinear.EMPTY); + private static final class NonTreeDurableBeforeEntrySerializer implements UnversionedSerializer + { + private NonTreeDurableBeforeEntrySerializer() {} + + @Override + public void serialize(DurableBefore.Entry t, DataOutputPlus out) throws IOException + { + CommandSerializers.txnId.serialize(t.quorum, out); + CommandSerializers.txnId.serialize(t.universal, out); + } + + @Override + public DurableBefore.Entry deserialize(DataInputPlus in) throws IOException + { + TxnId quorumBefore = CommandSerializers.txnId.deserialize(in); + TxnId universalBefore = CommandSerializers.txnId.deserialize(in); + return DurableBefore.Entry.constructWithoutRange(quorumBefore, universalBefore); + } + + @Override + public long serializedSize(DurableBefore.Entry t) + { + return CommandSerializers.txnId.serializedSize(t.quorum) + + CommandSerializers.txnId.serializedSize(t.universal); + } + } } \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/service/accord/serializers/CommandsForKeySerializerTest.java b/test/unit/org/apache/cassandra/service/accord/serializers/CommandsForKeySerializerTest.java index e9718d32a009..18fb17078b34 100644 --- a/test/unit/org/apache/cassandra/service/accord/serializers/CommandsForKeySerializerTest.java +++ b/test/unit/org/apache/cassandra/service/accord/serializers/CommandsForKeySerializerTest.java @@ -55,6 +55,7 @@ import accord.api.ProgressLog; import accord.api.RoutingKey; import accord.api.Timeouts; +import accord.impl.AbstractReplayer; import accord.impl.AbstractSafeCommandStore; import accord.impl.DefaultLocalListeners; import accord.impl.DefaultRemoteListeners; @@ -655,7 +656,7 @@ protected TestCommandStore() @Override public AsyncChain chain(PreLoadContext context, Consumer consumer) { throw new UnsupportedOperationException();} @Override public AsyncChain chain(PreLoadContext context, Function apply) { throw new UnsupportedOperationException(); } - @Override public Journal.Replayer replayer() { throw new UnsupportedOperationException(); } + @Override public Journal.Replayer replayer(AbstractReplayer.Mode mode) { throw new UnsupportedOperationException(); } @Override protected void ensureDurable(Ranges ranges, RedundantBefore onSuccess) {} @Override public Agent agent() { return this; } diff --git a/test/unit/org/apache/cassandra/service/accord/serializers/TokenKeyTest.java b/test/unit/org/apache/cassandra/service/accord/serializers/TokenKeyTest.java index e04259869aea..5d058c079f0f 100644 --- a/test/unit/org/apache/cassandra/service/accord/serializers/TokenKeyTest.java +++ b/test/unit/org/apache/cassandra/service/accord/serializers/TokenKeyTest.java @@ -142,11 +142,11 @@ public void serde() Assertions.assertThat(roundTrip).isEqualTo(key); } { - TokenKey roundTrip = serializer.deserializeWithPrefix(key.prefix(), serializer.serializedSizeWithoutPrefix(key), serializer.serializeWithoutPrefixOrLength(key), partitioner); + TokenKey roundTrip = serializer.deserializeWithPrefix(key.prefix(), serializer.serializedSizeWithoutPrefixOrLength(key), serializer.serializeWithoutPrefixOrLength(key), partitioner); Assertions.assertThat(roundTrip).isEqualTo(key); } { - TokenKey roundTrip = serializer.deserializeWithPrefix(key.prefix(), serializer.serializedSizeWithoutPrefix(key), serializer.serializeWithoutPrefixOrLength(key), ByteBufferAccessor.instance, 0, partitioner); + TokenKey roundTrip = serializer.deserializeWithPrefix(key.prefix(), serializer.serializedSizeWithoutPrefixOrLength(key), serializer.serializeWithoutPrefixOrLength(key), ByteBufferAccessor.instance, 0, partitioner); Assertions.assertThat(roundTrip).isEqualTo(key); } output.clear(); diff --git a/test/unit/org/apache/cassandra/service/accord/AccordFastPathCoordinatorTest.java b/test/unit/org/apache/cassandra/service/accord/topology/AccordFastPathCoordinatorTest.java similarity index 98% rename from test/unit/org/apache/cassandra/service/accord/AccordFastPathCoordinatorTest.java rename to test/unit/org/apache/cassandra/service/accord/topology/AccordFastPathCoordinatorTest.java index 0a299406a060..5beb49ab4d54 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordFastPathCoordinatorTest.java +++ b/test/unit/org/apache/cassandra/service/accord/topology/AccordFastPathCoordinatorTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.topology; import java.util.ArrayList; import java.util.List; @@ -37,7 +37,7 @@ import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.schema.TableId; -import org.apache.cassandra.service.accord.AccordFastPath.Status; +import org.apache.cassandra.service.accord.topology.AccordFastPath.Status; import org.apache.cassandra.tcm.ClusterMetadata; import static org.apache.cassandra.service.accord.AccordTestUtils.id; diff --git a/test/unit/org/apache/cassandra/service/accord/AccordSyncPropagatorTest.java b/test/unit/org/apache/cassandra/service/accord/topology/AccordSyncPropagatorTest.java similarity index 99% rename from test/unit/org/apache/cassandra/service/accord/AccordSyncPropagatorTest.java rename to test/unit/org/apache/cassandra/service/accord/topology/AccordSyncPropagatorTest.java index 7c269138846c..4d00ffeb4a05 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordSyncPropagatorTest.java +++ b/test/unit/org/apache/cassandra/service/accord/topology/AccordSyncPropagatorTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord; +package org.apache.cassandra.service.accord.topology; import java.net.InetAddress; import java.net.UnknownHostException; @@ -79,6 +79,7 @@ import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessageDelivery; import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.tcm.ValidatingClusterMetadataService; import org.apache.cassandra.tcm.serialization.Version; import org.apache.cassandra.utils.AccordGenerators; diff --git a/test/unit/org/apache/cassandra/service/accord/fastpath/FastPathParsingTest.java b/test/unit/org/apache/cassandra/service/accord/topology/FastPathParsingTest.java similarity index 98% rename from test/unit/org/apache/cassandra/service/accord/fastpath/FastPathParsingTest.java rename to test/unit/org/apache/cassandra/service/accord/topology/FastPathParsingTest.java index 83f1bb6166b8..c434c35b716e 100644 --- a/test/unit/org/apache/cassandra/service/accord/fastpath/FastPathParsingTest.java +++ b/test/unit/org/apache/cassandra/service/accord/topology/FastPathParsingTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord.fastpath; +package org.apache.cassandra.service.accord.topology; import java.util.Arrays; import java.util.HashMap; diff --git a/test/unit/org/apache/cassandra/service/accord/fastpath/ParameterizedFastPathStrategyTest.java b/test/unit/org/apache/cassandra/service/accord/topology/ParameterizedFastPathStrategyTest.java similarity index 96% rename from test/unit/org/apache/cassandra/service/accord/fastpath/ParameterizedFastPathStrategyTest.java rename to test/unit/org/apache/cassandra/service/accord/topology/ParameterizedFastPathStrategyTest.java index 5142dbaf0fc0..51d2d7633af7 100644 --- a/test/unit/org/apache/cassandra/service/accord/fastpath/ParameterizedFastPathStrategyTest.java +++ b/test/unit/org/apache/cassandra/service/accord/topology/ParameterizedFastPathStrategyTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord.fastpath; +package org.apache.cassandra.service.accord.topology; import java.util.HashMap; import java.util.Map; @@ -30,13 +30,13 @@ import accord.utils.SortedArrays.SortedArrayList; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.service.accord.fastpath.ParameterizedFastPathStrategy.WeightedDc; +import org.apache.cassandra.service.accord.topology.ParameterizedFastPathStrategy.WeightedDc; import static java.util.Collections.emptySet; import static org.apache.cassandra.service.accord.AccordTestUtils.id; import static org.apache.cassandra.service.accord.AccordTestUtils.idList; import static org.apache.cassandra.service.accord.AccordTestUtils.idSet; -import static org.apache.cassandra.service.accord.fastpath.FastPathParsingTest.pfs; +import static org.apache.cassandra.service.accord.topology.FastPathParsingTest.pfs; import static org.junit.Assert.assertEquals; public class ParameterizedFastPathStrategyTest diff --git a/test/unit/org/apache/cassandra/service/accord/fastpath/SimpleFastPathStrategyTest.java b/test/unit/org/apache/cassandra/service/accord/topology/SimpleFastPathStrategyTest.java similarity index 96% rename from test/unit/org/apache/cassandra/service/accord/fastpath/SimpleFastPathStrategyTest.java rename to test/unit/org/apache/cassandra/service/accord/topology/SimpleFastPathStrategyTest.java index f19cac90ee45..293fa531a366 100644 --- a/test/unit/org/apache/cassandra/service/accord/fastpath/SimpleFastPathStrategyTest.java +++ b/test/unit/org/apache/cassandra/service/accord/topology/SimpleFastPathStrategyTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.service.accord.fastpath; +package org.apache.cassandra.service.accord.topology; import java.util.Collections; import java.util.Map; diff --git a/test/unit/org/apache/cassandra/tcm/ClusterMetadataSerializerTest.java b/test/unit/org/apache/cassandra/tcm/ClusterMetadataSerializerTest.java index 2f2b986e5716..e5446df1734a 100644 --- a/test/unit/org/apache/cassandra/tcm/ClusterMetadataSerializerTest.java +++ b/test/unit/org/apache/cassandra/tcm/ClusterMetadataSerializerTest.java @@ -26,8 +26,8 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.DataOutputBuffer; -import org.apache.cassandra.service.accord.AccordFastPath; -import org.apache.cassandra.service.accord.AccordStaleReplicas; +import org.apache.cassandra.service.accord.topology.AccordFastPath; +import org.apache.cassandra.service.accord.topology.AccordStaleReplicas; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; import org.apache.cassandra.tcm.membership.NodeVersion; import org.apache.cassandra.tcm.serialization.AsymmetricMetadataSerializers; diff --git a/test/unit/org/apache/cassandra/utils/AccordGenerators.java b/test/unit/org/apache/cassandra/utils/AccordGenerators.java index 30925f00a3dd..601e16f939b4 100644 --- a/test/unit/org/apache/cassandra/utils/AccordGenerators.java +++ b/test/unit/org/apache/cassandra/utils/AccordGenerators.java @@ -83,11 +83,11 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.accord.AccordTestUtils; -import org.apache.cassandra.service.accord.FetchTopologies; import org.apache.cassandra.service.accord.TokenRange; import org.apache.cassandra.service.accord.api.PartitionKey; import org.apache.cassandra.service.accord.api.TokenKey; import org.apache.cassandra.service.accord.serializers.TableMetadatas; +import org.apache.cassandra.service.accord.topology.FetchTopologies; import org.apache.cassandra.service.accord.txn.TxnData; import org.apache.cassandra.service.accord.txn.TxnWrite; diff --git a/test/unit/org/apache/cassandra/utils/CassandraGenerators.java b/test/unit/org/apache/cassandra/utils/CassandraGenerators.java index 36ce78e21a73..3322b1eccf8e 100644 --- a/test/unit/org/apache/cassandra/utils/CassandraGenerators.java +++ b/test/unit/org/apache/cassandra/utils/CassandraGenerators.java @@ -131,12 +131,12 @@ import org.apache.cassandra.schema.Types; import org.apache.cassandra.schema.UserFunctions; import org.apache.cassandra.schema.Views; -import org.apache.cassandra.service.accord.AccordFastPath; -import org.apache.cassandra.service.accord.AccordStaleReplicas; -import org.apache.cassandra.service.accord.fastpath.FastPathStrategy; -import org.apache.cassandra.service.accord.fastpath.InheritKeyspaceFastPathStrategy; -import org.apache.cassandra.service.accord.fastpath.ParameterizedFastPathStrategy; -import org.apache.cassandra.service.accord.fastpath.SimpleFastPathStrategy; +import org.apache.cassandra.service.accord.topology.AccordFastPath; +import org.apache.cassandra.service.accord.topology.AccordStaleReplicas; +import org.apache.cassandra.service.accord.topology.FastPathStrategy; +import org.apache.cassandra.service.accord.topology.InheritKeyspaceFastPathStrategy; +import org.apache.cassandra.service.accord.topology.ParameterizedFastPathStrategy; +import org.apache.cassandra.service.accord.topology.SimpleFastPathStrategy; import org.apache.cassandra.service.consensus.TransactionalMode; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; import org.apache.cassandra.tcm.ClusterMetadata; diff --git a/test/unit/org/apache/cassandra/utils/SimpleBitSetSerializersTest.java b/test/unit/org/apache/cassandra/utils/SimpleBitSetSerializersTest.java index fee72a011504..9c92c7f1f0d8 100644 --- a/test/unit/org/apache/cassandra/utils/SimpleBitSetSerializersTest.java +++ b/test/unit/org/apache/cassandra/utils/SimpleBitSetSerializersTest.java @@ -53,7 +53,7 @@ public void any() { @SuppressWarnings({ "resource", "IOResourceOpenedButNotSafelyClosed" }) DataOutputBuffer output = new DataOutputBuffer(); qt().forAll(anyGen()).check(bits -> { - Serializers.testSerde(output, SimpleBitSetSerializers.any, bits, (actual, expected) -> { + Serializers.testSerde(output, SimpleBitSetSerializers.any, bits, (expected, actual) -> { if (actual.getClass() == expected.getClass()) { Assertions.assertThat(actual)