From 754574855b726436eabc0a8200a1e43acdfb6387 Mon Sep 17 00:00:00 2001 From: Blake Eggleston Date: Fri, 31 Oct 2025 11:21:34 -0700 Subject: [PATCH] CEP-45: Replication type migration --- .../statements/ModificationStatement.java | 24 +- .../db/AbstractMutationVerbHandler.java | 41 ++ .../db/CassandraKeyspaceWriteHandler.java | 2 + .../cassandra/db/ColumnFamilyStore.java | 7 +- .../org/apache/cassandra/db/Keyspace.java | 12 +- .../org/apache/cassandra/db/Mutation.java | 14 + .../db/partitions/PartitionUpdate.java | 5 + .../tracked/TrackedKeyspaceWriteHandler.java | 3 + src/java/org/apache/cassandra/hints/Hint.java | 13 +- .../cassandra/hints/HintsDispatcher.java | 10 +- .../apache/cassandra/journal/Segments.java | 6 +- .../apache/cassandra/metrics/TCMMetrics.java | 2 + .../cassandra/repair/AbstractRepairTask.java | 1 + .../cassandra/repair/RepairCoordinator.java | 15 +- .../apache/cassandra/repair/RepairJob.java | 4 +- .../repair/RepairMessageVerbHandler.java | 23 + .../apache/cassandra/repair/RepairResult.java | 5 +- .../cassandra/repair/RepairSession.java | 10 +- .../repair/autorepair/AutoRepairState.java | 3 +- .../repair/messages/PrepareMessage.java | 20 +- .../replication/ActiveLogReconciler.java | 41 +- .../MutableCoordinatorLogOffsets.java | 6 + .../replication/MutationTrackingService.java | 12 +- .../NonBlockingCoordinatorLogOffsets.java | 2 + .../replication/PushMutationRequest.java | 27 + .../service/ActiveRepairService.java | 16 +- .../cassandra/service/StorageProxy.java | 124 +--- .../cassandra/service/StorageService.java | 2 +- .../ConsensusMigrationMutationHelper.java | 28 +- .../service/reads/AbstractReadExecutor.java | 11 +- .../cassandra/service/reads/ReadExecutor.java | 79 ++ .../service/reads/TrackedReadExecutor.java | 84 +++ .../reads/range/RangeCommandIterator.java | 120 ++- .../service/reads/tracked/TrackedRead.java | 5 + .../migration/KeyspaceMigrationInfo.java | 305 ++++++++ .../migration/MigrationRouter.java | 392 ++++++++++ ...MutationTrackingMigrationRepairResult.java | 47 ++ .../MutationTrackingMigrationState.java | 361 +++++++++ .../MutationTrackingRepairHandler.java | 114 +++ .../apache/cassandra/tcm/ClusterMetadata.java | 48 ++ .../cassandra/tcm/ClusterMetadataService.java | 2 + .../apache/cassandra/tcm/MetadataKeys.java | 2 + .../tcm/StubClusterMetadataService.java | 2 + .../apache/cassandra/tcm/Transformation.java | 2 + .../tcm/compatibility/GossipHelper.java | 4 + .../cassandra/tcm/serialization/Version.java | 1 + .../AdvanceMutationTrackingMigration.java | 144 ++++ .../tcm/transformations/AlterSchema.java | 65 +- .../test/MutationTrackingMigrationTest.java | 583 +++++++++++++++ .../AccordMigrationReadRaceTestBase.java | 2 +- .../AccordMigrationWriteRaceTestBase.java | 2 +- .../test/log/ClusterMetadataTestHelper.java | 2 + .../org/apache/cassandra/cql3/CQLTester.java | 2 +- ...ckingMutationVerbHandlerMigrationTest.java | 225 ++++++ .../db/virtual/LocalRepairTablesTest.java | 3 +- .../cassandra/journal/SegmentsTest.java | 106 +++ .../cassandra/locator/MetaStrategyTest.java | 2 + .../apache/cassandra/repair/FuzzTestBase.java | 3 +- .../cassandra/repair/RepairJobTest.java | 7 +- ...epairMessageVerbHandlerOutOfRangeTest.java | 3 +- .../cassandra/repair/RepairSessionTest.java | 3 +- .../RepairMessageSerializationsTest.java | 3 +- .../service/ActiveRepairServiceTest.java | 3 +- .../migration/KeyspaceMigrationInfoTest.java | 407 +++++++++++ .../migration/MigrationRouterTest.java | 687 ++++++++++++++++++ .../MutationTrackingMigrationStateTest.java | 291 ++++++++ .../ClusterMetadataTransformationTest.java | 2 + .../AdvanceMutationTrackingMigrationTest.java | 325 +++++++++ .../AlterSchemaMutationTrackingTest.java | 386 ++++++++++ .../cassandra/utils/CassandraGenerators.java | 4 +- 70 files changed, 5071 insertions(+), 246 deletions(-) create mode 100644 src/java/org/apache/cassandra/service/reads/ReadExecutor.java create mode 100644 src/java/org/apache/cassandra/service/reads/TrackedReadExecutor.java create mode 100644 src/java/org/apache/cassandra/service/replication/migration/KeyspaceMigrationInfo.java create mode 100644 src/java/org/apache/cassandra/service/replication/migration/MigrationRouter.java create mode 100644 src/java/org/apache/cassandra/service/replication/migration/MutationTrackingMigrationRepairResult.java create mode 100644 src/java/org/apache/cassandra/service/replication/migration/MutationTrackingMigrationState.java create mode 100644 src/java/org/apache/cassandra/service/replication/migration/MutationTrackingRepairHandler.java create mode 100644 src/java/org/apache/cassandra/tcm/transformations/AdvanceMutationTrackingMigration.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/MutationTrackingMigrationTest.java create mode 100644 test/unit/org/apache/cassandra/db/MutationTrackingMutationVerbHandlerMigrationTest.java create mode 100644 test/unit/org/apache/cassandra/service/replication/migration/KeyspaceMigrationInfoTest.java create mode 100644 test/unit/org/apache/cassandra/service/replication/migration/MigrationRouterTest.java create mode 100644 test/unit/org/apache/cassandra/service/replication/migration/MutationTrackingMigrationStateTest.java create mode 100644 test/unit/org/apache/cassandra/tcm/transformations/AdvanceMutationTrackingMigrationTest.java create mode 100644 test/unit/org/apache/cassandra/tcm/transformations/AlterSchemaMutationTrackingTest.java diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java index b2d916a2de78..2cd78a1d8a28 100644 --- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java @@ -73,7 +73,6 @@ import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.IMutation; import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.Mutation; import org.apache.cassandra.db.ReadCommand.PotentialTxnConflicts; import org.apache.cassandra.db.ReadExecutionController; import org.apache.cassandra.db.RegularAndStaticColumns; @@ -121,6 +120,7 @@ import org.apache.cassandra.service.accord.txn.TxnReferenceOperation; import org.apache.cassandra.service.accord.txn.TxnReferenceOperations; import org.apache.cassandra.service.accord.txn.TxnWrite; +import org.apache.cassandra.service.replication.migration.MigrationRouter; import org.apache.cassandra.service.disk.usage.DiskUsageBroadcaster; import org.apache.cassandra.service.paxos.Ballot; import org.apache.cassandra.service.paxos.BallotGenerator; @@ -817,27 +817,29 @@ public ResultMessage executeInternalWithoutCondition(QueryState queryState, Quer long timestamp = options.getTimestamp(queryState); long nowInSeconds = options.getNowInSeconds(queryState); List mutations = getMutations(queryState.getClientState(), options, true, timestamp, nowInSeconds, requestTime); - boolean isTracked = !mutations.isEmpty() && Schema.instance.getKeyspaceMetadata(mutations.get(0).getKeyspaceName()).params.replicationType.isTracked(); - if (isTracked) + + MigrationRouter.RoutedMutations routed = MigrationRouter.routeMutations(mutations); + + if (!routed.trackedMutations.isEmpty()) { - if (mutations.stream().anyMatch(m -> m instanceof CounterMutation)) + if (routed.trackedMutations.stream().anyMatch(m -> m instanceof CounterMutation)) throw new InvalidRequestException("Mutation tracking is currently unsupported with counters"); - if (mutations.size() > 1) - throw new InvalidRequestException("Mutation tracking is currently unsupported with unlogged batches"); - - Mutation mutation = (Mutation) mutations.get(0); + } + for (IMutation mutation : routed.trackedMutations) + { String keyspaceName = mutation.getKeyspaceName(); Token token = mutation.key().getToken(); MutationId id = MutationTrackingService.instance.nextMutationId(keyspaceName, token); mutation = mutation.withMutationId(id); mutation.apply(); } - else + + for (IMutation mutation : routed.untrackedMutations) { - for (IMutation mutation : mutations) - mutation.apply(); + mutation.apply(); } + return null; } diff --git a/src/java/org/apache/cassandra/db/AbstractMutationVerbHandler.java b/src/java/org/apache/cassandra/db/AbstractMutationVerbHandler.java index 05512e5a8e3f..e83d70bbc189 100644 --- a/src/java/org/apache/cassandra/db/AbstractMutationVerbHandler.java +++ b/src/java/org/apache/cassandra/db/AbstractMutationVerbHandler.java @@ -35,6 +35,8 @@ import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.replication.migration.MigrationRouter; +import org.apache.cassandra.service.replication.migration.MigrationRouter.MutationRouting; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.Epoch; @@ -61,6 +63,7 @@ protected void processMessage(Message message, InetAddressAndPort respondTo) ClusterMetadata metadata = ClusterMetadata.current(); metadata = checkTokenOwnership(metadata, message, respondTo); metadata = checkSchemaVersion(metadata, message, respondTo); + metadata = checkReplicationMigration(metadata, message, respondTo); } try @@ -193,6 +196,44 @@ else if (message.epoch().isBefore(metadata.schema.lastModified())) return metadata; } + /** + * Confirm that the presence/absence of a mutation id matches our expectations for the given keyspace/table/token. If + * it doesn't, then we're not on the same epoch as the coordinator, or there's a bug. + */ + private ClusterMetadata checkReplicationMigration(ClusterMetadata metadata, Message message, InetAddressAndPort respondTo) + { + IMutation mutation = message.payload; + MutationRouting expected = mutation.id().isNone() ? MutationRouting.UNTRACKED : MutationRouting.TRACKED; + if (expected == MigrationRouter.getMutationRouting(metadata, mutation)) + return metadata; + + if (message.epoch().isAfter(metadata.epoch)) + { + // coordinator is ahead, fetch log and recheck + metadata = ClusterMetadataService.instance().fetchLogFromPeerOrCMS(metadata, respondTo, message.epoch()); + + // recheck, we may now be ahead of the coordinator + return checkReplicationMigration(metadata, message, respondTo); + } + else if (message.epoch().isBefore(metadata.epoch)) + { + TCMMetrics.instance.coordinatorBehindReplication.mark(); + throw new CoordinatorBehindException(String.format("Replication type / migration mismatch for keyspace: %s token %s, coordinator: %s is behind, our epoch = %s, their epoch = %s", + mutation.getKeyspaceName(), + mutation.key(), + respondTo, + metadata.epoch, message.epoch())); + } + else + { + // same epoch but different routing should not be possible + throw new IllegalStateException(String.format("Inconsistent mutation routing at epoch = %s. Keyspace: %s key: %s ", + metadata.epoch, + mutation.getKeyspaceName(), + mutation.key())); + } + } + private static VersionedEndpoints.ForToken writePlacements(ClusterMetadata metadata, String keyspace, DecoratedKey key) { return metadata.placements.get(metadata.schema.getKeyspace(keyspace).getMetadata().params.replication).writes.forToken(key.getToken()); diff --git a/src/java/org/apache/cassandra/db/CassandraKeyspaceWriteHandler.java b/src/java/org/apache/cassandra/db/CassandraKeyspaceWriteHandler.java index 825ef12292cd..9b68acc708e1 100644 --- a/src/java/org/apache/cassandra/db/CassandraKeyspaceWriteHandler.java +++ b/src/java/org/apache/cassandra/db/CassandraKeyspaceWriteHandler.java @@ -26,6 +26,7 @@ import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.exceptions.RequestExecutionException; import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.service.replication.migration.MigrationRouter; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.concurrent.OpOrder; @@ -46,6 +47,7 @@ public WriteContext beginWrite(Mutation mutation, boolean makeDurable) throws Re { group = Keyspace.writeOrder.start(); + MigrationRouter.validateUntrackedMutation(mutation); // write the mutation to the commitlog and memtables CommitLogPosition position = null; if (makeDurable) diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index b8b77bdf9a76..f4631e689a54 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -1159,10 +1159,9 @@ public CommitLogPosition call() CommitLogPosition commitLogLowerBound = mainMemtable.getCommitLogLowerBound(); commitLogUpperBound = mainMemtable.getFinalCommitLogUpperBound(); TableMetadata metadata = metadata(); - if (metadata.replicationType().isTracked()) - MutationJournal.instance.notifyFlushed(metadata.id, commitLogLowerBound, commitLogUpperBound); - else - CommitLog.instance.discardCompletedSegments(metadata.id, commitLogLowerBound, commitLogUpperBound); + + MutationJournal.instance.notifyFlushed(metadata.id, commitLogLowerBound, commitLogUpperBound); + CommitLog.instance.discardCompletedSegments(metadata.id, commitLogLowerBound, commitLogUpperBound); } metric.pendingFlushes.dec(); diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java index e7132703f769..a527d0e105d9 100644 --- a/src/java/org/apache/cassandra/db/Keyspace.java +++ b/src/java/org/apache/cassandra/db/Keyspace.java @@ -65,6 +65,7 @@ import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationMutationHelper; +import org.apache.cassandra.service.replication.migration.MigrationRouter; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.ByteBufferUtil; @@ -403,9 +404,10 @@ public void initCf(TableMetadata metadata, boolean loadSSTables) public Future applyFuture(Mutation mutation, boolean writeCommitLog, boolean updateIndexes) { - return getMetadata().useMutationTracking() - ? applyInternalTracked(mutation, new AsyncPromise<>()) - : applyInternal(mutation, writeCommitLog, updateIndexes, true, true, new AsyncPromise<>()); + if (mutation.id().isNone()) + return applyInternal(mutation, writeCommitLog, updateIndexes, true, true, new AsyncPromise<>()); + else + return applyInternalTracked(mutation, new AsyncPromise<>()); } public void apply(Mutation mutation, boolean writeCommitLog, boolean updateIndexes) @@ -435,7 +437,7 @@ public void apply(final Mutation mutation, boolean updateIndexes, boolean isDroppable) { - if (getMetadata().useMutationTracking()) + if (MigrationRouter.isFullyTracked(mutation)) applyInternalTracked(mutation, null); else applyInternal(mutation, makeDurable, updateIndexes, isDroppable, false, null); @@ -612,7 +614,7 @@ else if (isDeferrable) */ private Future applyInternalTracked(Mutation mutation, Promise future) { - Preconditions.checkState(getMetadata().useMutationTracking() && !mutation.id().isNone()); + Preconditions.checkState(MigrationRouter.isFullyTracked(mutation) && !mutation.id().isNone()); ClusterMetadata cm = ClusterMetadata.current(); if (TEST_FAIL_WRITES && getMetadata().name.equals(TEST_FAIL_WRITES_KS)) diff --git a/src/java/org/apache/cassandra/db/Mutation.java b/src/java/org/apache/cassandra/db/Mutation.java index afe5da091a10..94bceb43d938 100644 --- a/src/java/org/apache/cassandra/db/Mutation.java +++ b/src/java/org/apache/cassandra/db/Mutation.java @@ -681,6 +681,20 @@ public Pair deserializeKeyAndTableMetadata(DataInpu return PartitionUpdate.serializer.deserializeMetadataAndKey(in, version, flag); } + public TableId deserializeTableId(DataInputBuffer in, int version, DeserializationHelper.Flag flag) throws IOException + { + if (version >= VERSION_51) + in.skipBytes(1); // flags + + if (version >= VERSION_52) + MutationId.serializer.skip(in, version); + + int size = in.readUnsignedVInt32(); + assert size > 0; + + return PartitionUpdate.serializer.deserializeTableId(in, version, flag); + } + public Mutation deserialize(DataInputPlus in, int version) throws IOException { return deserialize(in, version, DeserializationHelper.Flag.FROM_REMOTE); diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java index 3b19b82bf875..933da0eab4a2 100644 --- a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java +++ b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java @@ -834,6 +834,11 @@ public Pair deserializeMetadataAndKey(DataInputBuff return Pair.create(header.key, tableMetadata); } + public TableId deserializeTableId(DataInputBuffer in, int version, DeserializationHelper.Flag flag) throws IOException + { + return TableId.deserialize(in); + } + public PartitionUpdate deserialize(PartitionKey key, TableMetadatas tables, DataInputPlus in, int version, DeserializationHelper.Flag flag) throws IOException { TableMetadata tableMetadata = tables.deserialize(in); diff --git a/src/java/org/apache/cassandra/db/tracked/TrackedKeyspaceWriteHandler.java b/src/java/org/apache/cassandra/db/tracked/TrackedKeyspaceWriteHandler.java index 8c06e3c30884..50470f4aff0b 100644 --- a/src/java/org/apache/cassandra/db/tracked/TrackedKeyspaceWriteHandler.java +++ b/src/java/org/apache/cassandra/db/tracked/TrackedKeyspaceWriteHandler.java @@ -25,6 +25,7 @@ import org.apache.cassandra.db.commitlog.CommitLogPosition; import org.apache.cassandra.exceptions.RequestExecutionException; import org.apache.cassandra.replication.MutationJournal; +import org.apache.cassandra.service.replication.migration.MigrationRouter; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.concurrent.OpOrder; @@ -38,6 +39,8 @@ public WriteContext beginWrite(Mutation mutation, boolean makeDurable) throws Re { group = Keyspace.writeOrder.start(); + MigrationRouter.validateTrackedMutation(mutation); + Tracing.trace("Appending to mutation journal"); CommitLogPosition pointer = MutationJournal.instance.write(mutation.id(), mutation); diff --git a/src/java/org/apache/cassandra/hints/Hint.java b/src/java/org/apache/cassandra/hints/Hint.java index 886e781ee638..d1cf71d9afbe 100644 --- a/src/java/org/apache/cassandra/hints/Hint.java +++ b/src/java/org/apache/cassandra/hints/Hint.java @@ -33,6 +33,8 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.service.replication.migration.MigrationRouter; +import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.concurrent.Future; import org.apache.cassandra.utils.concurrent.ImmediateFuture; import org.apache.cassandra.utils.vint.VIntCoding; @@ -99,13 +101,22 @@ Future applyFuture() { if (isLive()) { + ClusterMetadata cm = ClusterMetadata.current(); // filter out partition update for tables that have been truncated since hint's creation Mutation filtered = mutation; for (TableId id : mutation.getTableIds()) + { + if (MigrationRouter.shouldUseTrackedForWrites(cm, mutation.getKeyspaceName(), id, mutation.key().getToken())) + { + filtered = filtered.without(id); + continue; + } + if (creationTime <= SystemKeyspace.getTruncatedAt(id)) filtered = filtered.without(id); + } - if (!filtered.isEmpty()) + if (filtered != null && !filtered.isEmpty()) return filtered.applyFuture(); } diff --git a/src/java/org/apache/cassandra/hints/HintsDispatcher.java b/src/java/org/apache/cassandra/hints/HintsDispatcher.java index c15b46eaea4f..950d3911eb25 100644 --- a/src/java/org/apache/cassandra/hints/HintsDispatcher.java +++ b/src/java/org/apache/cassandra/hints/HintsDispatcher.java @@ -341,7 +341,9 @@ private Action sendHints(Iterator hints, @Nullable BitSet hintsFilter, Co if (hintsFilter != null && !hintsFilter.get(hintIndex)) continue; - callbacks.add(sendFunction.apply(hint)); + Callback callback = sendFunction.apply(hint); + if (callback != null) + callbacks.add(callback); } return Action.CONTINUE; } @@ -417,7 +419,11 @@ private SplitHint splitHintIntoAccordAndNormal(ClusterMetadata cm, Hint hint) { SplitMutation splitMutation = ConsensusMigrationMutationHelper.instance().splitMutation(hint.mutation, cm); if (splitMutation.trackedMutation != null) - throw new IllegalStateException("Cannot generate hints for tracked mutations"); + { + logger.debug("Discarding tracked component of hint"); + if (splitMutation.accordMutation == null && splitMutation.untrackedMutation == null) + return new SplitHint(null, null); + } if (splitMutation.accordMutation == null) return new SplitHint(null, hint); if (splitMutation.untrackedMutation == null) diff --git a/src/java/org/apache/cassandra/journal/Segments.java b/src/java/org/apache/cassandra/journal/Segments.java index dae1bbeaacf3..9fc20931648e 100644 --- a/src/java/org/apache/cassandra/journal/Segments.java +++ b/src/java/org/apache/cassandra/journal/Segments.java @@ -157,6 +157,10 @@ void select(long minTimestamp, long maxTimestamp, Collection> into } } + /** + * Find index of first segment with timestamp >= given timestamp. + * Returns sorted.size() if timestamp greater than all segments. + */ int findIdxFor(long timestamp) { List> sorted = allSorted(true); @@ -172,7 +176,7 @@ else if (res == 0) else high = mid - 1; } - throw new IllegalStateException(String.format("Could not find a segment with timestamp %d among %s", timestamp, sorted)); + return low; } boolean isSwitched(ActiveSegment active) diff --git a/src/java/org/apache/cassandra/metrics/TCMMetrics.java b/src/java/org/apache/cassandra/metrics/TCMMetrics.java index 909280c5771e..80019e916c28 100644 --- a/src/java/org/apache/cassandra/metrics/TCMMetrics.java +++ b/src/java/org/apache/cassandra/metrics/TCMMetrics.java @@ -65,6 +65,7 @@ public class TCMMetrics public final Meter progressBarrierCLRelax; public final Meter coordinatorBehindSchema; public final Meter coordinatorBehindPlacements; + public final Meter coordinatorBehindReplication; public final Gauge epochAwareDebounceTrackerSize; public final Meter reconstructLogStateCall; @@ -128,6 +129,7 @@ private TCMMetrics() coordinatorBehindSchema = Metrics.meter(factory.createMetricName("CoordinatorBehindSchema")); coordinatorBehindPlacements = Metrics.meter(factory.createMetricName("CoordinatorBehindPlacements")); + coordinatorBehindReplication = Metrics.meter(factory.createMetricName("CoordinatorBehindReplication")); reconstructLogStateCall = Metrics.meter(factory.createMetricName("ReconstructLogStateCall")); } diff --git a/src/java/org/apache/cassandra/repair/AbstractRepairTask.java b/src/java/org/apache/cassandra/repair/AbstractRepairTask.java index ff3d5e6f9cbf..d6396bd2c12b 100644 --- a/src/java/org/apache/cassandra/repair/AbstractRepairTask.java +++ b/src/java/org/apache/cassandra/repair/AbstractRepairTask.java @@ -81,6 +81,7 @@ private List submitRepairSessions(TimeUUID parentSession, options.repairAccord(), executor, validationScheduler, + coordinator.minEpoch, cfnames); if (session == null) continue; diff --git a/src/java/org/apache/cassandra/repair/RepairCoordinator.java b/src/java/org/apache/cassandra/repair/RepairCoordinator.java index fa2036b02f13..b511d081c984 100644 --- a/src/java/org/apache/cassandra/repair/RepairCoordinator.java +++ b/src/java/org/apache/cassandra/repair/RepairCoordinator.java @@ -75,6 +75,7 @@ import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tracing.TraceKeyspace; import org.apache.cassandra.tracing.TraceState; import org.apache.cassandra.tracing.Tracing; @@ -108,25 +109,27 @@ public class RepairCoordinator implements Runnable, ProgressEventNotifier, Repai private final List listeners = new ArrayList<>(); private final AtomicReference firstError = new AtomicReference<>(null); + public final Epoch minEpoch; final SharedContext ctx; final Scheduler validationScheduler; private TraceState traceState; - public RepairCoordinator(StorageService storageService, int cmd, RepairOption options, String keyspace) + public RepairCoordinator(StorageService storageService, int cmd, RepairOption options, String keyspace, Epoch minEpoch) { this(SharedContext.Global.instance, - (ks, tables) -> storageService.getValidColumnFamilies(false, false, ks, tables), - storageService::getLocalReplicas, - cmd, options, keyspace); + (ks, tables) -> storageService.getValidColumnFamilies(false, false, ks, tables), + storageService::getLocalReplicas, + cmd, options, keyspace, minEpoch); } RepairCoordinator(SharedContext ctx, BiFunction> validColumnFamilies, Function getLocalReplicas, - int cmd, RepairOption options, String keyspace) + int cmd, RepairOption options, String keyspace, Epoch minEpoch) { this.ctx = ctx; + this.minEpoch = minEpoch; this.validationScheduler = Scheduler.build(DatabaseDescriptor.getConcurrentMerkleTreeRequests()); this.state = new CoordinatorState(ctx, cmd, keyspace, options); this.tag = "repair:" + cmd; @@ -483,7 +486,7 @@ private Future prepare(List columnFamilies, Set { timer.update(ctx.clock().nanoTime() - startNanos, TimeUnit.NANOSECONDS); state.phase.prepareComplete(); diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java index 4bebca17e44d..28640a60a4d2 100644 --- a/src/java/org/apache/cassandra/repair/RepairJob.java +++ b/src/java/org/apache/cassandra/repair/RepairJob.java @@ -56,6 +56,7 @@ import org.apache.cassandra.service.accord.repair.AccordRepair; import org.apache.cassandra.service.accord.repair.AccordRepair.AccordRepairResult; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationRepairResult; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationRepairResult; import org.apache.cassandra.service.paxos.cleanup.PaxosCleanup; import org.apache.cassandra.service.paxos.cleanup.PaxosUpdateLowBallot; import org.apache.cassandra.streaming.PreviewKind; @@ -278,7 +279,8 @@ public void onSuccess(List stats) cfs.metric.repairsCompleted.inc(); logger.info("Completing repair with excludedDeadNodes {}", session.excludedDeadNodes); ConsensusMigrationRepairResult cmrs = ConsensusMigrationRepairResult.fromRepair(repairStartingEpoch, getUnchecked(accordRepair), session.repairData, doPaxosRepair, doAccordRepair, session.excludedDeadNodes, session.isIncremental); - trySuccess(new RepairResult(desc, stats, cmrs)); + MutationTrackingMigrationRepairResult mtmrs = MutationTrackingMigrationRepairResult.fromRepair(repairStartingEpoch, session.excludedDeadNodes, session.previewKind.isPreview()); + trySuccess(new RepairResult(desc, stats, cmrs, mtmrs)); } /** diff --git a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java index ea9742609fed..200c246fb30c 100644 --- a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java +++ b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.concurrent.TimeoutException; import java.util.function.BiFunction; import java.util.function.Function; @@ -27,6 +28,7 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.IVerbHandler; import org.apache.cassandra.net.Message; @@ -48,8 +50,10 @@ import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.TimeUUID; +import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; /** * Handles all repair related message. @@ -108,6 +112,25 @@ public void doVerb(final Message message) case PREPARE_MSG: { PrepareMessage prepareMessage = (PrepareMessage) message.payload; + + // Ensure repair participant sees same migration state as coordinator. This prevents an unlikely data + // resurrection issue when migrating from tracked to untracked replication when combined with the + // ActiveLogReconciler logic that stops forwarding mutations when a keyspaces starts migration to + // untracked replication + try + { + ClusterMetadataService.instance().awaitAtLeast(prepareMessage.minEpoch); + Keyspace.writeOrder.awaitNewBarrier(); + } + catch (InterruptedException e) + { + throw new UncheckedInterruptedException(e); + } + catch (TimeoutException e) + { + throw new RuntimeException(e); + } + logger.debug("Preparing, {}", prepareMessage); ParticipateState state = new ParticipateState(ctx.clock(), message.from(), prepareMessage); if (!ctx.repair().register(state)) diff --git a/src/java/org/apache/cassandra/repair/RepairResult.java b/src/java/org/apache/cassandra/repair/RepairResult.java index 6c04f6be0760..d3a8a71d40f0 100644 --- a/src/java/org/apache/cassandra/repair/RepairResult.java +++ b/src/java/org/apache/cassandra/repair/RepairResult.java @@ -20,6 +20,7 @@ import java.util.List; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationRepairResult; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationRepairResult; /** * RepairJob's result @@ -29,11 +30,13 @@ public class RepairResult public final RepairJobDesc desc; public final List stats; public final ConsensusMigrationRepairResult consensusMigrationRepairResult; + public final MutationTrackingMigrationRepairResult mutationTrackingMigrationRepairResult; - public RepairResult(RepairJobDesc desc, List stats, ConsensusMigrationRepairResult consensusMigrationRepairResult) + public RepairResult(RepairJobDesc desc, List stats, ConsensusMigrationRepairResult consensusMigrationRepairResult, MutationTrackingMigrationRepairResult mutationTrackingMigrationRepairResult) { this.desc = desc; this.stats = stats; this.consensusMigrationRepairResult = consensusMigrationRepairResult; + this.mutationTrackingMigrationRepairResult = mutationTrackingMigrationRepairResult; } } diff --git a/src/java/org/apache/cassandra/repair/RepairSession.java b/src/java/org/apache/cassandra/repair/RepairSession.java index 3285fec9dbc3..106950bb4afd 100644 --- a/src/java/org/apache/cassandra/repair/RepairSession.java +++ b/src/java/org/apache/cassandra/repair/RepairSession.java @@ -60,7 +60,9 @@ import org.apache.cassandra.schema.SystemDistributedKeyspace; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.consensus.migration.ConsensusTableMigration; +import org.apache.cassandra.service.replication.migration.MutationTrackingRepairHandler; import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.MerkleTrees; @@ -125,6 +127,7 @@ public class RepairSession extends AsyncFuture implements I public final boolean repairAccord; public final boolean dontPurgeTombstones; public final boolean excludedDeadNodes; + public final Epoch minEpoch; private final AtomicBoolean isFailed = new AtomicBoolean(false); @@ -169,6 +172,7 @@ public RepairSession(SharedContext ctx, boolean repairPaxos, boolean dontPurgeTombstones, boolean repairAccord, + Epoch minEpoch, String... cfnames) { this.ctx = ctx; @@ -176,6 +180,7 @@ public RepairSession(SharedContext ctx, this.repairData = repairData; this.repairPaxos = repairPaxos; this.repairAccord = repairAccord; + this.minEpoch = minEpoch; assert cfnames.length > 0 : "Repairing no column families seems pointless, doesn't it"; this.state = new SessionState(ctx, parentRepairSession, keyspace, cfnames, commonRange); this.parallelismDegree = parallelismDegree; @@ -350,8 +355,11 @@ public void start(ExecutorPlus executor) for (String cfname : state.cfnames) { RepairJob job = new RepairJob(this, cfname); - // Repairs can drive forward progress for consensus migration so always check + + // Repairs can advance progress for consensus and mutation tracking migration so always check job.addCallback(ConsensusTableMigration.completedRepairJobHandler); + job.addCallback(MutationTrackingRepairHandler.completedRepairJobHandler); + state.register(job.state); executor.execute(job); jobs.add(job); diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java index 6822f20cf023..a5c4fdc2c16f 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java @@ -35,6 +35,7 @@ import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.Clock; import org.slf4j.Logger; @@ -98,7 +99,7 @@ protected AutoRepairState(RepairType repairType) protected RepairCoordinator getRepairRunnable(String keyspace, RepairOption options) { return new RepairCoordinator(StorageService.instance, StorageService.nextRepairCommand.incrementAndGet(), - options, keyspace); + options, keyspace, ClusterMetadata.current().epoch); } public long getLastRepairTime() diff --git a/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java b/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java index 81ae098349c3..2ee6b58ceb74 100644 --- a/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java +++ b/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java @@ -35,6 +35,7 @@ import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.TimeUUID; @@ -50,8 +51,9 @@ public class PrepareMessage extends RepairMessage public final long repairedAt; public final boolean isGlobal; public final PreviewKind previewKind; + public final Epoch minEpoch; - public PrepareMessage(TimeUUID parentRepairSession, List tableIds, IPartitioner partitioner, Collection> ranges, boolean isIncremental, long repairedAt, boolean isGlobal, PreviewKind previewKind) + public PrepareMessage(TimeUUID parentRepairSession, List tableIds, IPartitioner partitioner, Collection> ranges, boolean isIncremental, long repairedAt, boolean isGlobal, PreviewKind previewKind, Epoch minEpoch) { super(null); this.parentRepairSession = parentRepairSession; @@ -62,6 +64,7 @@ public PrepareMessage(TimeUUID parentRepairSession, List tableIds, IPar this.repairedAt = repairedAt; this.isGlobal = isGlobal; this.previewKind = previewKind; + this.minEpoch = minEpoch; } @Override @@ -83,13 +86,14 @@ public boolean equals(Object o) repairedAt == other.repairedAt && tableIds.equals(other.tableIds) && partitioner.getClass().equals(other.partitioner.getClass()) && - ranges.equals(other.ranges); + ranges.equals(other.ranges) && + minEpoch.equals(other.minEpoch); } @Override public int hashCode() { - return Objects.hash(parentRepairSession, isGlobal, previewKind, isIncremental, repairedAt, tableIds, ranges, partitioner); + return Objects.hash(parentRepairSession, isGlobal, previewKind, isIncremental, repairedAt, tableIds, ranges, partitioner, minEpoch); } private static final String MIXED_MODE_ERROR = "Some nodes involved in repair are on an incompatible major version. " + @@ -117,6 +121,8 @@ public void serialize(PrepareMessage message, DataOutputPlus out, int version) t out.writeLong(message.repairedAt); out.writeBoolean(message.isGlobal); out.writeInt(message.previewKind.getSerializationVal()); + if (version >= MessagingService.VERSION_52) + Epoch.messageSerializer.serialize(message.minEpoch, out, version); } public PrepareMessage deserialize(DataInputPlus in, int version) throws IOException @@ -139,7 +145,10 @@ public PrepareMessage deserialize(DataInputPlus in, int version) throws IOExcept long timestamp = in.readLong(); boolean isGlobal = in.readBoolean(); PreviewKind previewKind = PreviewKind.deserialize(in.readInt()); - return new PrepareMessage(parentRepairSession, tableIds, partitioner, ranges, isIncremental, timestamp, isGlobal, previewKind); + Epoch minEpoch = version >= MessagingService.VERSION_52 + ? Epoch.messageSerializer.deserialize(in, version) + : Epoch.EMPTY; + return new PrepareMessage(parentRepairSession, tableIds, partitioner, ranges, isIncremental, timestamp, isGlobal, previewKind, minEpoch); } public long serializedSize(PrepareMessage message, int version) @@ -158,6 +167,8 @@ public long serializedSize(PrepareMessage message, int version) size += TypeSizes.sizeof(message.repairedAt); size += TypeSizes.sizeof(message.isGlobal); size += TypeSizes.sizeof(message.previewKind.getSerializationVal()); + if (version >= MessagingService.VERSION_52) + size += Epoch.messageSerializer.serializedSize(message.minEpoch, version); return size; } }; @@ -172,6 +183,7 @@ public String toString() ", isIncremental=" + isIncremental + ", timestamp=" + repairedAt + ", isGlobal=" + isGlobal + + ", minEpoch=" + minEpoch + '}'; } } diff --git a/src/java/org/apache/cassandra/replication/ActiveLogReconciler.java b/src/java/org/apache/cassandra/replication/ActiveLogReconciler.java index 8667c766018e..e7d5504e9070 100644 --- a/src/java/org/apache/cassandra/replication/ActiveLogReconciler.java +++ b/src/java/org/apache/cassandra/replication/ActiveLogReconciler.java @@ -18,6 +18,7 @@ package org.apache.cassandra.replication; import java.util.Collections; +import java.io.IOException; import java.util.concurrent.TimeUnit; import com.google.common.annotations.VisibleForTesting; @@ -29,7 +30,10 @@ import org.agrona.concurrent.ManyToOneConcurrentLinkedQueue; import org.apache.cassandra.concurrent.Interruptible; import org.apache.cassandra.concurrent.Shutdownable; +import org.apache.cassandra.db.Mutation; +import org.apache.cassandra.db.rows.DeserializationHelper; import org.apache.cassandra.exceptions.RequestFailure; +import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.journal.RecordPointer; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.Message; @@ -38,6 +42,10 @@ import org.apache.cassandra.net.NoPayload; import org.apache.cassandra.net.RequestCallback; import org.apache.cassandra.net.Verb; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.concurrent.Semaphore; import static java.util.concurrent.TimeUnit.MINUTES; @@ -168,11 +176,34 @@ void send() RecordPointer pointer = MutationJournal.instance.lookUp(mutationId); Preconditions.checkNotNull(pointer, "Mutation %s not found in the journal", mutationId); - Message message = - Message.outWithFlag(Verb.PUSH_MUTATION_REQ, - new PushMutationRequest.Referenced(mutationId, pointer), - MessageFlag.CALL_BACK_ON_FAILURE); - MessagingService.instance().sendWithCallback(message, toHost, this); + MutationJournal.instance.read(pointer, (segment, position, key, buffer, version) -> { + + // don't send mutations to nodes that have migrated to, or are in the process of migrating to untracked replication + try (DataInputBuffer in = new DataInputBuffer(buffer, true)) + { + ClusterMetadata metadata = ClusterMetadata.current(); + TableId tableId = Mutation.serializer.deserializeTableId(in, version, DeserializationHelper.Flag.LOCAL); + + TableMetadata tableMetadata = metadata.schema.getTableMetadata(tableId); + if (tableMetadata == null) + return; + + KeyspaceMetadata ksm = metadata.schema.getKeyspaceMetadata(tableMetadata.keyspace); + if (ksm == null || !ksm.useMutationTracking()) + return; + + } + catch (IOException e) + { + throw new RuntimeException(e); + } + + Message message = + Message.outWithFlag(Verb.PUSH_MUTATION_REQ, + new PushMutationRequest.Buffer(version, buffer), + MessageFlag.CALL_BACK_ON_FAILURE); + MessagingService.instance().sendWithCallback(message, toHost, this); + }); } } diff --git a/src/java/org/apache/cassandra/replication/MutableCoordinatorLogOffsets.java b/src/java/org/apache/cassandra/replication/MutableCoordinatorLogOffsets.java index 9ecb389554d1..078ebf6316cf 100644 --- a/src/java/org/apache/cassandra/replication/MutableCoordinatorLogOffsets.java +++ b/src/java/org/apache/cassandra/replication/MutableCoordinatorLogOffsets.java @@ -32,6 +32,9 @@ default void addAll(Mutations mutations) { for (long logId : mutations) { + if (CoordinatorLogId.none().asLong() == logId) + continue; + Offsets offsets = mutations.offsets(logId); offsets.forEach(this::add); } @@ -39,6 +42,9 @@ default void addAll(Mutations mutations) default void addAll(Offsets from) { + if (from.logId().isNone()) + return; + from.forEach(this::add); } diff --git a/src/java/org/apache/cassandra/replication/MutationTrackingService.java b/src/java/org/apache/cassandra/replication/MutationTrackingService.java index 28f3d6533c32..18394bdf5390 100644 --- a/src/java/org/apache/cassandra/replication/MutationTrackingService.java +++ b/src/java/org/apache/cassandra/replication/MutationTrackingService.java @@ -711,6 +711,8 @@ private static ConcurrentHashMap applyUpdatedMetadata(Ma KeyspaceShards.UpdateDecision decision = KeyspaceShards.UpdateDecision.decisionForTopologyChange(keyspace, prev, next, current != null); switch (decision) { + case MIGRATE_FROM: + // TODO (expected): Implement shard deletion for tracked → untracked migration completion (CASSANDRA-20955) case NONE: if (current != null) updated.put(keyspace, current); @@ -719,7 +721,7 @@ private static ConcurrentHashMap applyUpdatedMetadata(Ma // Don't carry forward the state for the dropped keyspace break; case REPLICA_GROUP: - // if there's an existing keyspace shards instance, update it, otherwise call through to CREATE + // if there's an existing keyspace shards instance, update it, otherwise fall through to CREATE if (current != null) { KeyspaceShards ksShards = current.withUpdatedMetadata(next.schema.getKeyspaceMetadata(keyspace), next, logIdProvider, onNewLog); @@ -727,6 +729,7 @@ private static ConcurrentHashMap applyUpdatedMetadata(Ma break; } case CREATE: + case MIGRATE_TO: Preconditions.checkState(current == null, "Attempted to create a new keyspace shard for keyspace %s, but it already exists", keyspace); KeyspaceShards ksShards = KeyspaceShards.make(next.schema.getKeyspaceMetadata(keyspace), @@ -735,8 +738,6 @@ private static ConcurrentHashMap applyUpdatedMetadata(Ma onNewLog); updated.put(keyspace, ksShards); break; - case MIGRATE_TO: - case MIGRATE_FROM: default: throw new IllegalStateException("Unsupported keyspace shard update: " + decision); } @@ -810,7 +811,8 @@ static UpdateDecision decisionForTopologyChange(String keyspace, ClusterMetadata if (!prevKsm.useMutationTracking() && !nextKsm.useMutationTracking()) { - Preconditions.checkState(!hasExisting, "Existing shards found for keyspace, but prev & current ksm has mutation tracking disabled"); + // TODO: drop shards after migration to untracked +// Preconditions.checkState(!hasExisting, "Existing shards found for keyspace, but prev & current ksm has mutation tracking disabled"); return UpdateDecision.NONE; } @@ -877,7 +879,7 @@ private static Set> splitRange(Range range) static KeyspaceShards make(KeyspaceMetadata keyspace, ClusterMetadata cluster, LongSupplier logIdProvider, BiConsumer onNewLog) { - Preconditions.checkArgument(keyspace.params.replicationType.isTracked()); + Preconditions.checkArgument(keyspace.params.replicationType.isTracked() || cluster.mutationTrackingMigrationState.getKeyspaceInfo(keyspace.name) != null); Map, Shard> shards = new HashMap<>(); Map, VersionedEndpoints.ForRange> groups = new HashMap<>(); diff --git a/src/java/org/apache/cassandra/replication/NonBlockingCoordinatorLogOffsets.java b/src/java/org/apache/cassandra/replication/NonBlockingCoordinatorLogOffsets.java index 3f54ad847b08..5b3c79bbc28d 100644 --- a/src/java/org/apache/cassandra/replication/NonBlockingCoordinatorLogOffsets.java +++ b/src/java/org/apache/cassandra/replication/NonBlockingCoordinatorLogOffsets.java @@ -67,6 +67,8 @@ private Entry(CoordinatorLogId logId) public void add(ShortMutationId mutationId) { + if (mutationId.isNone()) + return; computeIfAbsent(mutationId.logId(), logId -> factory.create(new CoordinatorLogId(logId))).add(mutationId); } diff --git a/src/java/org/apache/cassandra/replication/PushMutationRequest.java b/src/java/org/apache/cassandra/replication/PushMutationRequest.java index 8e673816b5f3..84f50ed97e04 100644 --- a/src/java/org/apache/cassandra/replication/PushMutationRequest.java +++ b/src/java/org/apache/cassandra/replication/PushMutationRequest.java @@ -18,6 +18,7 @@ package org.apache.cassandra.replication; import java.io.IOException; +import java.nio.ByteBuffer; import com.google.common.base.Preconditions; @@ -75,6 +76,7 @@ public void serialize(DataOutputPlus out, int version) throws IOException { try { + Preconditions.checkState(userVersion == version); out.write(buffer); // TODO (expected): handle mismatched (messaging) versions } catch (IOException e) @@ -87,6 +89,31 @@ public void serialize(DataOutputPlus out, int version) throws IOException } } + class Buffer implements PushMutationRequest + { + private final int userVersion; + private final ByteBuffer buffer; + + public Buffer(int userVersion, ByteBuffer buffer) + { + this.userVersion = userVersion; + this.buffer = buffer; + } + + @Override + public long serializedSize(int version) + { + return buffer.remaining(); + } + + @Override + public void serialize(DataOutputPlus out, int version) throws IOException + { + Preconditions.checkState(userVersion == version); + out.write(buffer); // TODO (expected): handle mismatched (messaging) versions + } + } + class Materialized implements PushMutationRequest { public final Mutation mutation; diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java index bee41a7fa3a3..47687551cfd7 100644 --- a/src/java/org/apache/cassandra/service/ActiveRepairService.java +++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java @@ -109,6 +109,7 @@ import org.apache.cassandra.service.paxos.cleanup.PaxosCleanup; import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.utils.MerkleTrees; @@ -468,6 +469,7 @@ public RepairSession submitRepairSession(TimeUUID parentRepairSession, boolean repairAccord, ExecutorPlus executor, Scheduler validationScheduler, + Epoch minEpoch, String... cfnames) { if (repairPaxos && previewKind != PreviewKind.NONE) @@ -483,7 +485,7 @@ public RepairSession submitRepairSession(TimeUUID parentRepairSession, range, excludedDeadNodes, keyspace, parallelismDegree, isIncremental, pullRepair, previewKind, optimiseStreams, repairData, repairPaxos, - dontPurgeTombstones, repairAccord, cfnames); + dontPurgeTombstones, repairAccord, minEpoch, cfnames); repairs.getIfPresent(parentRepairSession).register(session.state); sessions.put(session.getId(), session); @@ -668,7 +670,7 @@ public boolean verifyCompactionsPendingThreshold(TimeUUID parentRepairSession, P return true; } - public Future prepareForRepair(TimeUUID parentRepairSession, InetAddressAndPort coordinator, Set endpoints, RepairOption options, boolean isForcedRepair, List columnFamilyStores) + public Future prepareForRepair(TimeUUID parentRepairSession, InetAddressAndPort coordinator, Set endpoints, RepairOption options, boolean isForcedRepair, List columnFamilyStores, Epoch minEpoch) { if (!verifyDiskHeadroomThreshold(parentRepairSession, options.getPreviewKind(), options.isIncremental())) failRepair(parentRepairSession, "Rejecting incoming repair, disk usage above threshold"); // failRepair throws exception @@ -693,7 +695,15 @@ public Future prepareForRepair(TimeUUID parentRepairSession, InetAddressAndPo if (partitioners.size() > 1) failRepair(parentRepairSession, "The tables involved in repair are configured with multiple partitioners."); - PrepareMessage message = new PrepareMessage(parentRepairSession, tableIds, columnFamilyStores.get(0).getPartitioner(), options.getRanges(), options.isIncremental(), repairedAt, options.isGlobal(), options.getPreviewKind()); + PrepareMessage message = new PrepareMessage(parentRepairSession, + tableIds, + columnFamilyStores.get(0).getPartitioner(), + options.getRanges(), + options.isIncremental(), + repairedAt, + options.isGlobal(), + options.getPreviewKind(), + minEpoch); register(new ParticipateState(ctx.clock(), ctx.broadcastAddressAndPort(), message)); for (InetAddressAndPort neighbour : endpoints) { diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java index 355d20d19abb..cde6ad1d58c3 100644 --- a/src/java/org/apache/cassandra/service/StorageProxy.java +++ b/src/java/org/apache/cassandra/service/StorageProxy.java @@ -164,12 +164,11 @@ import org.apache.cassandra.service.paxos.PaxosState; import org.apache.cassandra.service.paxos.v1.PrepareCallback; import org.apache.cassandra.service.paxos.v1.ProposeCallback; -import org.apache.cassandra.service.reads.AbstractReadExecutor; import org.apache.cassandra.service.reads.ReadCallback; import org.apache.cassandra.service.reads.ReadCoordinator; +import org.apache.cassandra.service.reads.ReadExecutor; import org.apache.cassandra.service.reads.range.RangeCommands; import org.apache.cassandra.service.reads.repair.ReadRepair; -import org.apache.cassandra.service.reads.tracked.TrackedRead; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.membership.NodeState; import org.apache.cassandra.tcm.ownership.VersionedEndpoints; @@ -2722,65 +2721,19 @@ public RowIterator next() }; } - private static PartitionIterator fetchRowsTracked(List commands, - ConsistencyLevel consistencyLevel, - Dispatcher.RequestTime requestTime) - { - int cmdCount = commands.size(); - TrackedRead.Partition[] reads = new TrackedRead.Partition[cmdCount]; - ClusterMetadata metadata = ClusterMetadata.current(); - - for (int i=0; i iterators = new ArrayList<>(cmdCount); - for (TrackedRead.Partition read : reads) - iterators.add(read.awaitResults()); - - return PartitionIterators.concat(iterators); - } - - /** - * This function executes local and remote reads, and blocks for the results: - * - * 1. Get the replica locations, sorted by response time according to the snitch - * 2. Send a data request to the closest replica, and digest requests to either - * a) all the replicas, if read repair is enabled - * b) the closest R-1 replicas, where R is the number required to satisfy the ConsistencyLevel - * 3. Wait for a response from R replicas - * 4. If the digests (if any) match the data return the data - * 5. else carry out read repair by getting data from all the nodes. - * - * This should not be called directly because it bypasses statistics and error handling. It is public - * so it can be used by Accord to fetch rows and the statistics will be tracked by Accord. - */ - public static PartitionIterator fetchRowsUntracked(List commands, - ConsistencyLevel consistencyLevel, - ReadCoordinator coordinator, - Dispatcher.RequestTime requestTime) - throws UnavailableException, ReadFailureException, ReadTimeoutException + public static PartitionIterator fetchRows(List commands, + ConsistencyLevel consistencyLevel, + ReadCoordinator coordinator, + Dispatcher.RequestTime requestTime) { - int cmdCount = commands.size(); + ClusterMetadata metadata = ClusterMetadata.current(); - AbstractReadExecutor[] reads = new AbstractReadExecutor[cmdCount]; + List executors = ReadExecutor.createExecutors(metadata, commands, consistencyLevel, coordinator, requestTime); - ClusterMetadata metadata = ClusterMetadata.current(); - // Get the replica locations, sorted by response time according to the snitch, and create a read executor - // for type of speculation we'll use in this read - for (int i=0; i results = new ArrayList<>(cmdCount); - List> repairs = new ArrayList<>(cmdCount); - for (int i=0; i iterators = new ArrayList<>(executors.size()); + List> repairs = new ArrayList<>(executors.size()); + for (ReadExecutor executor : executors) { - results.add(reads[i].getResult()); - repairs.add(reads[i].getReadRepair()); + iterators.add(executor.getResult()); + repairs.add(executor.getReadRepair()); } // if we did a read repair, assemble repair mutation and block on them - return concatAndBlockOnRepair(results, repairs); - } - - public static PartitionIterator fetchRows(List commands, - ConsistencyLevel consistencyLevel, - ReadCoordinator coordinator, - Dispatcher.RequestTime requestTime) - { - if (commands.get(0).metadata().replicationType().isTracked()) - { - return fetchRowsTracked(commands, consistencyLevel, requestTime); - } - else - { - return fetchRowsUntracked(commands, consistencyLevel, coordinator, requestTime); - } - + return concatAndBlockOnRepair(iterators, repairs); } public static class LocalReadRunnable extends DroppableRunnable implements RunnableDebuggableTask @@ -2967,6 +2894,7 @@ public static PartitionIterator getRangeSlice(PartitionRangeReadCommand command, tokens)); } } + return RangeCommands.partitions(command, consistencyLevel, readCoordinator, requestTime); } diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 653c889f0d3e..1c497544a70b 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -3139,7 +3139,7 @@ private FutureTask createRepairTask(final int cmd, final String keyspace throw new IllegalArgumentException("data center(s) " + datacenters.toString() + " not found"); } - RepairCoordinator task = new RepairCoordinator(this, cmd, options, keyspace); + RepairCoordinator task = new RepairCoordinator(this, cmd, options, keyspace, ClusterMetadata.current().epoch); task.addProgressListener(progressSupport); for (ProgressListener listener : listeners) task.addProgressListener(listener); diff --git a/src/java/org/apache/cassandra/service/consensus/migration/ConsensusMigrationMutationHelper.java b/src/java/org/apache/cassandra/service/consensus/migration/ConsensusMigrationMutationHelper.java index 282eca84eb13..6e5e4190fab0 100644 --- a/src/java/org/apache/cassandra/service/consensus/migration/ConsensusMigrationMutationHelper.java +++ b/src/java/org/apache/cassandra/service/consensus/migration/ConsensusMigrationMutationHelper.java @@ -49,6 +49,7 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.TableParams; import org.apache.cassandra.service.PreserveTimestamp; +import org.apache.cassandra.service.replication.migration.MigrationRouter; import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.service.accord.IAccordService; import org.apache.cassandra.service.accord.IAccordService.IAccordResult; @@ -211,12 +212,6 @@ public static void splitMutations(ClusterMetadata cm, List } } - private static boolean isTrackedMutation(IMutation mutation) - { - return Schema.instance.getKeyspaceMetadata(mutation.getKeyspaceName()).params.replicationType.isTracked(); - } - - /** * Splits mutations into tracked/untracked/accord mutations */ @@ -275,21 +270,30 @@ public SplitMutation(@Nullable T accordMutation, @Nullable T untrackedMutation, public SplitMutation splitMutation(T mutation, ClusterMetadata cm) { - boolean isTracked = isTrackedMutation(mutation); + Token token = mutation.key().getToken(); + Predicate isTrackedUpdate = tableId -> MigrationRouter.shouldUseTrackedForWrites(cm, mutation.getKeyspaceName(), tableId, token); + Predicate isUntrackedUpdate = not(isTrackedUpdate); if (mutation.potentialTxnConflicts().allowed) - return new SplitMutation<>(null, isTracked ? null : mutation, isTracked ? mutation : null); + { + return new SplitMutation<>(null, (T) mutation.filter(isUntrackedUpdate), (T) mutation.filter(isTrackedUpdate)); + } - Token token = mutation.key().getToken(); Predicate isAccordUpdate = tableId -> tokenShouldBeWrittenThroughAccord(cm, tableId, token, TransactionalMode::nonSerialWritesThroughAccord, TransactionalMigrationFromMode::nonSerialWritesThroughAccord); T accordMutation = (T)mutation.filter(isAccordUpdate); - T normalMutation = (T)mutation.filter(not(isAccordUpdate)); + T untrackedMutation = (T)mutation.filter(tid -> !isAccordUpdate.test(tid) && isUntrackedUpdate.test(tid)); + T trackedMutation = (T)mutation.filter(tid -> !isAccordUpdate.test(tid) && isTrackedUpdate.test(tid)); + for (PartitionUpdate pu : mutation.getPartitionUpdates()) checkState((accordMutation == null ? false : accordMutation.hasUpdateForTable(pu.metadata().id)) - || (normalMutation == null ? false : normalMutation.hasUpdateForTable(pu.metadata().id)), + || (untrackedMutation == null ? false : untrackedMutation.hasUpdateForTable(pu.metadata().id)) + || (trackedMutation == null ? false : trackedMutation.hasUpdateForTable(pu.metadata().id)), "All partition updates should still be present after splitting"); - return new SplitMutation(accordMutation, isTracked ? null : normalMutation, isTracked ? normalMutation : null); + if (trackedMutation != null && accordMutation != null) + throw new IllegalStateException("Accord cannot be used on keyspaces using tracked replication"); + + return new SplitMutation(accordMutation, untrackedMutation, trackedMutation); } public IAccordResult mutateWithAccordAsync(ClusterMetadata cm, Mutation mutation, @Nullable ConsistencyLevel consistencyLevel, Dispatcher.RequestTime requestTime, PreserveTimestamp preserveTimestamps) diff --git a/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java b/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java index 13cca1d7185a..7464a1eb8958 100644 --- a/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java +++ b/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java @@ -47,7 +47,6 @@ import org.apache.cassandra.tracing.TraceState; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.transport.Dispatcher; -import org.apache.cassandra.utils.FBUtilities; import static com.google.common.collect.Iterables.all; import static java.util.concurrent.TimeUnit.MICROSECONDS; @@ -62,7 +61,7 @@ * SpeculatingReadExecutor will wait until it looks like the original request is in danger * of timing out before performing extra reads. */ -public abstract class AbstractReadExecutor +public abstract class AbstractReadExecutor implements ReadExecutor { private static final Logger logger = LoggerFactory.getLogger(AbstractReadExecutor.class); @@ -223,11 +222,6 @@ public static AbstractReadExecutor getReadExecutor(ClusterMetadata metadata, return new SpeculatingReadExecutor(coordinator, cfs, command, replicaPlan, requestTime); } - public boolean hasLocalRead() - { - return replicaPlan().lookup(FBUtilities.getBroadcastAddressAndPort()) != null; - } - /** * Returns true if speculation should occur and if it should then block until it is time to * send the speculative reads @@ -258,7 +252,8 @@ boolean shouldSpeculateAndMaybeWait() return !handler.awaitUntil(requestTime.startedAtNanos() + sampleLatencyNanos); } - ReplicaPlan.ForTokenRead replicaPlan() + @Override + public ReplicaPlan.ForTokenRead replicaPlan() { return replicaPlan.get(); } diff --git a/src/java/org/apache/cassandra/service/reads/ReadExecutor.java b/src/java/org/apache/cassandra/service/reads/ReadExecutor.java new file mode 100644 index 000000000000..3adb00b45a3b --- /dev/null +++ b/src/java/org/apache/cassandra/service/reads/ReadExecutor.java @@ -0,0 +1,79 @@ +/* + * 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.reads; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.SinglePartitionReadCommand; +import org.apache.cassandra.db.partitions.PartitionIterator; +import org.apache.cassandra.locator.ReplicaPlan; +import org.apache.cassandra.service.reads.repair.ReadRepair; +import org.apache.cassandra.service.replication.migration.MigrationRouter; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.transport.Dispatcher; +import org.apache.cassandra.utils.FBUtilities; + +public interface ReadExecutor +{ + static List createExecutors(ClusterMetadata metadata, + List commands, + ConsistencyLevel consistencyLevel, + ReadCoordinator coordinator, + Dispatcher.RequestTime requestTime) + { + List executors = new ArrayList<>(commands.size()); + + for (SinglePartitionReadCommand command : commands) + { + if (MigrationRouter.shouldUseTracked(command)) + { + executors.add(new TrackedReadExecutor(metadata, command, consistencyLevel, requestTime)); + } + else + { + executors.add(AbstractReadExecutor.getReadExecutor(metadata, command, consistencyLevel, coordinator, requestTime)); + } + } + + return executors; + } + + ReplicaPlan.ForTokenRead replicaPlan(); + + default boolean hasLocalRead() + { + return replicaPlan().lookup(FBUtilities.getBroadcastAddressAndPort()) != null; + } + + void executeAsync(); + + void maybeTryAdditionalReplicas(); + + void awaitResponses(boolean logBlockingReadRepairAttempts); + + void maybeSendAdditionalDataRequests(); + + void awaitReadRepair(); + + PartitionIterator getResult(); + + ReadRepair getReadRepair(); +} diff --git a/src/java/org/apache/cassandra/service/reads/TrackedReadExecutor.java b/src/java/org/apache/cassandra/service/reads/TrackedReadExecutor.java new file mode 100644 index 000000000000..e3ccffe3d37d --- /dev/null +++ b/src/java/org/apache/cassandra/service/reads/TrackedReadExecutor.java @@ -0,0 +1,84 @@ +/* + * 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.reads; + +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.SinglePartitionReadCommand; +import org.apache.cassandra.db.partitions.PartitionIterator; +import org.apache.cassandra.locator.ReplicaPlan; +import org.apache.cassandra.service.reads.repair.NoopReadRepair; +import org.apache.cassandra.service.reads.repair.ReadRepair; +import org.apache.cassandra.service.reads.tracked.TrackedRead; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.transport.Dispatcher; + +public class TrackedReadExecutor implements ReadExecutor +{ + private final TrackedRead.Partition read; + private final Dispatcher.RequestTime requestTime; + private PartitionIterator result; + + public TrackedReadExecutor(ClusterMetadata metadata, + SinglePartitionReadCommand command, + ConsistencyLevel consistencyLevel, + Dispatcher.RequestTime requestTime) + { + this.read = TrackedRead.Partition.create(metadata, command, consistencyLevel, requestTime); + this.requestTime = requestTime; + } + + @Override + public ReplicaPlan.ForTokenRead replicaPlan() + { + return (ReplicaPlan.ForTokenRead) read.replicaPlan(); + } + + @Override + public void executeAsync() + { + read.start(requestTime); + } + + @Override + public void maybeTryAdditionalReplicas() {} + + @Override + public void awaitResponses(boolean logBlockingReadRepairAttempts) + { + result = read.awaitResults(); + } + + @Override + public void maybeSendAdditionalDataRequests() {} + + @Override + public void awaitReadRepair() {} + + @Override + public PartitionIterator getResult() + { + return result; + } + + @Override + public ReadRepair getReadRepair() + { + return NoopReadRepair.instance; + } +} diff --git a/src/java/org/apache/cassandra/service/reads/range/RangeCommandIterator.java b/src/java/org/apache/cassandra/service/reads/range/RangeCommandIterator.java index 0b867710d32f..02d46807849f 100644 --- a/src/java/org/apache/cassandra/service/reads/range/RangeCommandIterator.java +++ b/src/java/org/apache/cassandra/service/reads/range/RangeCommandIterator.java @@ -57,6 +57,8 @@ import org.apache.cassandra.service.reads.ReadCoordinator; import org.apache.cassandra.service.reads.repair.ReadRepair; import org.apache.cassandra.service.reads.tracked.TrackedRead; +import org.apache.cassandra.service.replication.migration.MigrationRouter; +import org.apache.cassandra.service.replication.migration.MigrationRouter.RangeReadWithReplication; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.transport.Dispatcher; @@ -255,37 +257,34 @@ private PartitionIterator query(ClusterMetadata cm, ReplicaPlan.ForRangeRead rep return response; } - List reads = ConsensusRequestRouter.splitReadIntoAccordAndNormal(cm, rangeCommand, readCoordinator, requestTime); + List accordSplits = ConsensusRequestRouter.splitReadIntoAccordAndNormal(cm, rangeCommand, readCoordinator, requestTime); + // Special case returning directly to avoid wrapping the iterator and applying the limits an extra time - if (reads.size() == 1) + if (accordSplits.size() == 1) { - RangeReadWithTarget rangeReadWithTarget = reads.get(0); - checkState(rangeReadWithTarget.read.dataRange().keyRange().equals(rangeCommand.dataRange().keyRange())); - if (rangeReadWithTarget.target == RangeReadTarget.accord && readCoordinator.isEventuallyConsistent()) + RangeReadWithTarget accordSplit = accordSplits.get(0); + checkState(accordSplit.read.dataRange().keyRange().equals(rangeCommand.dataRange().keyRange())); + + if (accordSplit.target == RangeReadTarget.accord && readCoordinator.isEventuallyConsistent()) { - return executeAccord(cm, - rangeReadWithTarget.read, - replicaPlan.consistencyLevel()); + return executeAccord(cm, accordSplit.read, replicaPlan.consistencyLevel()); } else { - SingleRangeResponse response = executeNormal(replicaPlan, rangeReadWithTarget.read, readCoordinator); - readRepairs.add(response.getReadRepair()); - return response; + return executeNormalWithMigrationSplit(cm, replicaPlan, readCoordinator, readRepairs, accordSplit.read); } } // TODO (review): Should this be reworked to execute the queries serially from the iterator? It would respect // any provided limits better but the number of queries created will generally be low (2-3) - List responses = new ArrayList<>(reads.size() + 1); - // Dummy iterator that checks all the responses for retry on different system hasNext so we don't read - // from the first iterator when the second needs to be retried because the split was wrong + List responses = new ArrayList<>(accordSplits.size() + 1); + + // Dummy iterator for retry checking across all splits responses.add(new PartitionIterator() { @Override public void close() { - } @Override @@ -303,63 +302,68 @@ public RowIterator next() } }); - for (RangeReadWithTarget rangeReadWithTarget : reads) + for (RangeReadWithTarget accordSplit : accordSplits) { - if (rangeReadWithTarget.target == RangeReadTarget.accord && readCoordinator.isEventuallyConsistent()) - responses.add(executeAccord(cm, rangeReadWithTarget.read, replicaPlan.consistencyLevel())); + if (accordSplit.target == RangeReadTarget.accord && readCoordinator.isEventuallyConsistent()) + { + responses.add(executeAccord(cm, accordSplit.read, replicaPlan.consistencyLevel())); + } else { - SingleRangeResponse response = executeNormal(replicaPlan, rangeReadWithTarget.read, readCoordinator); - responses.add(response); - readRepairs.add(response.getReadRepair()); + responses.add(executeNormalWithMigrationSplit(cm, replicaPlan, readCoordinator, readRepairs, accordSplit.read)); } } - /* - * We have to apply limits here if the query spans different systems because each subquery we created - * could have gaps in the results since the limit is pushed down independently to each subquery. - * So if we don't meet the limit in the first subquery, it's not safe to go to the next one unless - * we fully exhausted the data the first subquery might have reached - */ + // Apply limits since splits may have gaps in results return command.limits().filter(PartitionIterators.concat(responses), 0, command.selectsFullPartition(), command.metadata().enforceStrictLiveness()); } - private PartitionIterator sendNextRequestsTracked() + private PartitionIterator executeSplit(RangeReadWithReplication split, ReplicaPlan.ForRangeRead replicaPlan, List> readRepairs) { - List concurrentQueries = new ArrayList<>(concurrencyFactor); - - try + if (split.useTracked) { - for (int i = 0; i < concurrencyFactor && replicaPlans.hasNext(); ) - { - ReplicaPlan.ForRangeRead replicaPlan = replicaPlans.next(); - PartitionRangeReadCommand rangeCommand = command.forSubRange(replicaPlan.range(), i == 0); - - TrackedRead.Range read = TrackedRead.Range.create(rangeCommand, replicaPlan, requestTime); - read.start(requestTime); - concurrentQueries.add(read.iterator()); - - // due to RangeMerger, coordinator may fetch more ranges than required by concurrency factor. - rangesQueried += replicaPlan.vnodeCount(); - i += replicaPlan.vnodeCount(); - } - batchesRequested++; + TrackedRead.Range read = TrackedRead.Range.create(split.read, replicaPlan, requestTime); + read.start(requestTime); + return read.iterator(); } - catch (Throwable t) + else { - for (PartitionIterator response : concurrentQueries) - response.close(); - throw t; + SingleRangeResponse response = executeNormal(replicaPlan, split.read, readCoordinator); + readRepairs.add(response.getReadRepair()); + return response; } - Tracing.trace("Submitted {} concurrent range requests", concurrentQueries.size()); + } + + /** + * Execute a normal C* range, splitting for migration if needed. + */ + private PartitionIterator executeNormalWithMigrationSplit(ClusterMetadata cm, + ReplicaPlan.ForRangeRead replicaPlan, + ReadCoordinator readCoordinator, + List> readRepairs, + PartitionRangeReadCommand rangeCommand) + { + List migrationSplits = MigrationRouter.splitRangeRead(cm, rangeCommand); + + if (migrationSplits.size() == 1) + return executeSplit(migrationSplits.get(0), replicaPlan, readRepairs); + + List responses = new ArrayList<>(migrationSplits.size()); - return PartitionIterators.concat(concurrentQueries); + for (RangeReadWithReplication split : migrationSplits) + responses.add(executeSplit(split, replicaPlan, readRepairs)); + + // Apply limits since migration splits may have gaps in results + return rangeCommand.limits().filter(PartitionIterators.concat(responses), + 0, + rangeCommand.selectsFullPartition(), + rangeCommand.metadata().enforceStrictLiveness()); } - PartitionIterator sendNextRequestsUntracked() + PartitionIterator splitAndSendNextRequests() { List concurrentQueries = new ArrayList<>(concurrencyFactor); List> readRepairs = new ArrayList<>(concurrencyFactor); @@ -403,16 +407,8 @@ PartitionIterator sendNextRequestsUntracked() PartitionIterator sendNextRequests() { - PartitionIterator result; - if (command.metadata().replicationType().isTracked()) - { - result = sendNextRequestsTracked(); - - } - else - { - result = sendNextRequestsUntracked(); - } + // query() handles Accord and migration splitting + PartitionIterator result = splitAndSendNextRequests(); // We want to count the results for the sake of updating the concurrency factor (see updateConcurrencyFactor) // but we don't want to enforce any particular limit at this point (this could break code than rely on diff --git a/src/java/org/apache/cassandra/service/reads/tracked/TrackedRead.java b/src/java/org/apache/cassandra/service/reads/tracked/TrackedRead.java index 27642583b246..6a10357cf858 100644 --- a/src/java/org/apache/cassandra/service/reads/tracked/TrackedRead.java +++ b/src/java/org/apache/cassandra/service/reads/tracked/TrackedRead.java @@ -168,6 +168,11 @@ public TrackedRead(ReadCommand command, ReplicaPlan.AbstractForRead replic this.requestTime = requestTime; } + public ReplicaPlan.AbstractForRead replicaPlan() + { + return replicaPlan; + } + @Override public String toString() { diff --git a/src/java/org/apache/cassandra/service/replication/migration/KeyspaceMigrationInfo.java b/src/java/org/apache/cassandra/service/replication/migration/KeyspaceMigrationInfo.java new file mode 100644 index 000000000000..ed193d61c293 --- /dev/null +++ b/src/java/org/apache/cassandra/service/replication/migration/KeyspaceMigrationInfo.java @@ -0,0 +1,305 @@ +/* + * 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.replication.migration; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import javax.annotation.Nonnull; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.dht.NormalizedRanges; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.tcm.serialization.MetadataSerializer; +import org.apache.cassandra.tcm.serialization.Version; + +import static org.apache.cassandra.db.TypeSizes.sizeof; +import static org.apache.cassandra.utils.CollectionSerializers.deserializeList; +import static org.apache.cassandra.utils.CollectionSerializers.deserializeMap; +import static org.apache.cassandra.utils.CollectionSerializers.serializeCollection; +import static org.apache.cassandra.utils.CollectionSerializers.serializeMap; +import static org.apache.cassandra.utils.CollectionSerializers.serializedCollectionSize; +import static org.apache.cassandra.utils.CollectionSerializers.serializedMapSize; + +/** + * TCM state tracking mutation tracking migration for a keyspace. Since repair advances the migration, and + * and repair sessions operate against tables, this class tracks repairs on every table that existed in the + * keyspace when the migration started. + * At the beginning of a migration, the full range is added to the pendingRanges, and as repairs are completed, the + * repaired ranges are subtracted from the pending ranges. When the pending range list is empty, the migration is finished. + */ +public class KeyspaceMigrationInfo +{ + @Nonnull public final String keyspace; + @Nonnull public final Map> pendingRangesPerTable; + @Nonnull public final Epoch startedAtEpoch; + + public KeyspaceMigrationInfo(@Nonnull String keyspace, + @Nonnull Map> pendingRangesPerTable, + @Nonnull Epoch startedAtEpoch) + { + this.keyspace = Objects.requireNonNull(keyspace); + this.pendingRangesPerTable = ImmutableMap.copyOf(pendingRangesPerTable); + this.startedAtEpoch = Objects.requireNonNull(startedAtEpoch); + } + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) return false; + KeyspaceMigrationInfo that = (KeyspaceMigrationInfo) o; + return Objects.equals(keyspace, that.keyspace) && Objects.equals(pendingRangesPerTable, that.pendingRangesPerTable) && Objects.equals(startedAtEpoch, that.startedAtEpoch); + } + + @Override + public int hashCode() + { + throw new UnsupportedOperationException(); + } + + /** + * Reverse migration direction. Since unfinished migrations can be aborted, ranges that have not completed migrating + * in the previous direction are immediately rolled back. For ranges that did complete migration, or tables that were + * added since migration started, migration in the other direction is now required, so they're marked pending. + */ + public KeyspaceMigrationInfo withDirectionReversed(@Nonnull Collection allTableIds, + @Nonnull Epoch epoch) + { + Token minimumToken = DatabaseDescriptor.getPartitioner().getMinimumToken(); + + // Reset all tables to full ring pending (includes tables currently migrating, added during migration, or already migrated) + ImmutableMap.Builder> reversedPendingBuilder = ImmutableMap.builder(); + + for (TableId tableId : allTableIds) + { + Range fullRing = new Range<>(minimumToken, minimumToken); + NormalizedRanges reversedRanges = NormalizedRanges.normalizedRanges(Collections.singleton(fullRing)); + + NormalizedRanges existingPending = pendingRangesPerTable.get(tableId); + if (existingPending != null) + { + Set> ranges = Range.subtract(Collections.singletonList(fullRing), existingPending); + reversedRanges = NormalizedRanges.normalizedRanges(ranges); + } + + if (!reversedRanges.isEmpty()) + reversedPendingBuilder.put(tableId, reversedRanges); + } + + return new KeyspaceMigrationInfo( + keyspace, + reversedPendingBuilder.build(), + epoch + ); + } + + /** + * Remove tables from migration state. Returns null if all tables removed. + */ + public KeyspaceMigrationInfo withTablesRemoved(@Nonnull Set tablesToRemove) + { + if (tablesToRemove.isEmpty()) + return this; + + ImmutableMap.Builder> builder = ImmutableMap.builder(); + boolean anyRemoved = false; + + for (Map.Entry> entry : pendingRangesPerTable.entrySet()) + { + if (!tablesToRemove.contains(entry.getKey())) + { + builder.put(entry.getKey(), entry.getValue()); + } + else + { + anyRemoved = true; + } + } + + if (!anyRemoved) + return this; + + Map> newPending = builder.build(); + + if (newPending.isEmpty()) + return null; + + return new KeyspaceMigrationInfo( + keyspace, + newPending, + startedAtEpoch + ); + } + + /** + * Subtract repaired ranges from table's pending set. + * Automatically removes table if all ranges repaired. + */ + public KeyspaceMigrationInfo withRangesRepairedForTable(@Nonnull Epoch repairStartedEpoch, + @Nonnull TableId tableId, + @Nonnull Collection> repairedRanges) + { + if (repairStartedEpoch.isBefore(startedAtEpoch)) + return this; + + NormalizedRanges currentPendingForTable = pendingRangesPerTable.get(tableId); + if (currentPendingForTable == null) + { + return this; + } + + NormalizedRanges normalizedRepaired = NormalizedRanges.normalizedRanges(repairedRanges); + NormalizedRanges remainingForTable = currentPendingForTable.subtract(normalizedRepaired); + + ImmutableMap.Builder> builder = ImmutableMap.builder(); + for (Map.Entry> entry : pendingRangesPerTable.entrySet()) + { + if (entry.getKey().equals(tableId)) + { + if (!remainingForTable.isEmpty()) + builder.put(tableId, remainingForTable); + } + else + { + builder.put(entry.getKey(), entry.getValue()); + } + } + + return new KeyspaceMigrationInfo(keyspace, builder.build(), startedAtEpoch); + } + + /** + * Check if migration is complete (no tables have pending ranges). + * Migration is complete when all tables have been fully repaired and removed from the map. + */ + public boolean isComplete() + { + return pendingRangesPerTable.isEmpty(); + } + + public NormalizedRanges getPendingRangesForTable(@Nonnull TableId tableId) + { + NormalizedRanges ranges = pendingRangesPerTable.get(tableId); + return ranges != null ? ranges : NormalizedRanges.empty(); + } + + /** + * Check if token is in any pending range. + * Used for routing decisions during migration. + * + * @param token token to check + * @return true if token is in a pending range + */ + public boolean isTokenInPendingRange(TableId tableId, Token token) + { + NormalizedRanges tableRanges = pendingRangesPerTable.get(tableId); + if (tableRanges == null) + return false; + return tableRanges.intersects(token); + } + + /** + * Determine if read operations on a token should use tracked replication during migration. + * + * We only use tracked reads for ranges that have completed migrating _to_ tracked replication. + */ + public boolean shouldUseTrackedForReads(boolean isTracked, TableId tableId, Token token) + { + return isTracked && !isTokenInPendingRange(tableId, token); + } + + /** + * Determine if write operations on a token should use tracked replication during migration. + * + * The only time we don't use tracked writes is when a range has completed migration to untracked replication + */ + public boolean shouldUseTrackedForWrites(boolean isTracked, TableId tableId, Token token) + { + return isTracked || isTokenInPendingRange(tableId, token); + } + + @Override + public String toString() + { + return String.format("KeyspaceMigrationInfo{keyspace=%s, pendingTables=%d, started=%s}", + keyspace, pendingRangesPerTable.size(), startedAtEpoch); + } + + private static final MetadataSerializer> normalizedRangesSerializer = new MetadataSerializer>() + { + @Override + public void serialize(NormalizedRanges ranges, DataOutputPlus out, Version version) throws IOException + { + serializeCollection(ranges, out, version, Range.serializer); + } + + @Override + public NormalizedRanges deserialize(DataInputPlus in, Version version) throws IOException + { + List> rangeList = deserializeList(in, version, Range.serializer); + return NormalizedRanges.normalizedRanges(rangeList); + } + + @Override + public long serializedSize(NormalizedRanges ranges, Version version) + { + return serializedCollectionSize(ranges, version, Range.serializer); + } + }; + + public static final MetadataSerializer serializer = new MetadataSerializer() + { + @Override + public void serialize(KeyspaceMigrationInfo info, DataOutputPlus out, Version version) throws IOException + { + out.writeUTF(info.keyspace); + serializeMap(info.pendingRangesPerTable, out, version, TableId.metadataSerializer, normalizedRangesSerializer); + Epoch.serializer.serialize(info.startedAtEpoch, out, version); + } + + @Override + public KeyspaceMigrationInfo deserialize(DataInputPlus in, Version version) throws IOException + { + String keyspace = in.readUTF(); + Map> pendingRangesPerTable = deserializeMap(in, version, TableId.metadataSerializer, normalizedRangesSerializer, Maps::newHashMapWithExpectedSize); + Epoch startedAtEpoch = Epoch.serializer.deserialize(in, version); + return new KeyspaceMigrationInfo(keyspace, pendingRangesPerTable, startedAtEpoch); + } + + @Override + public long serializedSize(KeyspaceMigrationInfo info, Version version) + { + return sizeof(info.keyspace) + + serializedMapSize(info.pendingRangesPerTable, version, TableId.metadataSerializer, normalizedRangesSerializer) + + Epoch.serializer.serializedSize(info.startedAtEpoch, version); + } + }; +} diff --git a/src/java/org/apache/cassandra/service/replication/migration/MigrationRouter.java b/src/java/org/apache/cassandra/service/replication/migration/MigrationRouter.java new file mode 100644 index 000000000000..a363a6a8b4ef --- /dev/null +++ b/src/java/org/apache/cassandra/service/replication/migration/MigrationRouter.java @@ -0,0 +1,392 @@ +/* + * 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.replication.migration; + +import java.util.ArrayList; +import java.util.List; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.db.*; +import org.apache.cassandra.db.virtual.VirtualMutation; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.NormalizedRanges; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.exceptions.CoordinatorBehindException; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.utils.Pair; + +import static com.google.common.base.Preconditions.checkState; + +/** + * Routes read and write requests based on schema and migration state. + * + * During migration in either direction, reads are untracked and writes are tracked + */ +public class MigrationRouter +{ + public static boolean shouldUseTracked(SinglePartitionReadCommand command) + { + // System keyspaces never use tracked replication + if (SchemaConstants.isSystemKeyspace(command.metadata().keyspace)) + return false; + + ClusterMetadata metadata = ClusterMetadata.current(); + + KeyspaceMigrationInfo migrationInfo = metadata.mutationTrackingMigrationState.getKeyspaceInfo(command.metadata().keyspace); + + if (migrationInfo == null) + return command.metadata().replicationType().isTracked(); + + Token token = command.partitionKey().getToken(); + boolean isTracked = command.metadata().replicationType().isTracked(); + + return migrationInfo.shouldUseTrackedForReads(isTracked, command.metadata().id(), token); + } + + /** + * Wrapper for a range read command paired with its routing decision. + */ + public static class RangeReadWithReplication + { + public final PartitionRangeReadCommand read; + public final boolean useTracked; + + public RangeReadWithReplication(PartitionRangeReadCommand read, boolean useTracked) + { + this.read = read; + this.useTracked = useTracked; + } + } + + /** + * Helper to create and add a range split to the result list. + */ + private static void addSplit(List result, + PartitionRangeReadCommand command, + AbstractBounds range, + boolean isTracked) + { + boolean isFirst = result.isEmpty(); + result.add(new RangeReadWithReplication(command.forSubRange(range, isFirst), isTracked)); + } + + /** + * Adds a split for the non-pending region before pendingRange, if one exists. + * + * @param isTracked the target replication type (TO_TRACKED=true, TO_UNTRACKED=false) + * @return true if remainder ends before pendingRange (no intersection possible) + */ + private static boolean addNonPendingGapIfExists(List result, + PartitionRangeReadCommand command, + AbstractBounds remainder, + Range pendingRange, + boolean isTracked) + { + Token pendingStart = pendingRange.left; + Token remainderStart = remainder.left.getToken(); + Token remainderEnd = remainder.right.getToken(); + + if (remainderStart.compareTo(pendingStart) >= 0) + return false; // No gap before pending range + + // Check if remainder ends before pending range starts + if (remainderEnd.compareTo(pendingStart) <= 0) + { + // Entire remainder is before this pending range - no intersection + // Non-pending regions use the new protocol (isTracked) + addSplit(result, command, remainder, isTracked); + return true; + } + + // Add the non-pending gap before pending range + AbstractBounds gap = remainder.withNewRight(pendingStart.maxKeyBound()); + + if (!gap.left.equals(gap.right)) + addSplit(result, command, gap, isTracked); + + return false; + } + + /** + * Split a range by pending ranges, creating sub-ranges for each contiguous region. + *

+ * If we're migrating to tracked replication, pending ranges use untracked reads, non-pending uses tracked + *

+ * If we're migrating to untracked replication, pending uses tracked reads, and non-pending uses untracked + */ + private static List splitRangeByPendingRanges(PartitionRangeReadCommand command, + AbstractBounds keyRange, + NormalizedRanges pendingRanges, + boolean isTracked) + { + Preconditions.checkArgument(!AbstractBounds.strictlyWrapsAround(keyRange.left, keyRange.right)); + + List result = new ArrayList<>(); + AbstractBounds remainder = keyRange; + + for (Range pendingRange : pendingRanges) + { + // Add non-pending gap before this pending range (if exists) + if (addNonPendingGapIfExists(result, command, remainder, pendingRange, isTracked)) + { + remainder = null; + break; // No more remainder to process + } + + // Add intersection with pending range + Pair, AbstractBounds> split = + Range.intersectionAndRemainder(remainder, pendingRange); + + // Pending regions use the old protocol (!isTracked) + if (split.left != null) + addSplit(result, command, split.left, !isTracked); + + remainder = split.right; + if (remainder == null) + break; + } + + // Add final non-pending remainder + if (remainder != null) + addSplit(result, command, remainder, isTracked); + + return result; + } + + /** + * Validate that splits are contiguous, cover the entire original range, and alternate protocols. + */ + private static void validateSplitContiguity(PartitionRangeReadCommand originalCommand, + List splits) + { + checkState(!splits.isEmpty(), "Shouldn't have empty result"); + + // Validate coverage + checkState(splits.get(0).read.dataRange().startKey() + .equals(originalCommand.dataRange().startKey()), + "Split reads should encompass entire range"); + checkState(splits.get(splits.size() - 1).read.dataRange().stopKey() + .equals(originalCommand.dataRange().stopKey()), + "Split reads should encompass entire range"); + + // Validate contiguity and alternating protocols + if (splits.size() > 1) + { + for (int i = 0; i < splits.size() - 1; i++) + { + checkState(splits.get(i).read.dataRange().stopKey() + .equals(splits.get(i + 1).read.dataRange().startKey()), + "Split reads should all be adjacent"); + checkState(splits.get(i).useTracked != splits.get(i + 1).useTracked, + "Split reads should be for different replication protocols"); + } + } + } + + /** + * Split a range read command into sub-ranges based on migration state. + */ + public static List splitRangeRead(ClusterMetadata metadata, + PartitionRangeReadCommand command) + { + // System keyspaces never use tracked replication + if (SchemaConstants.isSystemKeyspace(command.metadata().keyspace)) + return ImmutableList.of(new RangeReadWithReplication(command, false)); + + KeyspaceMigrationInfo migrationInfo = metadata.mutationTrackingMigrationState + .getKeyspaceInfo(command.metadata().keyspace); + + boolean isTracked = command.metadata().replicationType().isTracked(); + + // During migration, reads use untracked replication except for ranges that have + // completed migration to tracked. Therefore, we only need to split ranges when + // migrating to tracked replication. For untracked migrations, all reads use untracked. + if (!isTracked || migrationInfo == null) + return ImmutableList.of(new RangeReadWithReplication(command, isTracked)); + + // Get pending ranges for this table + NormalizedRanges tablePendingRanges = migrationInfo.pendingRangesPerTable.get(command.metadata().id()); + + // No pending ranges for this table - entire range uses current protocol + if (tablePendingRanges == null) + return ImmutableList.of(new RangeReadWithReplication(command, isTracked)); + + // split into pending (untracked) and non-pending (tracked) ranges + List result = splitRangeByPendingRanges( + command, + command.dataRange().keyRange(), + tablePendingRanges, + isTracked); + + // Validate the splits + validateSplitContiguity(command, result); + + return result; + } + + public static boolean shouldUseTrackedForWrites(ClusterMetadata metadata, String keyspace, TableId tableId, Token token) + { + if (SchemaConstants.isSystemKeyspace(keyspace)) + return false; + + KeyspaceMigrationInfo migrationInfo = metadata.mutationTrackingMigrationState + .getKeyspaceInfo(keyspace); + + if (migrationInfo == null) + return metadata.schema.getKeyspaceMetadata(keyspace).params.replicationType.isTracked(); + + boolean isTracked = metadata.schema.getKeyspaceMetadata(keyspace).params.replicationType.isTracked(); + return migrationInfo.shouldUseTrackedForWrites(isTracked, tableId, token); + } + + public static class RoutedMutations + { + public final List trackedMutations; + public final List untrackedMutations; + + public RoutedMutations(List tracked, List untracked) + { + this.trackedMutations = tracked; + this.untrackedMutations = untracked; + } + } + + /** + * Route a list of mutations, splitting them into tracked and untracked groups. + */ + @VisibleForTesting + static RoutedMutations routeMutations(ClusterMetadata cm, List mutations) + { + List tracked = new ArrayList<>(); + List untracked = new ArrayList<>(); + + for (IMutation mutation : mutations) + { + if (mutation instanceof VirtualMutation) + { + untracked.add(mutation); + continue; + } + + // we need to router system keyspace mutations before CMS is ready + if (cm == null && !SchemaConstants.isSystemKeyspace(mutation.getKeyspaceName())) + cm = ClusterMetadata.current(); + + { + ClusterMetadata cm0 = cm; + IMutation untrackedMutation = mutation.filter(tid -> !shouldUseTrackedForWrites(cm0, mutation.getKeyspaceName(), tid, mutation.key().getToken())); + if (untrackedMutation != null) + untracked.add(untrackedMutation); + + IMutation trackedMutation = mutation.filter(tid -> shouldUseTrackedForWrites(cm0, mutation.getKeyspaceName(), tid, mutation.key().getToken())); + if (trackedMutation != null) + tracked.add(trackedMutation); + } + + } + + return new RoutedMutations(tracked, untracked); + } + + public static RoutedMutations routeMutations(List mutations) + { + return routeMutations(null, mutations); + } + + public enum MutationRouting + { + TRACKED, UNTRACKED, MIXED + } + + public static MutationRouting getMutationRouting(ClusterMetadata cm, IMutation mutation) + { + // System keyspaces always use untracked replication + if (SchemaConstants.isSystemKeyspace(mutation.getKeyspaceName())) + return MutationRouting.UNTRACKED; + + if (cm == null) + cm = ClusterMetadata.current(); + String keyspace = mutation.getKeyspaceName(); + Token token = mutation.key().getToken(); + + MutationRouting result = null; + for (TableId tableId : mutation.getTableIds()) + { + MutationRouting routing = shouldUseTrackedForWrites(cm, keyspace, tableId, token) + ? MutationRouting.TRACKED + : MutationRouting.UNTRACKED; + + if (result == null) + result = routing; + else if (result != routing) + return MutationRouting.MIXED; + } + + return result != null ? result : MutationRouting.UNTRACKED; + } + + public static MutationRouting getMutationRouting(IMutation mutation) + { + return getMutationRouting(null, mutation); + } + + + public static boolean isFullyTracked(IMutation mutation) + { + return getMutationRouting(mutation) == MutationRouting.TRACKED; + } + + private static void validateMutationReplication(IMutation mutation, MutationRouting expected) + { + switch (expected) + { + case TRACKED: + if (mutation.id().isNone()) + throw new IllegalArgumentException(); + break; + case UNTRACKED: + if (!mutation.id().isNone()) + throw new IllegalArgumentException(); + break; + default: + throw new IllegalArgumentException(); + + } + + MutationRouting actual = getMutationRouting(mutation); + if (expected != actual) + throw new CoordinatorBehindException("Mutation replication mismatch: expected " + expected + ", actual " + actual); + } + + public static void validateTrackedMutation(IMutation mutation) + { + validateMutationReplication(mutation, MutationRouting.TRACKED); + } + + public static void validateUntrackedMutation(IMutation mutation) + { + validateMutationReplication(mutation, MutationRouting.UNTRACKED); + } +} diff --git a/src/java/org/apache/cassandra/service/replication/migration/MutationTrackingMigrationRepairResult.java b/src/java/org/apache/cassandra/service/replication/migration/MutationTrackingMigrationRepairResult.java new file mode 100644 index 000000000000..da773035b2b2 --- /dev/null +++ b/src/java/org/apache/cassandra/service/replication/migration/MutationTrackingMigrationRepairResult.java @@ -0,0 +1,47 @@ +/* + * 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.replication.migration; + +import org.apache.cassandra.tcm.Epoch; + +/** + * Tracks repair eligibility for mutation tracking migration advancement. + * + * // TODO: merge this with the accord migration state + */ +public class MutationTrackingMigrationRepairResult +{ + private static final MutationTrackingMigrationRepairResult INELIGIBLE = new MutationTrackingMigrationRepairResult(Epoch.EMPTY, false); + + public final Epoch minEpoch; + public final boolean eligible; + + private MutationTrackingMigrationRepairResult(Epoch minEpoch, boolean eligible) + { + this.minEpoch = minEpoch; + this.eligible = eligible; + } + + public static MutationTrackingMigrationRepairResult fromRepair(Epoch minEpoch, boolean deadNodesExcluded, boolean isPreview) + { + if (deadNodesExcluded) return INELIGIBLE; + if (isPreview) return INELIGIBLE; + return new MutationTrackingMigrationRepairResult(minEpoch, true); + } +} diff --git a/src/java/org/apache/cassandra/service/replication/migration/MutationTrackingMigrationState.java b/src/java/org/apache/cassandra/service/replication/migration/MutationTrackingMigrationState.java new file mode 100644 index 000000000000..6b7850dad9a9 --- /dev/null +++ b/src/java/org/apache/cassandra/service/replication/migration/MutationTrackingMigrationState.java @@ -0,0 +1,361 @@ +/* + * 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.replication.migration; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import javax.annotation.Nonnull; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.dht.NormalizedRanges; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.schema.DistributedSchema; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.tcm.MetadataValue; +import org.apache.cassandra.tcm.serialization.MetadataSerializer; +import org.apache.cassandra.tcm.serialization.Version; + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; +import static org.apache.cassandra.utils.CollectionSerializers.deserializeMap; +import static org.apache.cassandra.utils.CollectionSerializers.serializeMap; +import static org.apache.cassandra.utils.CollectionSerializers.serializedMapSize; + +/** + * Cluster wide per keyspace mutation tracking migration state. + * Tracks ranges needing migration per keyspace + * Only schema changes and repair coordinators execute TCM transformations; replicas read for routing of reads/writes. + */ +public class MutationTrackingMigrationState implements MetadataValue +{ + public static final MutationTrackingMigrationState EMPTY = + new MutationTrackingMigrationState(Epoch.EMPTY, ImmutableMap.of()); + + @Nonnull + public final ImmutableMap keyspaceInfo; + + @Nonnull + public final Epoch lastModified; + + public MutationTrackingMigrationState(@Nonnull Epoch lastModified, + @Nonnull Map keyspaceInfo) + { + checkNotNull(lastModified); + checkNotNull(keyspaceInfo); + this.lastModified = lastModified; + this.keyspaceInfo = ImmutableMap.copyOf(keyspaceInfo); + } + + @Override + public MutationTrackingMigrationState withLastModified(Epoch epoch) + { + return new MutationTrackingMigrationState(epoch, keyspaceInfo); + } + + @Override + public Epoch lastModified() + { + return lastModified; + } + + private MutationTrackingMigrationState withUpdatedKeyspaceInfo(Epoch epoch, KeyspaceMigrationInfo info) + { + ImmutableMap.Builder updated = ImmutableMap.builder(); + for (Map.Entry entry : keyspaceInfo.entrySet()) + { + if (!entry.getKey().equals(info.keyspace)) + updated.put(entry.getKey(), entry.getValue()); + } + + if (info != null && !info.isComplete()) + updated.put(info.keyspace, info); + + return new MutationTrackingMigrationState(epoch, updated.build()); + } + + private MutationTrackingMigrationState withoutKeyspace(Epoch epoch, String keyspace) + { + ImmutableMap.Builder updated = ImmutableMap.builder(); + for (Map.Entry entry : keyspaceInfo.entrySet()) + { + if (!entry.getKey().equals(keyspace)) + updated.put(entry.getKey(), entry.getValue()); + } + + return new MutationTrackingMigrationState(epoch, updated.build()); + } + + /** + * Start migration for keyspace with full ring pending for all tables. + * + * @param keyspace keyspace name + * @param tableIds collection of table IDs to migrate + * @param epoch epoch for this state change + */ + public MutationTrackingMigrationState withKeyspaceMigrating(String keyspace, Collection tableIds, Epoch epoch) + { + checkNotNull(keyspace); + checkNotNull(tableIds); + checkNotNull(epoch); + + KeyspaceMigrationInfo existingInfo = keyspaceInfo.get(keyspace); + + // If migration info already exists, we need to reverse direction + if (existingInfo != null) + return withUpdatedKeyspaceInfo(epoch, existingInfo.withDirectionReversed(tableIds, epoch)); + + // Compute full ring range... + Token minimumToken = DatabaseDescriptor.getPartitioner().getMinimumToken(); + Range fullRing = new Range<>(minimumToken, minimumToken); + Set> fullRingSet = Collections.singleton(fullRing); + + // ...and assign it to each table + ImmutableMap.Builder> pendingRangesBuilder = ImmutableMap.builder(); + for (TableId tableId : tableIds) + { + pendingRangesBuilder.put(tableId, NormalizedRanges.normalizedRanges(fullRingSet)); + } + + // Create new migration info with all tables pending full ring + KeyspaceMigrationInfo newInfo = new KeyspaceMigrationInfo( + keyspace, + pendingRangesBuilder.build(), + epoch + ); + + return withUpdatedKeyspaceInfo(epoch, newInfo); + } + + /** + * Subtract the repaired ranges from table's pending set. + * Removes keyspace from state when all tables have been fully repaired. + */ + public MutationTrackingMigrationState withRangesRepairedForTable(@Nonnull String keyspace, + @Nonnull TableId tableId, + @Nonnull Collection> repairedRanges, + @Nonnull Epoch epoch) + { + checkNotNull(keyspace); + checkNotNull(tableId); + checkNotNull(repairedRanges); + checkNotNull(epoch); + + // noop if we raced with a migration completing repair + KeyspaceMigrationInfo info = keyspaceInfo.get(keyspace); + if (info == null) + return this; + + // Subtract repaired ranges from table's pending set + KeyspaceMigrationInfo updated = info.withRangesRepairedForTable(epoch, tableId, repairedRanges); + + // if all tables fully repaired, remove keyspace (migration complete) + if (updated.isComplete()) + return withoutKeyspace(epoch, keyspace); + + return withUpdatedKeyspaceInfo(epoch, updated); + } + + /** + * Remove keyspaces from migration state + */ + public MutationTrackingMigrationState dropKeyspaces(Epoch nextEpoch, @Nonnull Set removed) + { + checkNotNull(removed); + + if (keyspaceInfo.isEmpty() || Sets.intersection(keyspaceInfo.keySet(), removed).isEmpty()) + return this; + + MutationTrackingMigrationState nextState = this; + for (String keyspace : removed) + nextState = nextState.withoutKeyspace(nextEpoch, keyspace); + + return nextState; + } + + /** + * Remove dropped tables from migration states. + * Completes keyspace migration if all tables removed. + */ + public MutationTrackingMigrationState dropTables(@Nonnull Set tableIds, + @Nonnull Epoch epoch) + { + checkNotNull(tableIds); + checkNotNull(epoch); + + if (tableIds.isEmpty() || keyspaceInfo.isEmpty()) + return this; + + ImmutableMap.Builder updated = ImmutableMap.builder(); + boolean anyChanged = false; + + for (Map.Entry entry : keyspaceInfo.entrySet()) + { + String keyspace = entry.getKey(); + KeyspaceMigrationInfo info = entry.getValue(); + + // Remove dropped tables from this keyspace's migration + KeyspaceMigrationInfo newInfo = info.withTablesRemoved(tableIds); + + if (newInfo == null || newInfo.isComplete()) + { + // All tables removed - migration complete, don't add back to map + anyChanged = true; + } + else if (newInfo != info) + { + // Some tables removed + updated.put(keyspace, newInfo); + anyChanged = true; + } + else + { + // No tables removed (none were in this keyspace) + updated.put(keyspace, info); + } + } + + if (!anyChanged) + return this; + + return new MutationTrackingMigrationState(epoch, updated.build()); + } + + public KeyspaceMigrationInfo getKeyspaceInfo(String keyspace) + { + return keyspaceInfo.get(keyspace); + } + + public boolean hasMigratingKeyspaces() + { + return !keyspaceInfo.isEmpty(); + } + + /** + * Validate migration state against schema. + * Lenient for keyspaces being added/removed in same transaction. + */ + public void validateAgainstSchema(DistributedSchema schema) + { + for (Map.Entry entry : keyspaceInfo.entrySet()) + { + String keyspace = entry.getKey(); + KeyspaceMigrationInfo info = entry.getValue(); + + // Skip validation if keyspace doesn't exist - it may be being created or was dropped + KeyspaceMetadata ksm = schema.getKeyspaces().getNullable(keyspace); + if (ksm == null) + continue; + + // Validate all tables in migration exist in schema + for (TableId tableId : info.pendingRangesPerTable.keySet()) + { + TableMetadata table = schema.getTableMetadata(tableId); + checkState(table != null, + "Migration state for keyspace %s references non-existent table: %s", + keyspace, tableId); + } + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + MutationTrackingMigrationState that = (MutationTrackingMigrationState) o; + return keyspaceInfo.equals(that.keyspaceInfo) && + lastModified.equals(that.lastModified); + } + + @Override + public int hashCode() + { + return Objects.hash(keyspaceInfo, lastModified); + } + + @Override + public String toString() + { + return "MutationTrackingMigrationState{" + + "keyspaceInfo=" + keyspaceInfo.keySet() + + ", lastModified=" + lastModified + + '}'; + } + + private static final MetadataSerializer stringSerializer = new MetadataSerializer<>() + { + @Override + public void serialize(String t, DataOutputPlus out, Version version) throws IOException + { + out.writeUTF(t); + } + + @Override + public String deserialize(DataInputPlus in, Version version) throws IOException + { + return in.readUTF(); + } + + @Override + public long serializedSize(String t, Version version) + { + return TypeSizes.sizeof(t); + } + }; + + public static final MetadataSerializer serializer = new MetadataSerializer<>() + { + @Override + public void serialize(MutationTrackingMigrationState t, DataOutputPlus out, Version version) throws IOException + { + Epoch.serializer.serialize(t.lastModified, out, version); + serializeMap(t.keyspaceInfo, out, version, stringSerializer, KeyspaceMigrationInfo.serializer); + } + + @Override + public MutationTrackingMigrationState deserialize(DataInputPlus in, Version version) throws IOException + { + Epoch lastModified = Epoch.serializer.deserialize(in, version); + Map keyspaceInfo = deserializeMap(in, version, stringSerializer, KeyspaceMigrationInfo.serializer, Maps::newHashMapWithExpectedSize); + return new MutationTrackingMigrationState(lastModified, keyspaceInfo); + } + + @Override + public long serializedSize(MutationTrackingMigrationState t, Version version) + { + return Epoch.serializer.serializedSize(t.lastModified, version) + + serializedMapSize(t.keyspaceInfo, version, stringSerializer, KeyspaceMigrationInfo.serializer); + } + }; +} diff --git a/src/java/org/apache/cassandra/service/replication/migration/MutationTrackingRepairHandler.java b/src/java/org/apache/cassandra/service/replication/migration/MutationTrackingRepairHandler.java new file mode 100644 index 000000000000..6bdcd20946fd --- /dev/null +++ b/src/java/org/apache/cassandra/service/replication/migration/MutationTrackingRepairHandler.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.replication.migration; + +import java.util.Collection; + +import com.google.common.util.concurrent.FutureCallback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.repair.RepairResult; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.transformations.AdvanceMutationTrackingMigration; + +/** + * Repair callback handler for mutation tracking migration. + * Registered on repair coordinator to advance migration state on successful repairs. + */ +public class MutationTrackingRepairHandler +{ + private static final Logger logger = LoggerFactory.getLogger(MutationTrackingRepairHandler.class); + + public static final FutureCallback completedRepairJobHandler = + new FutureCallback<>() + { + @Override + public void onSuccess(RepairResult repairResult) + { + try + { + String keyspace = repairResult.desc.keyspace; + String tableName = repairResult.desc.columnFamily; + Collection> repairedRanges = repairResult.desc.ranges; + + ClusterMetadata clusterMetadata = ClusterMetadata.current(); + + // Check if keyspace is migrating + KeyspaceMigrationInfo migrationInfo = clusterMetadata.mutationTrackingMigrationState.getKeyspaceInfo(keyspace); + + if (migrationInfo == null) + { + return; + } + + // Get table metadata + TableMetadata tableMetadata = clusterMetadata.schema.getKeyspaceMetadata(keyspace).getTableOrViewNullable(tableName); + + if (tableMetadata == null) + { + logger.warn("Repair completed for unknown table {}.{}, cannot advance migration", + keyspace, tableName); + return; + } + + if (migrationInfo.getPendingRangesForTable(tableMetadata.id).isEmpty()) + { + // Table already fully migrated + return; + } + + // Epoch eligibility check: Only count repairs started after the migration started + if (repairResult.mutationTrackingMigrationRepairResult.minEpoch.isBefore(migrationInfo.startedAtEpoch)) + { + logger.debug("Repair completed for {}.{} but current epoch {} is before migration start epoch {}, ignoring", + keyspace, tableName, clusterMetadata.epoch, migrationInfo.startedAtEpoch); + return; + } + + if (!repairResult.mutationTrackingMigrationRepairResult.eligible) + { + logger.debug("Repair completed for {}.{} but repair is ineligible for mutation tracking migration, ignoring", + keyspace, tableName); + return; + } + + logger.info("Repair completed for {}.{}, proposing migration advancement for {} ranges", + keyspace, tableName, repairedRanges.size()); + + ClusterMetadataService.instance().commit( + new AdvanceMutationTrackingMigration(keyspace, tableMetadata.id, repairedRanges)); + } + catch (Exception e) + { + logger.error("Error handling repair completion for mutation tracking migration", e); + } + } + + @Override + public void onFailure(Throwable throwable) + { + // noop + } + }; +} diff --git a/src/java/org/apache/cassandra/tcm/ClusterMetadata.java b/src/java/org/apache/cassandra/tcm/ClusterMetadata.java index d57df4de5fca..5da217399d52 100644 --- a/src/java/org/apache/cassandra/tcm/ClusterMetadata.java +++ b/src/java/org/apache/cassandra/tcm/ClusterMetadata.java @@ -62,6 +62,7 @@ import org.apache.cassandra.service.accord.AccordTopology; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; import org.apache.cassandra.service.consensus.migration.TableMigrationState; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationState; import org.apache.cassandra.tcm.extensions.ExtensionKey; import org.apache.cassandra.tcm.extensions.ExtensionValue; import org.apache.cassandra.service.accord.AccordFastPath; @@ -83,6 +84,7 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.LINE_SEPARATOR; import static org.apache.cassandra.db.TypeSizes.sizeof; import static org.apache.cassandra.tcm.serialization.Version.MIN_ACCORD_VERSION; +import static org.apache.cassandra.tcm.serialization.Version.MIN_MUTATION_TRACKING_VERSION; public class ClusterMetadata { @@ -102,6 +104,7 @@ public class ClusterMetadata public final LockedRanges lockedRanges; public final InProgressSequences inProgressSequences; public final ConsensusMigrationState consensusMigrationState; + public final MutationTrackingMigrationState mutationTrackingMigrationState; public final ImmutableMap, ExtensionValue> extensions; public final AccordStaleReplicas accordStaleReplicas; @@ -139,6 +142,7 @@ public ClusterMetadata(IPartitioner partitioner, Directory directory, Distribute LockedRanges.EMPTY, InProgressSequences.EMPTY, ConsensusMigrationState.EMPTY, + MutationTrackingMigrationState.EMPTY, ImmutableMap.of(), AccordStaleReplicas.EMPTY); } @@ -153,6 +157,7 @@ public ClusterMetadata(Epoch epoch, LockedRanges lockedRanges, InProgressSequences inProgressSequences, ConsensusMigrationState consensusMigrationState, + MutationTrackingMigrationState mutationTrackingMigrationState, Map, ExtensionValue> extensions, AccordStaleReplicas accordStaleReplicas) { @@ -167,6 +172,7 @@ public ClusterMetadata(Epoch epoch, lockedRanges, inProgressSequences, consensusMigrationState, + mutationTrackingMigrationState, extensions, accordStaleReplicas); } @@ -182,6 +188,7 @@ private ClusterMetadata(int metadataIdentifier, LockedRanges lockedRanges, InProgressSequences inProgressSequences, ConsensusMigrationState consensusMigrationState, + MutationTrackingMigrationState mutationTrackingMigrationState, Map, ExtensionValue> extensions, AccordStaleReplicas accordStaleReplicas) { @@ -200,6 +207,7 @@ private ClusterMetadata(int metadataIdentifier, this.lockedRanges = lockedRanges; this.inProgressSequences = inProgressSequences; this.consensusMigrationState = consensusMigrationState; + this.mutationTrackingMigrationState = mutationTrackingMigrationState; this.extensions = ImmutableMap.copyOf(extensions); this.locator = Locator.usingDirectory(directory); this.accordStaleReplicas = accordStaleReplicas; @@ -258,6 +266,7 @@ public ClusterMetadata forceEpoch(Epoch epoch) capLastModified(lockedRanges, epoch), capLastModified(inProgressSequences, epoch), capLastModified(consensusMigrationState, epoch), + capLastModified(mutationTrackingMigrationState, epoch), capLastModified(extensions, epoch), capLastModified(accordStaleReplicas, epoch)); } @@ -281,6 +290,7 @@ public ClusterMetadata initializeClusterIdentifier(int clusterIdentifier) lockedRanges, inProgressSequences, consensusMigrationState, + mutationTrackingMigrationState, extensions, accordStaleReplicas); } @@ -409,6 +419,7 @@ public static class Transformer private LockedRanges lockedRanges; private InProgressSequences inProgressSequences; private ConsensusMigrationState consensusMigrationState; + private MutationTrackingMigrationState mutationTrackingMigrationState; private final Map, ExtensionValue> extensions; private final Set modifiedKeys; private AccordStaleReplicas accordStaleReplicas; @@ -426,6 +437,7 @@ private Transformer(ClusterMetadata metadata, Epoch epoch) this.lockedRanges = metadata.lockedRanges; this.inProgressSequences = metadata.inProgressSequences; this.consensusMigrationState = metadata.consensusMigrationState; + this.mutationTrackingMigrationState = metadata.mutationTrackingMigrationState; extensions = new HashMap<>(metadata.extensions); modifiedKeys = new HashSet<>(); accordStaleReplicas = metadata.accordStaleReplicas; @@ -620,6 +632,12 @@ public Transformer with(ConsensusMigrationState consensusMigrationState) return this; } + public Transformer with(MutationTrackingMigrationState mutationTrackingMigrationState) + { + this.mutationTrackingMigrationState = mutationTrackingMigrationState; + return this; + } + public Transformer with(ExtensionKey key, ExtensionValue obj) { if (MetadataKeys.CORE_METADATA.containsKey(key)) @@ -727,11 +745,22 @@ public Transformed build() consensusMigrationState = consensusMigrationState.withLastModified(epoch); } + if (mutationTrackingMigrationState != base.mutationTrackingMigrationState) + { + modifiedKeys.add(MetadataKeys.MUTATION_TRACKING_MIGRATION_STATE); + mutationTrackingMigrationState = mutationTrackingMigrationState.withLastModified(epoch); + } + if (consensusMigrationState != base.consensusMigrationState || schema != base.schema) { consensusMigrationState.validateAgainstSchema(schema); } + if (mutationTrackingMigrationState != base.mutationTrackingMigrationState || schema != base.schema) + { + mutationTrackingMigrationState.validateAgainstSchema(schema); + } + return new Transformed(new ClusterMetadata(base.metadataIdentifier, epoch, partitioner, @@ -743,6 +772,7 @@ public Transformed build() lockedRanges, inProgressSequences, consensusMigrationState, + mutationTrackingMigrationState, extensions, accordStaleReplicas), ImmutableSet.copyOf(modifiedKeys)); @@ -761,6 +791,7 @@ public ClusterMetadata buildForGossipMode() lockedRanges, inProgressSequences, consensusMigrationState, + mutationTrackingMigrationState, extensions, accordStaleReplicas); } @@ -1026,6 +1057,8 @@ public void serialize(ClusterMetadata metadata, DataOutputPlus out, Version vers ConsensusMigrationState.serializer.serialize(metadata.consensusMigrationState, out, version); AccordStaleReplicas.serializer.serialize(metadata.accordStaleReplicas, out, version); } + if (version.isAtLeast(MIN_MUTATION_TRACKING_VERSION)) + MutationTrackingMigrationState.serializer.serialize(metadata.mutationTrackingMigrationState, out, version); LockedRanges.serializer.serialize(metadata.lockedRanges, out, version); InProgressSequences.serializer.serialize(metadata.inProgressSequences, out, version); @@ -1066,6 +1099,7 @@ public ClusterMetadata deserialize(DataInputPlus in, Version version) throws IOE AccordFastPath accordFastPath; ConsensusMigrationState consensusMigrationState; + MutationTrackingMigrationState mutationTrackingMigrationState; AccordStaleReplicas staleReplicas; if (version.isAtLeast(MIN_ACCORD_VERSION)) @@ -1081,6 +1115,15 @@ public ClusterMetadata deserialize(DataInputPlus in, Version version) throws IOE staleReplicas = AccordStaleReplicas.EMPTY; } + if (version.isAtLeast(MIN_MUTATION_TRACKING_VERSION)) + { + mutationTrackingMigrationState = MutationTrackingMigrationState.serializer.deserialize(in, version); + } + else + { + mutationTrackingMigrationState = MutationTrackingMigrationState.EMPTY; + } + LockedRanges lockedRanges = LockedRanges.serializer.deserialize(in, version); InProgressSequences ips = InProgressSequences.serializer.deserialize(in, version); int items = in.readInt(); @@ -1103,6 +1146,7 @@ public ClusterMetadata deserialize(DataInputPlus in, Version version) throws IOE lockedRanges, ips, consensusMigrationState, + mutationTrackingMigrationState, extensions, staleReplicas); } @@ -1129,9 +1173,13 @@ public long serializedSize(ClusterMetadata metadata, Version version) { size += AccordFastPath.serializer.serializedSize(metadata.accordFastPath, version) + ConsensusMigrationState.serializer.serializedSize(metadata.consensusMigrationState, version) + + AccordStaleReplicas.serializer.serializedSize(metadata.accordStaleReplicas, version); } + if (version.isAtLeast(MIN_MUTATION_TRACKING_VERSION)) + size += MutationTrackingMigrationState.serializer.serializedSize(metadata.mutationTrackingMigrationState, version); + size += LockedRanges.serializer.serializedSize(metadata.lockedRanges, version) + InProgressSequences.serializer.serializedSize(metadata.inProgressSequences, version); diff --git a/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java b/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java index 23ba4385e477..5664cf168b52 100644 --- a/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java +++ b/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java @@ -38,6 +38,7 @@ import org.apache.cassandra.service.accord.AccordFastPath; import org.apache.cassandra.service.accord.AccordStaleReplicas; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationState; import org.apache.cassandra.tcm.membership.Directory; import org.apache.cassandra.tcm.ownership.DataPlacements; import org.apache.cassandra.tcm.ownership.TokenMap; @@ -309,6 +310,7 @@ public static void empty(Keyspaces keyspaces) LockedRanges.EMPTY, InProgressSequences.EMPTY, ConsensusMigrationState.EMPTY, + MutationTrackingMigrationState.EMPTY, Collections.emptyMap(), AccordStaleReplicas.EMPTY); diff --git a/src/java/org/apache/cassandra/tcm/MetadataKeys.java b/src/java/org/apache/cassandra/tcm/MetadataKeys.java index 0be621b20189..721f36adf5a3 100644 --- a/src/java/org/apache/cassandra/tcm/MetadataKeys.java +++ b/src/java/org/apache/cassandra/tcm/MetadataKeys.java @@ -45,6 +45,7 @@ public class MetadataKeys public static final MetadataKey LOCKED_RANGES = make(CORE_NS, "sequences", "locked_ranges"); public static final MetadataKey IN_PROGRESS_SEQUENCES = make(CORE_NS, "sequences", "in_progress"); public static final MetadataKey CONSENSUS_MIGRATION_STATE = make(CORE_NS, "consensus", "migration_state"); + public static final MetadataKey MUTATION_TRACKING_MIGRATION_STATE = make(CORE_NS, "mutation_tracking", "migration_state"); public static final ImmutableMap>> CORE_METADATA = ImmutableMap.>>builder() @@ -57,6 +58,7 @@ public class MetadataKeys .put(ACCORD_FAST_PATH, cm -> cm.accordFastPath) .put(ACCORD_STALE_REPLICAS, cm -> cm.accordStaleReplicas) .put(CONSENSUS_MIGRATION_STATE, cm -> cm.consensusMigrationState) + .put(MUTATION_TRACKING_MIGRATION_STATE, cm -> cm.mutationTrackingMigrationState) .build(); public static MetadataKey make(String...parts) diff --git a/src/java/org/apache/cassandra/tcm/StubClusterMetadataService.java b/src/java/org/apache/cassandra/tcm/StubClusterMetadataService.java index 6036554e0d00..a06668986726 100644 --- a/src/java/org/apache/cassandra/tcm/StubClusterMetadataService.java +++ b/src/java/org/apache/cassandra/tcm/StubClusterMetadataService.java @@ -31,6 +31,7 @@ import org.apache.cassandra.service.accord.AccordFastPath; import org.apache.cassandra.service.accord.AccordStaleReplicas; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationState; import org.apache.cassandra.tcm.Commit.Replicator; import org.apache.cassandra.tcm.log.Entry; import org.apache.cassandra.tcm.log.LocalLog; @@ -183,6 +184,7 @@ public StubClusterMetadataService build() LockedRanges.EMPTY, InProgressSequences.EMPTY, ConsensusMigrationState.EMPTY, + MutationTrackingMigrationState.EMPTY, ImmutableMap.of(), AccordStaleReplicas.EMPTY); } diff --git a/src/java/org/apache/cassandra/tcm/Transformation.java b/src/java/org/apache/cassandra/tcm/Transformation.java index 928ce59aca12..4994dff509ab 100644 --- a/src/java/org/apache/cassandra/tcm/Transformation.java +++ b/src/java/org/apache/cassandra/tcm/Transformation.java @@ -40,6 +40,7 @@ import org.apache.cassandra.tcm.serialization.Version; import org.apache.cassandra.tcm.transformations.AccordMarkRejoining; import org.apache.cassandra.tcm.transformations.AccordMarkStale; +import org.apache.cassandra.tcm.transformations.AdvanceMutationTrackingMigration; import org.apache.cassandra.tcm.transformations.AlterSchema; import org.apache.cassandra.tcm.transformations.AlterTopology; import org.apache.cassandra.tcm.transformations.Assassinate; @@ -250,6 +251,7 @@ enum Kind ACCORD_MARK_REJOINING(40, () -> AccordMarkRejoining.serializer), PREPARE_DROP_ACCORD_TABLE(41, () -> PrepareDropAccordTable.serializer), FINISH_DROP_ACCORD_TABLE(42, () -> FinishDropAccordTable.serializer), + ADVANCE_MUTATION_TRACKING_MIGRATION(43, () -> AdvanceMutationTrackingMigration.serializer), ; private final Supplier> serializer; diff --git a/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java b/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java index f569a9d1cbc5..86701292307e 100644 --- a/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java +++ b/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java @@ -56,6 +56,7 @@ import org.apache.cassandra.service.StorageService; import org.apache.cassandra.service.accord.AccordStaleReplicas; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationState; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.MultiStepOperation; @@ -301,6 +302,7 @@ public static ClusterMetadata emptyWithSchemaFromSystemTables(Set allKno LockedRanges.EMPTY, InProgressSequences.EMPTY, ConsensusMigrationState.EMPTY, + MutationTrackingMigrationState.EMPTY, Collections.emptyMap(), AccordStaleReplicas.EMPTY); } @@ -391,6 +393,7 @@ public static ClusterMetadata fromEndpointStates(Map values = new HashMap<>(); static diff --git a/src/java/org/apache/cassandra/tcm/transformations/AdvanceMutationTrackingMigration.java b/src/java/org/apache/cassandra/tcm/transformations/AdvanceMutationTrackingMigration.java new file mode 100644 index 000000000000..0416de67678c --- /dev/null +++ b/src/java/org/apache/cassandra/tcm/transformations/AdvanceMutationTrackingMigration.java @@ -0,0 +1,144 @@ +/* + * 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.tcm.transformations; + +import java.io.IOException; +import java.util.Collection; +import javax.annotation.Nonnull; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.service.replication.migration.KeyspaceMigrationInfo; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationState; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.Transformation; +import org.apache.cassandra.tcm.sequences.LockedRanges; +import org.apache.cassandra.tcm.serialization.AsymmetricMetadataSerializer; +import org.apache.cassandra.tcm.serialization.Version; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.cassandra.exceptions.ExceptionCode.INVALID; +import static org.apache.cassandra.tcm.ClusterMetadata.Transformer; +import static org.apache.cassandra.utils.CollectionSerializers.deserializeList; +import static org.apache.cassandra.utils.CollectionSerializers.serializeCollection; +import static org.apache.cassandra.utils.CollectionSerializers.serializedCollectionSize; + +/** + * Transformation to mark ranges as migrated for a keyspace. + * + * Called by repair coordinator callback to report completed ranges to TCM. + * Subtracts completed ranges from pendingRangesPerTable and automatically removes + * keyspace from migration state when all tables are fully repaired (migration complete). + */ +public class AdvanceMutationTrackingMigration implements Transformation +{ + private static final Logger logger = LoggerFactory.getLogger(AdvanceMutationTrackingMigration.class); + public static final Serializer serializer = new Serializer(); + + @Nonnull + public final String keyspace; + + @Nonnull + public final TableId tableId; + + @Nonnull + public final Collection> repairedRanges; + + public AdvanceMutationTrackingMigration(@Nonnull String keyspace, + @Nonnull TableId tableId, + @Nonnull Collection> repairedRanges) + { + checkNotNull(keyspace, "keyspace should not be null"); + checkNotNull(tableId, "tableId should not be null"); + checkArgument(repairedRanges != null && !repairedRanges.isEmpty(), + "repairedRanges should not be null/empty"); + this.keyspace = keyspace; + this.tableId = tableId; + this.repairedRanges = repairedRanges; + } + + @Override + public Kind kind() + { + return Kind.ADVANCE_MUTATION_TRACKING_MIGRATION; + } + + @Override + public Result execute(ClusterMetadata prev) + { + KeyspaceMigrationInfo ksInfo = prev.mutationTrackingMigrationState.getKeyspaceInfo(keyspace); + + if (ksInfo == null) + { + logger.warn("Attempted to advance mutation tracking migration for keyspace {} table {} which is not migrating", keyspace, tableId); + return new Rejected(INVALID, String.format("Keyspace %s is not migrating", keyspace)); + } + + Transformer transformer = prev.transformer(); + + // Subtract repaired ranges from table's pending set, auto-removes keyspace if all tables complete + MutationTrackingMigrationState newState = prev.mutationTrackingMigrationState + .withRangesRepairedForTable(keyspace, tableId, repairedRanges, transformer.epoch()); + + logger.info("Advanced mutation tracking migration for keyspace {}, table {}: {} ranges repaired", + keyspace, tableId, repairedRanges.size()); + + return Transformation.success( + transformer.with(newState), + LockedRanges.AffectedRanges.EMPTY); + } + + static class Serializer implements AsymmetricMetadataSerializer + { + @Override + public void serialize(Transformation t, DataOutputPlus out, Version version) throws IOException + { + AdvanceMutationTrackingMigration v = (AdvanceMutationTrackingMigration) t; + out.writeUTF(v.keyspace); + TableId.serializer.serialize(v.tableId, out, version.asInt()); + serializeCollection(v.repairedRanges, out, version, Range.serializer); + } + + @Override + public AdvanceMutationTrackingMigration deserialize(DataInputPlus in, Version version) throws IOException + { + String keyspace = in.readUTF(); + TableId tableId = TableId.serializer.deserialize(in, version.asInt()); + Collection> repairedRanges = deserializeList(in, version, Range.serializer); + return new AdvanceMutationTrackingMigration(keyspace, tableId, repairedRanges); + } + + @Override + public long serializedSize(Transformation t, Version version) + { + AdvanceMutationTrackingMigration v = (AdvanceMutationTrackingMigration) t; + return TypeSizes.sizeof(v.keyspace) + + TableId.serializer.serializedSize(v.tableId, version.asInt()) + + serializedCollectionSize(v.repairedRanges, version, Range.serializer); + } + } +} diff --git a/src/java/org/apache/cassandra/tcm/transformations/AlterSchema.java b/src/java/org/apache/cassandra/tcm/transformations/AlterSchema.java index 09c5b68bb5f1..3dbcae7504f5 100644 --- a/src/java/org/apache/cassandra/tcm/transformations/AlterSchema.java +++ b/src/java/org/apache/cassandra/tcm/transformations/AlterSchema.java @@ -19,6 +19,7 @@ package org.apache.cassandra.tcm.transformations; import java.io.IOException; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -51,6 +52,7 @@ import org.apache.cassandra.schema.KeyspaceMetadata.KeyspaceDiff; import org.apache.cassandra.schema.Keyspaces; import org.apache.cassandra.schema.ReplicationParams; +import org.apache.cassandra.schema.ReplicationType; import org.apache.cassandra.schema.SchemaTransformation; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; @@ -58,6 +60,7 @@ import org.apache.cassandra.schema.ViewMetadata; import org.apache.cassandra.schema.Views; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationState; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadata.Transformer; import org.apache.cassandra.tcm.ClusterMetadataService; @@ -73,6 +76,7 @@ import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static java.lang.String.format; import static org.apache.cassandra.cql3.statements.schema.AlterSchemaStatement.NO_EXECUTION_TIMESTAMP; import static org.apache.cassandra.exceptions.ExceptionCode.ALREADY_EXISTS; import static org.apache.cassandra.exceptions.ExceptionCode.CONFIG_ERROR; @@ -217,7 +221,7 @@ public final Result execute(ClusterMetadata prev) logger.debug("Schema change affects data placements, relevant keyspaces: {}", affectsPlacements); if (!prev.lockedRanges.locked.isEmpty()) return new Rejected(INVALID, - String.format("The requested schema changes cannot be executed as they conflict " + + format("The requested schema changes cannot be executed as they conflict " + "with ongoing range movements. The changes for keyspaces %s are blocked " + "by the locked ranges %s", affectsPlacements.stream().map(k -> k.name).collect(Collectors.joining(",", "[", "]")), @@ -248,6 +252,7 @@ public final Result execute(ClusterMetadata prev) next = next.with(newPlacementsBuilder.build()); } next = maybeUpdateConsensusMigrationState(prev.consensusMigrationState, next, diff.altered, diff.dropped); + next = maybeUpdateMutationTrackingMigrationState(nextEpoch, prev.mutationTrackingMigrationState, next, diff.altered, diff.dropped); return Transformation.success(next, LockedRanges.AffectedRanges.EMPTY); } @@ -332,6 +337,64 @@ public static Transformer maybeUpdateConsensusMigrationState(ConsensusMigrationS return next; } + /** + * Auto-start mutation tracking migration when keyspace replication type changes. + * Detects transitions between tracked and untracked replication and initializes + * migration state accordingly. + * Also handles removing dropped tables and keyspaces from migration state. + */ + public static Transformer maybeUpdateMutationTrackingMigrationState(Epoch nextEpoch, + MutationTrackingMigrationState prev, + Transformer next, + ImmutableList altered, + Keyspaces dropped) + { + MutationTrackingMigrationState migrationState = prev; + + // Handle dropped keyspaces - remove their migration state entirely + if (!dropped.isEmpty()) + { + Set droppedKeyspaceNames = dropped.stream() + .map(ks -> ks.name) + .collect(Collectors.toSet()); + migrationState = migrationState.dropKeyspaces(nextEpoch, droppedKeyspaceNames); + } + + // Handle dropped tables from altered keyspaces + Set droppedTableIds = altered.stream() + .flatMap(diff -> diff.tables.dropped.stream().map(TableMetadata::id)) + .collect(Collectors.toSet()); + + if (!droppedTableIds.isEmpty()) + migrationState = migrationState.dropTables(droppedTableIds, nextEpoch); + + // Handle keyspace replication type changes (new migrations or reversals) + for (KeyspaceDiff diff : altered) + { + ReplicationType beforeType = diff.before.params.replicationType; + ReplicationType afterType = diff.after.params.replicationType; + + // Check if replication type changed + if (beforeType != afterType) + { + // Auto-start migration for this keyspace + logger.info("Auto-starting mutation tracking migration for keyspace {} (replication_type={})", + diff.after.name, afterType); + + Collection tableIds = diff.after.tables.stream() + .map(table -> table.id) + .collect(Collectors.toList()); + + migrationState = migrationState.withKeyspaceMigrating(diff.after.name, tableIds, nextEpoch); + } + } + + if (migrationState != prev) + next = next.with(migrationState); + + return next; + } + private static Iterable normaliseTableEpochs(Epoch nextEpoch, Stream tables) { return tables.map(tm -> tm.epoch.is(nextEpoch) diff --git a/test/distributed/org/apache/cassandra/distributed/test/MutationTrackingMigrationTest.java b/test/distributed/org/apache/cassandra/distributed/test/MutationTrackingMigrationTest.java new file mode 100644 index 000000000000..e4b352101a8f --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/MutationTrackingMigrationTest.java @@ -0,0 +1,583 @@ +/* + * 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.distributed.test; + +import java.io.IOException; +import java.util.concurrent.TimeoutException; + +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.ServerTestUtils; +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.ICoordinator; +import org.apache.cassandra.replication.MutationJournal; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.service.replication.migration.KeyspaceMigrationInfo; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationState; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; + +import static java.lang.String.format; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +/** + * Tests for mutation tracking migration between tracked and untracked replication types. + */ +public class MutationTrackingMigrationTest extends TestBaseImpl +{ + private static final String TEST_TABLE = "tbl"; + private static final int NUM_NODES = 3; + + private static Cluster SHARED_CLUSTER; + private static ICoordinator coordinator; + + private enum ExpectedKeyspaceState + { + UNTRACKED, + MIGRATING_TO_TRACKED, + MIGRATING_TO_UNTRACKED, + TRACKED, + DROPPED + } + + @BeforeClass + public static void setupClass() throws IOException + { + ServerTestUtils.daemonInitialization(); + CassandraRelevantProperties.SYSTEM_TRACES_DEFAULT_RF.setInt(3); + + SHARED_CLUSTER = init(Cluster.build(NUM_NODES) + .withConfig(config -> config.with(Feature.NETWORK) + .with(Feature.GOSSIP)) + .start()); + + coordinator = SHARED_CLUSTER.coordinator(1); + } + + /** + * Wait for all nodes to catch up to the epoch of the given node + */ + private static void waitForEpochOf(Cluster cluster, int node) + { + long epoch = cluster.get(node).callOnInstance(() -> ClusterMetadata.current().epoch.getEpoch()); + + for (int nodeId = 1; nodeId <= NUM_NODES; nodeId++) + { + cluster.get(nodeId).runOnInstance(() -> { + try + { + ClusterMetadataService.instance().awaitAtLeast(Epoch.create(epoch)); + } + catch (InterruptedException e) + { + throw new UncheckedInterruptedException(e); + } + catch (TimeoutException e) + { + throw new RuntimeException(e); + } + }); + } + } + + private static int countJournalEntries() + { + return SHARED_CLUSTER.get(1).callOnInstance(() -> { + try + { + int[] count = new int[1]; + + MutationJournal.instance.snapshot().readAll((segment, position, key, buffer, version) -> { + count[0]++; + }); + + return count[0]; + } + catch (Exception e) + { + throw new RuntimeException("Failed to count journal entries", e); + } + }); + } + + /** + * Verify migration state on all nodes matches expected state. + */ + private void verifyKeyspaceState(String keyspace, ExpectedKeyspaceState expectedState) throws Exception + { + for (int nodeId = 1; nodeId <= NUM_NODES; nodeId++) + { + SHARED_CLUSTER.get(nodeId).runOnInstance(() -> { + ClusterMetadata metadata = ClusterMetadata.current(); + KeyspaceMetadata ksm = expectedState != ExpectedKeyspaceState.DROPPED ? metadata.schema.getKeyspaceMetadata(keyspace) : null; + MutationTrackingMigrationState migrationState = metadata.mutationTrackingMigrationState; + KeyspaceMigrationInfo migrationInfo = migrationState.getKeyspaceInfo(keyspace); + + switch (expectedState) + { + case UNTRACKED: + assertTrue(!ksm.params.replicationType.isTracked()); + assertNull(migrationInfo); + break; + + case MIGRATING_TO_TRACKED: + assertTrue(ksm.params.replicationType.isTracked()); + assertNotNull(migrationInfo); + break; + + case MIGRATING_TO_UNTRACKED: + assertTrue(!ksm.params.replicationType.isTracked()); + assertNotNull(migrationInfo); + break; + + case TRACKED: + assertTrue(ksm.params.replicationType.isTracked()); + assertNull(migrationInfo); + break; + case DROPPED: + assertNull(migrationInfo); + break; + default: + throw new AssertionError("Unexpected state: " + expectedState); + + } + }); + } + } + + @Test + public void testUntrackedToTrackedMigration() throws Exception + { + String testKeyspace = "untracked_to_tracked_test"; + + // untracked keyspace + coordinator.execute(format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='untracked'", + testKeyspace), + ConsistencyLevel.ALL); + + + coordinator.execute(format("CREATE TABLE %s.%s (pk int PRIMARY KEY, value text)", testKeyspace, TEST_TABLE), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.UNTRACKED); + + long journalEntriesBefore = countJournalEntries(); + + for (int i = 0; i < 100; i++) + { + coordinator.execute(format("INSERT INTO %s.%s (pk, value) VALUES (%d, 'initial_%d')", + testKeyspace, TEST_TABLE, i, i), + ConsistencyLevel.QUORUM); + } + + // no journal entries written while untracked + long journalEntriesAfterUntracked = countJournalEntries(); + assertEquals(journalEntriesBefore, journalEntriesAfterUntracked); + + Object[][] initialResults = coordinator.execute(format("SELECT * FROM %s.%s", testKeyspace, TEST_TABLE), + ConsistencyLevel.QUORUM); + assertEquals(100, initialResults.length); + + // start migration to tracked replication + coordinator.execute(format("ALTER KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked'", + testKeyspace), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.MIGRATING_TO_TRACKED); + + long journalEntriesBeforeMigrationWrites = countJournalEntries(); + + for (int i = 100; i < 200; i++) + { + coordinator.execute(format("INSERT INTO %s.%s (pk, value) VALUES (%d, 'migration_%d')", + testKeyspace, TEST_TABLE, i, i), + ConsistencyLevel.QUORUM); + } + + // writes should be tracked during migration + long journalEntriesAfterMigrationWrites = countJournalEntries(); + assertTrue(journalEntriesAfterMigrationWrites > journalEntriesBeforeMigrationWrites); + + // complete migration + SHARED_CLUSTER.get(1).nodetoolResult("repair", testKeyspace, TEST_TABLE).asserts().success(); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.TRACKED); + + long journalEntriesBeforeTracked = countJournalEntries(); + + for (int i = 200; i < 210; i++) + { + coordinator.execute(format("INSERT INTO %s.%s (pk, value) VALUES (%d, 'tracked_%d')", + testKeyspace, TEST_TABLE, i, i), + ConsistencyLevel.QUORUM); + } + + // writes should also be tracked after migration + long journalEntriesAfterTracked = countJournalEntries(); + assertTrue(journalEntriesAfterTracked > journalEntriesBeforeTracked); + + Object[][] finalResults = coordinator.execute(format("SELECT * FROM %s.%s", testKeyspace, TEST_TABLE), + ConsistencyLevel.QUORUM); + assertEquals(210, finalResults.length); + + Object[][] initialRecord = coordinator.execute(format("SELECT value FROM %s.%s WHERE pk = 50", testKeyspace, TEST_TABLE), + ConsistencyLevel.QUORUM); + assertEquals("initial_50", initialRecord[0][0]); + + Object[][] migrationRecord = coordinator.execute(format("SELECT value FROM %s.%s WHERE pk = 150", testKeyspace, TEST_TABLE), + ConsistencyLevel.QUORUM); + assertEquals("migration_150", migrationRecord[0][0]); + } + + @Test + public void testTrackedToUntrackedMigration() throws Exception + { + String testKeyspace = "tracked_to_untracked_test"; + + // tracked keyspace + coordinator.execute(format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked'", + testKeyspace), + ConsistencyLevel.ALL); + + coordinator.execute(format("CREATE TABLE %s.%s (pk int PRIMARY KEY, value text)", testKeyspace, TEST_TABLE), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.TRACKED); + + long journalEntriesBefore = countJournalEntries(); + + for (int i = 0; i < 100; i++) + { + coordinator.execute(format("INSERT INTO %s.%s (pk, value) VALUES (%d, 'initial_%d')", + testKeyspace, TEST_TABLE, i, i), + ConsistencyLevel.QUORUM); + } + + // writes should be tracked before migration + long journalEntriesAfterTracked = countJournalEntries(); + assertTrue(journalEntriesAfterTracked > journalEntriesBefore); + + Object[][] initialResults = coordinator.execute(format("SELECT * FROM %s.%s", testKeyspace, TEST_TABLE), + ConsistencyLevel.QUORUM); + assertEquals(100, initialResults.length); + + // start migration to untracked replication + coordinator.execute(format("ALTER KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='untracked'", + testKeyspace), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.MIGRATING_TO_UNTRACKED); + + long journalEntriesBeforeMigrationWrites = countJournalEntries(); + + // Write more data during migration + for (int i = 100; i < 200; i++) + { + coordinator.execute(format("INSERT INTO %s.%s (pk, value) VALUES (%d, 'migration_%d')", + testKeyspace, TEST_TABLE, i, i), + ConsistencyLevel.QUORUM); + } + + // writes should also be tracked during migration + long journalEntriesAfterMigrationWrites = countJournalEntries(); + assertTrue("Migration writes should still create journal entries (tracked mechanism still active)", + journalEntriesAfterMigrationWrites > journalEntriesBeforeMigrationWrites); + + // complete migration + SHARED_CLUSTER.get(1).nodetoolResult("repair", testKeyspace, TEST_TABLE).asserts().success(); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.UNTRACKED); + + long journalEntriesBeforeUntracked = countJournalEntries(); + + for (int i = 200; i < 210; i++) + { + coordinator.execute(format("INSERT INTO %s.%s (pk, value) VALUES (%d, 'untracked_%d')", + testKeyspace, TEST_TABLE, i, i), + ConsistencyLevel.QUORUM); + } + + // but they should not be tracked after migration + long journalEntriesAfterUntracked = countJournalEntries(); + assertEquals("Post-migration untracked writes should NOT create journal entries", + journalEntriesBeforeUntracked, journalEntriesAfterUntracked); + + Object[][] finalResults = coordinator.execute(format("SELECT * FROM %s.%s", testKeyspace, TEST_TABLE), + ConsistencyLevel.QUORUM); + assertEquals(210, finalResults.length); + + Object[][] initialRecord = coordinator.execute(format("SELECT value FROM %s.%s WHERE pk = 50", testKeyspace, TEST_TABLE), + ConsistencyLevel.QUORUM); + assertEquals("initial_50", initialRecord[0][0]); + + Object[][] migrationRecord = coordinator.execute(format("SELECT value FROM %s.%s WHERE pk = 150", testKeyspace, TEST_TABLE), + ConsistencyLevel.QUORUM); + assertEquals("migration_150", migrationRecord[0][0]); + } + + @Test + public void testMigrationReversal() throws Exception + { + String testKeyspace = "migration_reversal_test"; + + // untracked keyspace + coordinator.execute(format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='untracked'", + testKeyspace), + ConsistencyLevel.ALL); + + coordinator.execute(format("CREATE TABLE %s.%s (pk int PRIMARY KEY, value text)", testKeyspace, TEST_TABLE), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + for (int i = 0; i < 50; i++) + { + coordinator.execute(format("INSERT INTO %s.%s (pk, value) VALUES (%d, 'initial_%d')", + testKeyspace, TEST_TABLE, i, i), + ConsistencyLevel.QUORUM); + } + + // Start migration to tracked + coordinator.execute(format("ALTER KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked'", + testKeyspace), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.MIGRATING_TO_TRACKED); + + for (int i = 50; i < 100; i++) + { + coordinator.execute(format("INSERT INTO %s.%s (pk, value) VALUES (%d, 'migrating_%d')", + testKeyspace, TEST_TABLE, i, i), + ConsistencyLevel.QUORUM); + } + + // only repair the primary range so the migration isn't complete and we have something to reverse + SHARED_CLUSTER.get(1).nodetoolResult("repair", "-pr", testKeyspace, TEST_TABLE).asserts().success(); + waitForEpochOf(SHARED_CLUSTER, 1); + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.MIGRATING_TO_TRACKED); + + // Reverse the migration by changing back to untracked + coordinator.execute(format("ALTER KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='untracked'", + testKeyspace), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.MIGRATING_TO_UNTRACKED); + + // Complete the reversed migration + SHARED_CLUSTER.get(1).nodetoolResult("repair", "-pr", testKeyspace, TEST_TABLE).asserts().success(); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.UNTRACKED); + + Object[][] results = coordinator.execute(format("SELECT * FROM %s.%s", testKeyspace, TEST_TABLE), + ConsistencyLevel.QUORUM); + assertEquals(100, results.length); + } + + /** + * Test table added during migration then reversed: + * 1. Start untracked → tracked migration + * 2. Create new table during migration + * 3. ALTER back to untracked (reverses migration) + * 4. Verify new table included in reversed migration + */ + @Test + public void testTableAddedDuringMigrationThenReversed() throws Exception + { + String testKeyspace = "table_added_reversal_test"; + String newTable = "tbl2"; + + // untracked keyspace + coordinator.execute(format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='untracked'", + testKeyspace), + ConsistencyLevel.ALL); + + coordinator.execute(format("CREATE TABLE %s.%s (pk int PRIMARY KEY, value text)", testKeyspace, TEST_TABLE), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + // Start migration to tracked + coordinator.execute(format("ALTER KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked'", + testKeyspace), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.MIGRATING_TO_TRACKED); + + // add a new table during migration and write to it + coordinator.execute(format("CREATE TABLE %s.%s (pk int PRIMARY KEY, value text)", testKeyspace, newTable), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + coordinator.execute(format("INSERT INTO %s.%s (pk, value) VALUES (1, 'new_table_data')", testKeyspace, newTable), + ConsistencyLevel.QUORUM); + + // Reverse the migration + coordinator.execute(format("ALTER KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='untracked'", + testKeyspace), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.MIGRATING_TO_UNTRACKED); + + // Complete migration (both tables should be in migration) + SHARED_CLUSTER.get(1).nodetoolResult("repair", testKeyspace).asserts().success(); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.UNTRACKED); + + Object[][] results = coordinator.execute(format("SELECT value FROM %s.%s WHERE pk = 1", testKeyspace, newTable), + ConsistencyLevel.QUORUM); + assertEquals("New table data should be readable after reversal", "new_table_data", results[0][0]); + } + + /** + * Test table dropped during migration: + * 1. Start untracked → tracked migration + * 2. Drop one of the tables + * 3. Verify dropped table removed from migration state + * 4. Complete migration for remaining tables + */ + @Test + public void testTableDroppedDuringMigration() throws Exception + { + String testKeyspace = "table_dropped_test"; + String droppedTable = "tbl_to_drop"; + + // untracked keyspace + coordinator.execute(format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='untracked'", + testKeyspace), + ConsistencyLevel.ALL); + + coordinator.execute(format("CREATE TABLE %s.%s (pk int PRIMARY KEY, value text)", testKeyspace, TEST_TABLE), + ConsistencyLevel.ALL); + coordinator.execute(format("CREATE TABLE %s.%s (pk int PRIMARY KEY, value text)", testKeyspace, droppedTable), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + coordinator.execute(format("INSERT INTO %s.%s (pk, value) VALUES (1, 'keep_this')", testKeyspace, TEST_TABLE), + ConsistencyLevel.QUORUM); + coordinator.execute(format("INSERT INTO %s.%s (pk, value) VALUES (1, 'drop_this')", testKeyspace, droppedTable), + ConsistencyLevel.QUORUM); + + // Start migration to tracked + coordinator.execute(format("ALTER KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked'", + testKeyspace), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.MIGRATING_TO_TRACKED); + + // drop single table during migration + coordinator.execute(format("DROP TABLE %s.%s", testKeyspace, droppedTable), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + // Migration should still be in progress (remaining table not yet repaired) + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.MIGRATING_TO_TRACKED); + + // Complete migration for remaining table + SHARED_CLUSTER.get(1).nodetoolResult("repair", testKeyspace, TEST_TABLE).asserts().success(); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.TRACKED); + + Object[][] results = coordinator.execute(format("SELECT value FROM %s.%s WHERE pk = 1", testKeyspace, TEST_TABLE), + ConsistencyLevel.QUORUM); + assertEquals("Remaining table data should be readable", "keep_this", results[0][0]); + } + + /** + * Test keyspace dropped during migration: + * 1. Start untracked → tracked migration + * 2. Drop the entire keyspace + * 3. Verify migration state completely removed + */ + @Test + public void testKeyspaceDroppedDuringMigration() throws Exception + { + String testKeyspace = "keyspace_dropped_test"; + + // untracked keyspace + coordinator.execute(format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='untracked'", + testKeyspace), + ConsistencyLevel.ALL); + + coordinator.execute(format("CREATE TABLE %s.%s (pk int PRIMARY KEY, value text)", testKeyspace, TEST_TABLE), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + coordinator.execute(format("INSERT INTO %s.%s (pk, value) VALUES (1, 'test_data')", testKeyspace, TEST_TABLE), + ConsistencyLevel.QUORUM); + + // Start migration to tracked + coordinator.execute(format("ALTER KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked'", + testKeyspace), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.MIGRATING_TO_TRACKED); + + // Drop the entire keyspace + coordinator.execute(format("DROP KEYSPACE %s", testKeyspace), + ConsistencyLevel.ALL); + + waitForEpochOf(SHARED_CLUSTER, 1); + + // Verify migration state completely removed + verifyKeyspaceState(testKeyspace, ExpectedKeyspaceState.DROPPED); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordMigrationReadRaceTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordMigrationReadRaceTestBase.java index 5cb835c205e7..91d4aea9912d 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordMigrationReadRaceTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordMigrationReadRaceTestBase.java @@ -626,7 +626,7 @@ private IInvokableInstance setUpOutOfSyncNode(Cluster cluster) throws Throwable Ranges accordRanges = Ranges.of(range); AccordRepairResult accordRepairResult = new AccordRepairResult(accordRanges, TimeUnit.MILLISECONDS.toMicros(currentTimeMillis())); ConsensusMigrationRepairResult repairResult = ConsensusMigrationRepairResult.fromRepair(startEpoch, accordRepairResult, true, true, true, false, false); - ConsensusTableMigration.completedRepairJobHandler.onSuccess(new RepairResult(desc, null, repairResult)); + ConsensusTableMigration.completedRepairJobHandler.onSuccess(new RepairResult(desc, null, repairResult, null)); }).call(); result.get(); } diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordMigrationWriteRaceTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordMigrationWriteRaceTestBase.java index b286e71de19f..c4314c29b297 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordMigrationWriteRaceTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordMigrationWriteRaceTestBase.java @@ -800,7 +800,7 @@ private IInvokableInstance setUpOutOfSyncNode(Cluster cluster, Scenario scenario Ranges accordRanges = Ranges.of(range); AccordRepairResult accordRepairResult = new AccordRepairResult(accordRanges, TimeUnit.MILLISECONDS.toMicros(currentTimeMillis())); ConsensusMigrationRepairResult repairResult = ConsensusMigrationRepairResult.fromRepair(startEpoch, accordRepairResult, true, true, true, false, false); - ConsensusTableMigration.completedRepairJobHandler.onSuccess(new RepairResult(desc, null, repairResult)); + ConsensusTableMigration.completedRepairJobHandler.onSuccess(new RepairResult(desc, null, repairResult, null)); return epochAfterRepair.getEpoch(); }); // Make sure 1 and 2 are up to date and know the reverse migration happens 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 8811fdf7a423..98dab79f69e2 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataTestHelper.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataTestHelper.java @@ -60,6 +60,7 @@ import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.accord.AccordStaleReplicas; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationState; import org.apache.cassandra.tcm.AtomicLongBackedProcessor; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; @@ -160,6 +161,7 @@ public static ClusterMetadata minimalForTesting(Epoch epoch, IPartitioner partit LockedRanges.EMPTY, InProgressSequences.EMPTY, ConsensusMigrationState.EMPTY, + MutationTrackingMigrationState.EMPTY, ImmutableMap.of(), AccordStaleReplicas.EMPTY); } diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 23ff085c778f..0dd6ce203bb8 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -1617,7 +1617,7 @@ protected static void assertNoWarningContains(List warnings, String mess } } - protected static ResultMessage schemaChange(String query) + public static ResultMessage schemaChange(String query) { try { diff --git a/test/unit/org/apache/cassandra/db/MutationTrackingMutationVerbHandlerMigrationTest.java b/test/unit/org/apache/cassandra/db/MutationTrackingMutationVerbHandlerMigrationTest.java new file mode 100644 index 000000000000..025e0e9031d6 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/MutationTrackingMutationVerbHandlerMigrationTest.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.ServerTestUtils; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ReadCommand.PotentialTxnConflicts; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.rows.BTreeRow; +import org.apache.cassandra.db.rows.BufferCell; +import org.apache.cassandra.db.rows.Cell; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; +import org.apache.cassandra.exceptions.CoordinatorBehindException; +import org.apache.cassandra.metrics.TCMMetrics; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Verb; +import org.apache.cassandra.replication.MutationId; +import org.apache.cassandra.replication.MutationJournal; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.ReplicationType; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.Tables; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.utils.FBUtilities; +import org.hamcrest.core.StringContains; + +import static org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper.broadcastAddress; +import static org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper.bytesToken; +import static org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper.node1; +import static org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper.randomInt; +import static org.apache.cassandra.utils.ByteBufferUtil.bytes; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +public class MutationTrackingMutationVerbHandlerMigrationTest +{ + private static final String TEST_NAME = "mutation_migration_test_"; + private static final String TRACKED_KEYSPACE = TEST_NAME + "tracked_ks"; + private static final String UNTRACKED_KEYSPACE = TEST_NAME + "untracked_ks"; + private static final String TABLE = "table1"; + + private MutationVerbHandler handler; + private long startingCoordinatorBehindCount; + + @BeforeClass + public static void init() throws Exception + { + ServerTestUtils.prepareServerNoRegister(); + MutationJournal.instance.start(); + + TableMetadata trackedTable = TableMetadata.builder(TRACKED_KEYSPACE, TABLE) + .addPartitionKeyColumn("pk", Int32Type.instance) + .addRegularColumn("v1", Int32Type.instance) + .build(); + KeyspaceMetadata trackedKs = KeyspaceMetadata.create( + TRACKED_KEYSPACE, + KeyspaceParams.simple(1, ReplicationType.tracked), + Tables.of(trackedTable) + ); + + TableMetadata untrackedTable = TableMetadata.builder(UNTRACKED_KEYSPACE, TABLE) + .addPartitionKeyColumn("pk", Int32Type.instance) + .addRegularColumn("v1", Int32Type.instance) + .build(); + KeyspaceMetadata untrackedKs = KeyspaceMetadata.create( + UNTRACKED_KEYSPACE, + KeyspaceParams.simple(1, ReplicationType.untracked), + Tables.of(untrackedTable) + ); + + // Register the keyspaces + ClusterMetadataTestHelper.addOrUpdateKeyspace(trackedKs); + ClusterMetadataTestHelper.addOrUpdateKeyspace(untrackedKs); + + ServerTestUtils.markCMS(); + StorageService.instance.unsafeSetInitialized(); + DatabaseDescriptor.setMutationTrackingEnabled(true); + } + + @Before + public void setup() throws Exception + { + ServerTestUtils.resetCMS(); + ClusterMetadataTestHelper.addEndpoint(broadcastAddress, bytesToken(100)); + ClusterMetadataTestHelper.addEndpoint(node1, bytesToken(0)); + + MessagingService.instance().inboundSink.clear(); + MessagingService.instance().outboundSink.clear(); + + handler = new MutationVerbHandler(); + startingCoordinatorBehindCount = TCMMetrics.instance.coordinatorBehindReplication.getCount(); + } + + @Test + public void testAcceptMutationWithMatchingRouting() throws Exception + { + Epoch currentEpoch = ClusterMetadata.current().epoch; + + Mutation trackedMutation = createTrackedMutation(TRACKED_KEYSPACE); + handleWithEpoch(trackedMutation, currentEpoch); + + Mutation untrackedMutation = createUntrackedMutation(UNTRACKED_KEYSPACE); + handleWithEpoch(untrackedMutation, currentEpoch); + + assertEquals(startingCoordinatorBehindCount, TCMMetrics.instance.coordinatorBehindReplication.getCount()); + } + + @Test + public void testCoordinatorBehindThrowsException() throws Exception + { + Epoch currentEpoch = ClusterMetadata.current().epoch; + Epoch oldEpoch = Epoch.create(currentEpoch.getEpoch() - 1); + Mutation mutation = createUntrackedMutation(TRACKED_KEYSPACE); + + try + { + handleWithEpoch(mutation, oldEpoch); + fail("Expected CoordinatorBehindException"); + } + catch (CoordinatorBehindException e) + { + assertEquals(startingCoordinatorBehindCount + 1, TCMMetrics.instance.coordinatorBehindReplication.getCount()); + } + } + + @Test + public void testSameEpochDifferentRoutingThrowsException() throws Exception + { + Epoch currentEpoch = ClusterMetadata.current().epoch; + Mutation mutation = createTrackedMutation(UNTRACKED_KEYSPACE); + + try + { + handleWithEpoch(mutation, currentEpoch); + fail("Expected IllegalStateException"); + } + catch (IllegalStateException e) + { + assert e.getMessage().startsWith("Inconsistent mutation routing"); + } + } + + @Test + public void testCoordinatorAhead() throws Exception + { + Epoch currentEpoch = ClusterMetadata.current().epoch; + Epoch futureEpoch = Epoch.create(currentEpoch.getEpoch() + 1); + + Mutation mutation = createUntrackedMutation(TRACKED_KEYSPACE); + + // since this is a unit test, we can't actually fetch newer epochs, so we just + // make sure that the attempted fetch throws an exception + try + { + handleWithEpoch(mutation, futureEpoch); + fail("Expected IllegalStateException due CMS fetch timeout"); + } + catch (IllegalStateException e) + { + Assert.assertThat(e.getMessage(), StringContains.containsString("Could not catch up to epoch")); + } + + assertEquals(startingCoordinatorBehindCount, TCMMetrics.instance.coordinatorBehindReplication.getCount()); + } + + private void handleWithEpoch(Mutation mutation, Epoch epoch) throws Exception + { + handler.doVerb(Message.builder(Verb.MUTATION_REQ, mutation) + .from(node1) + .withId(randomInt()) + .withEpoch(epoch) + .build()); + } + + private Mutation createMutation(String keyspace, int key, int columnValue, MutationId mutationId) + { + ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(TABLE); + TableMetadata cfm = Schema.instance.getTableMetadata(keyspace, TABLE); + DecoratedKey dk = cfs.decorateKey(bytes(key)); + ColumnMetadata col = cfs.metadata().getColumn(bytes("v1")); + Cell cell = BufferCell.live(col, FBUtilities.timestampMicros(), bytes(columnValue)); + Row row = BTreeRow.singleCellRow(Clustering.EMPTY, cell); + PartitionUpdate update = PartitionUpdate.singleRowUpdate(cfm, dk, row); + return new Mutation(mutationId, update, PotentialTxnConflicts.DISALLOW); + } + + private Mutation createTrackedMutation(String keyspace) + { + return createMutation(keyspace, 50, 1, new MutationId(1L, 1L)); + } + + private Mutation createUntrackedMutation(String keyspace) + { + return createMutation(keyspace, 51, 1, MutationId.none()); + } +} diff --git a/test/unit/org/apache/cassandra/db/virtual/LocalRepairTablesTest.java b/test/unit/org/apache/cassandra/db/virtual/LocalRepairTablesTest.java index 5aa94cc666d5..724d12d69920 100644 --- a/test/unit/org/apache/cassandra/db/virtual/LocalRepairTablesTest.java +++ b/test/unit/org/apache/cassandra/db/virtual/LocalRepairTablesTest.java @@ -53,6 +53,7 @@ import org.apache.cassandra.schema.Schema; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.TimeUUID; @@ -325,7 +326,7 @@ private ValidationState validation() private ParticipateState participate() { List> ranges = Arrays.asList(new Range<>(new Murmur3Partitioner.LongToken(0), new Murmur3Partitioner.LongToken(42))); - ParticipateState state = new ParticipateState(Clock.Global.clock(), FBUtilities.getBroadcastAddressAndPort(), new PrepareMessage(TimeUUID.Generator.nextTimeUUID(), Collections.emptyList(), Murmur3Partitioner.instance, ranges, true, 42, true, PreviewKind.ALL)); + ParticipateState state = new ParticipateState(Clock.Global.clock(), FBUtilities.getBroadcastAddressAndPort(), new PrepareMessage(TimeUUID.Generator.nextTimeUUID(), Collections.emptyList(), Murmur3Partitioner.instance, ranges, true, 42, true, PreviewKind.ALL, Epoch.EMPTY)); ActiveRepairService.instance().register(state); return state; } diff --git a/test/unit/org/apache/cassandra/journal/SegmentsTest.java b/test/unit/org/apache/cassandra/journal/SegmentsTest.java index cbb97421b924..130be0715915 100644 --- a/test/unit/org/apache/cassandra/journal/SegmentsTest.java +++ b/test/unit/org/apache/cassandra/journal/SegmentsTest.java @@ -83,6 +83,112 @@ public void testSelect() }); } + @Test + public void testSelectWithTimestampBeforeFirstSegment() + { + withRandom(rng -> { + java.io.File file = File.createTempFile("segments", "test"); + // Create segments with timestamps starting at 1000 + List> segmentList = new ArrayList<>(); + segmentList.add(new TestSegment<>(file, 1000)); + segmentList.add(new TestSegment<>(file, 2000)); + segmentList.add(new TestSegment<>(file, 3000)); + + Segments segments = Segments.of(segmentList); + + // Search with timestamp before first segment + List> selected = new ArrayList<>(); + segments.select(500, 2500, selected); + + // Should select from first segment (1000) up to and including segments with timestamp <= maxTimestamp (2500) + List> expected = List.of(segmentList.get(0), segmentList.get(1)); + if (!Objects.equals(expected, selected)) + { + throw new AssertionError(String.format("Timestamp before first segment failed:\nExpected: %s\nSelected: %s", + expected, selected)); + } + }); + } + + @Test + public void testSelectWithTimestampBetweenSegments() + { + withRandom(rng -> { + java.io.File file = File.createTempFile("segments", "test"); + List> segmentList = new ArrayList<>(); + segmentList.add(new TestSegment<>(file, 1000)); + segmentList.add(new TestSegment<>(file, 2000)); + segmentList.add(new TestSegment<>(file, 3000)); + segmentList.add(new TestSegment<>(file, 4000)); + + Segments segments = Segments.of(segmentList); + + // Search with timestamp between segments + List> selected = new ArrayList<>(); + segments.select(1500, 3500, selected); + + // Should start from segment 2000 + List> expected = List.of(segmentList.get(1), segmentList.get(2)); + if (!Objects.equals(expected, selected)) + { + throw new AssertionError(String.format("Timestamp between segments failed:\nExpected: %s\nSelected: %s", + expected, selected)); + } + }); + } + + @Test + public void testSelectWithTimestampAfterLastSegment() + { + withRandom(rng -> { + java.io.File file = File.createTempFile("segments", "test"); + List> segmentList = new ArrayList<>(); + segmentList.add(new TestSegment<>(file, 1000)); + segmentList.add(new TestSegment<>(file, 2000)); + segmentList.add(new TestSegment<>(file, 3000)); + + Segments segments = Segments.of(segmentList); + + // Search with minTimestamp after all segments + List> selected = new ArrayList<>(); + segments.select(5000, 10000, selected); + + if (!selected.isEmpty()) + { + throw new AssertionError(String.format("Timestamp after last segment should return empty:\nSelected: %s", + selected)); + } + }); + } + + @Test + public void testSelectPreMigrationFlushScenario() + { + withRandom(rng -> { + java.io.File file = File.createTempFile("segments", "test"); + // Simulate migration scenario: journal segments only exist from post-migration time + List> segmentList = new ArrayList<>(); + segmentList.add(new TestSegment<>(file, 1762973770000L)); // Journal started here + segmentList.add(new TestSegment<>(file, 1762973780000L)); + + Segments segments = Segments.of(segmentList); + + // Flush notification for pre-migration data (timestamp before journal tracking started) + List> selected = new ArrayList<>(); + long preMigrationTimestamp = 1762973760306L; // Before first segment + segments.select(preMigrationTimestamp, 1762973775000L, selected); + + // Should start from first available segment (1762973770000) + // Include segments with timestamp <= 1762973775000: 1762973770000 (yes), 1762973780000 (no, > max) + List> expected = List.of(segmentList.get(0)); + if (!Objects.equals(expected, selected)) + { + throw new AssertionError(String.format("Pre-migration flush scenario failed:\nExpected: %s\nSelected: %s", + expected, selected)); + } + }); + } + private static class TestSegment extends Segment { TestSegment(File dir, long timestamp) diff --git a/test/unit/org/apache/cassandra/locator/MetaStrategyTest.java b/test/unit/org/apache/cassandra/locator/MetaStrategyTest.java index 35ab27d426b9..9444127dd6b3 100644 --- a/test/unit/org/apache/cassandra/locator/MetaStrategyTest.java +++ b/test/unit/org/apache/cassandra/locator/MetaStrategyTest.java @@ -35,6 +35,7 @@ import org.apache.cassandra.service.accord.AccordFastPath; import org.apache.cassandra.service.accord.AccordStaleReplicas; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationState; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.membership.Directory; @@ -94,6 +95,7 @@ public static ClusterMetadata metadata(NodeConfiguration... configurations) LockedRanges.EMPTY, InProgressSequences.EMPTY, ConsensusMigrationState.EMPTY, + MutationTrackingMigrationState.EMPTY, ImmutableMap.of(), AccordStaleReplicas.EMPTY); } diff --git a/test/unit/org/apache/cassandra/repair/FuzzTestBase.java b/test/unit/org/apache/cassandra/repair/FuzzTestBase.java index b6c73faed46b..cf2d2c00cc68 100644 --- a/test/unit/org/apache/cassandra/repair/FuzzTestBase.java +++ b/test/unit/org/apache/cassandra/repair/FuzzTestBase.java @@ -139,6 +139,7 @@ import org.apache.cassandra.streaming.StreamingChannel; import org.apache.cassandra.streaming.StreamingDataInputPlus; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.membership.Location; import org.apache.cassandra.tools.nodetool.Repair; import org.apache.cassandra.utils.AbstractTypeGenerators; @@ -1052,7 +1053,7 @@ public RepairCoordinator repair(String ks, RepairOption options) public RepairCoordinator repair(String ks, RepairOption options, boolean addFailureOnErrorNotification) { - RepairCoordinator repair = new RepairCoordinator(this, (name, tables) -> StorageService.instance.getValidColumnFamilies(false, false, name, tables), name -> StorageService.instance.getReplicas(name, broadcastAddressAndPort()), 42, options, ks); + RepairCoordinator repair = new RepairCoordinator(this, (name, tables) -> StorageService.instance.getValidColumnFamilies(false, false, name, tables), name -> StorageService.instance.getReplicas(name, broadcastAddressAndPort()), 42, options, ks, Epoch.EMPTY); if (addFailureOnErrorNotification) { repair.addProgressListener((tag, event) -> { diff --git a/test/unit/org/apache/cassandra/repair/RepairJobTest.java b/test/unit/org/apache/cassandra/repair/RepairJobTest.java index bea5cef88a86..b01b26bd83ea 100644 --- a/test/unit/org/apache/cassandra/repair/RepairJobTest.java +++ b/test/unit/org/apache/cassandra/repair/RepairJobTest.java @@ -68,6 +68,7 @@ import org.apache.cassandra.service.paxos.cleanup.PaxosCleanupResponse; import org.apache.cassandra.service.paxos.cleanup.PaxosRepairState; import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.MerkleTree; @@ -125,11 +126,11 @@ private static class MeasureableRepairSession extends RepairSession public MeasureableRepairSession(TimeUUID parentRepairSession, CommonRange commonRange, boolean excludedDeadNodes, String keyspace, RepairParallelism parallelismDegree, boolean isIncremental, boolean pullRepair, PreviewKind previewKind, boolean optimiseStreams, boolean repairData, boolean repairPaxos, - boolean dontPurgeTombstones, boolean repairAccord, String... cfnames) + boolean dontPurgeTombstones, boolean repairAccord, Epoch minEpoch, String... cfnames) { super(SharedContext.Global.instance, new Scheduler.NoopScheduler(), parentRepairSession, commonRange, excludedDeadNodes, keyspace, parallelismDegree, isIncremental, pullRepair, - previewKind, optimiseStreams, repairData, repairPaxos, dontPurgeTombstones, repairAccord, cfnames); + previewKind, optimiseStreams, repairData, repairPaxos, dontPurgeTombstones, repairAccord, minEpoch, cfnames); } @Override @@ -195,7 +196,7 @@ public void setup() this.session = new MeasureableRepairSession(parentRepairSession, new CommonRange(neighbors, emptySet(), FULL_RANGE), false, KEYSPACE, SEQUENTIAL, false, false, - NONE, false, true, true, false, true, CF); + NONE, false, true, true, false, true, Epoch.EMPTY, CF); this.job = new RepairJob(session, CF); this.sessionJobDesc = new RepairJobDesc(session.state.parentRepairSession, session.getId(), diff --git a/test/unit/org/apache/cassandra/repair/RepairMessageVerbHandlerOutOfRangeTest.java b/test/unit/org/apache/cassandra/repair/RepairMessageVerbHandlerOutOfRangeTest.java index e09a4cbe2949..86a02aae968c 100644 --- a/test/unit/org/apache/cassandra/repair/RepairMessageVerbHandlerOutOfRangeTest.java +++ b/test/unit/org/apache/cassandra/repair/RepairMessageVerbHandlerOutOfRangeTest.java @@ -55,6 +55,7 @@ import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.utils.TimeUUID; import static org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper.*; @@ -246,7 +247,7 @@ private static PrepareMessage prepareMsg(Collection> ranges) } private static PrepareMessage prepareMsg(TimeUUID parentRepairSession, Collection> ranges) { - return new PrepareMessage(parentRepairSession, tableIds, Murmur3Partitioner.instance, ranges, false, ActiveRepairService.UNREPAIRED_SSTABLE, true, PreviewKind.NONE); + return new PrepareMessage(parentRepairSession, tableIds, Murmur3Partitioner.instance, ranges, false, ActiveRepairService.UNREPAIRED_SSTABLE, true, PreviewKind.NONE, Epoch.EMPTY); } private static ValidationRequest validationMsg(Range range) diff --git a/test/unit/org/apache/cassandra/repair/RepairSessionTest.java b/test/unit/org/apache/cassandra/repair/RepairSessionTest.java index c59be70c7ba1..ad6660d82d36 100644 --- a/test/unit/org/apache/cassandra/repair/RepairSessionTest.java +++ b/test/unit/org/apache/cassandra/repair/RepairSessionTest.java @@ -37,6 +37,7 @@ import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.TimeUUID; @@ -67,7 +68,7 @@ public void testConviction() throws Exception new CommonRange(endpoints, Collections.emptySet(), Arrays.asList(repairRange)), false, "Keyspace1", RepairParallelism.SEQUENTIAL, false, false, PreviewKind.NONE, false, - false, false, false, false, "Standard1"); + false, false, false, false, Epoch.create(5), "Standard1"); // perform convict session.convict(remote, Double.MAX_VALUE); diff --git a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java index 750c6144553d..f15b2a618265 100644 --- a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java +++ b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java @@ -53,6 +53,7 @@ import org.apache.cassandra.schema.TableId; import org.apache.cassandra.streaming.SessionSummary; import org.apache.cassandra.streaming.StreamSummary; +import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.utils.MerkleTrees; import static java.util.Collections.emptyList; @@ -192,7 +193,7 @@ public void prepareMessage() throws IOException PrepareMessage msg = new PrepareMessage(nextTimeUUID(), new ArrayList() {{add(TableId.generate());}}, Murmur3Partitioner.instance, buildTokenRanges(), true, 100000L, false, - PreviewKind.NONE); + PreviewKind.NONE, Epoch.EMPTY); serializeRoundTrip(msg, PrepareMessage.serializer); } diff --git a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java index ee4fa293a25e..1c4e63dbe413 100644 --- a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java +++ b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java @@ -66,6 +66,7 @@ import org.apache.cassandra.service.snapshot.TableSnapshot; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.membership.NodeAddresses; import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.transformations.Register; @@ -561,7 +562,7 @@ public void testPrepareForRepairThrowsExceptionForInsufficientDisk() DiskUsageMonitor.instance = diskUsageMonitor; when(diskUsageMonitor.getDiskUsage()).thenReturn(1.5); - instance().prepareForRepair(TimeUUID.maxAtUnixMillis(0), null, null, opts(INCREMENTAL_KEY, b2s(true)), false, null); + instance().prepareForRepair(TimeUUID.maxAtUnixMillis(0), null, null, opts(INCREMENTAL_KEY, b2s(true)), false, null, Epoch.EMPTY); } private static class Task implements Runnable diff --git a/test/unit/org/apache/cassandra/service/replication/migration/KeyspaceMigrationInfoTest.java b/test/unit/org/apache/cassandra/service/replication/migration/KeyspaceMigrationInfoTest.java new file mode 100644 index 000000000000..e82c8f564ad7 --- /dev/null +++ b/test/unit/org/apache/cassandra/service/replication/migration/KeyspaceMigrationInfoTest.java @@ -0,0 +1,407 @@ +/* + * 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.replication.migration; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.ServerTestUtils; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.dht.IPartitioner; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.NormalizedRanges; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.tcm.membership.NodeVersion; + +import static org.junit.Assert.*; + +public class KeyspaceMigrationInfoTest +{ + private static IPartitioner partitioner; + private static TableId testTableId; + + @BeforeClass + public static void setup() throws Exception + { + CassandraRelevantProperties.PARTITIONER.setString(Murmur3Partitioner.class.getName()); + ServerTestUtils.prepareServerNoRegister(); + partitioner = DatabaseDescriptor.getPartitioner(); + assertTrue(partitioner instanceof Murmur3Partitioner); + testTableId = TableId.generate(); + } + + @Test + public void testConstruction() + { + Epoch epoch = Epoch.create(1); + Map> pendingRangesPerTable = Collections.emptyMap(); + + KeyspaceMigrationInfo info = new KeyspaceMigrationInfo( + "test_ks", + pendingRangesPerTable, + epoch + ); + + assertEquals("test_ks", info.keyspace); + assertEquals(epoch, info.startedAtEpoch); + assertTrue(info.isComplete()); + assertTrue(info.pendingRangesPerTable.isEmpty()); + } + + @Test + public void testWithRangesRepairedForTable() + { + List> ranges = createTestRanges(); + Epoch epoch1 = Epoch.create(1); + Epoch epoch2 = Epoch.create(2); + + // Start with full ring as pending + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + NormalizedRanges fullRingNormalized = NormalizedRanges.normalizedRanges(Collections.singleton(fullRing)); + Map> pendingRangesPerTable = Collections.singletonMap(testTableId, fullRingNormalized); + + KeyspaceMigrationInfo info = new KeyspaceMigrationInfo( + "test_ks", + pendingRangesPerTable, + epoch2 + ); + + // info should be unchanged if a repair started before migration started + KeyspaceMigrationInfo noop = info.withRangesRepairedForTable(epoch1, testTableId, Collections.singleton(ranges.get(0))); + assertSame(info, noop); + + // Subtract first range + KeyspaceMigrationInfo updated = info.withRangesRepairedForTable(epoch2, testTableId, Collections.singleton(ranges.get(0))); + + assertFalse(updated.pendingRangesPerTable.get(testTableId).intersects(ranges.get(0).right)); + assertTrue(updated.pendingRangesPerTable.get(testTableId).intersects(ranges.get(1).right)); + assertFalse(updated.isComplete()); + + // Subtract second range + KeyspaceMigrationInfo updated2 = updated.withRangesRepairedForTable(epoch2, testTableId, Collections.singleton(ranges.get(1))); + + assertFalse(updated2.pendingRangesPerTable.get(testTableId).intersects(ranges.get(0).right)); + assertFalse(updated2.pendingRangesPerTable.get(testTableId).intersects(ranges.get(1).right)); + assertFalse(updated2.isComplete()); + } + + @Test + public void testSerialization() throws IOException + { + List> ranges = createTestRanges(); + Epoch epoch = Epoch.create(42); + + NormalizedRanges normalizedRanges = NormalizedRanges.normalizedRanges(ranges.subList(0, 2)); + Map> pendingRangesPerTable = Collections.singletonMap(testTableId, normalizedRanges); + + KeyspaceMigrationInfo info = new KeyspaceMigrationInfo( + "test_ks", + pendingRangesPerTable, + epoch + ); + + // Serialize + DataOutputBuffer out = new DataOutputBuffer(); + KeyspaceMigrationInfo.serializer.serialize(info, out, NodeVersion.CURRENT.serializationVersion()); + + // Deserialize + DataInputBuffer in = new DataInputBuffer(out.toByteArray()); + KeyspaceMigrationInfo deserialized = KeyspaceMigrationInfo.serializer.deserialize(in, NodeVersion.CURRENT.serializationVersion()); + + assertEquals(info, deserialized); + } + + @Test + public void testWithDirectionReversed_PartialCompletion() + { + List> ranges = createTestRanges(); + Epoch epoch1 = Epoch.create(1); + Epoch epoch2 = Epoch.create(2); + + // Start with full ring + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + NormalizedRanges fullRingNormalized = NormalizedRanges.normalizedRanges(Collections.singleton(fullRing)); + Map> pendingRangesPerTable = Collections.singletonMap(testTableId, fullRingNormalized); + + KeyspaceMigrationInfo info = new KeyspaceMigrationInfo( + "test_ks", + pendingRangesPerTable, + epoch1 + ); + + // Repair one range + Range completedRange = ranges.get(0); + KeyspaceMigrationInfo afterRepair = info.withRangesRepairedForTable(epoch1, testTableId, Collections.singleton(completedRange)); + + // Verify the range was removed + Token tokenInCompletedRange = completedRange.right; + assertFalse(afterRepair.getPendingRangesForTable(testTableId).intersects(tokenInCompletedRange)); + + // reverse the direction + KeyspaceMigrationInfo reversed = afterRepair.withDirectionReversed(Collections.singletonList(testTableId), epoch2); + assertEquals(epoch2, reversed.startedAtEpoch); + + assertTrue(reversed.getPendingRangesForTable(testTableId).intersects(tokenInCompletedRange)); + + // Verify that the only ranges now being migratated are the only ranges that completed the initial migration + assertFalse(reversed.isComplete()); + assertEquals(NormalizedRanges.normalizedRanges(Collections.singletonList(completedRange)), reversed.pendingRangesPerTable.get(testTableId)); + } + + @Test + public void testWithDirectionReversed_NoCompletion() + { + Epoch epoch1 = Epoch.create(1); + Epoch epoch2 = Epoch.create(2); + + // Start with full ring + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + NormalizedRanges fullRingNormalized = NormalizedRanges.normalizedRanges(Collections.singleton(fullRing)); + Map> pendingRangesPerTable = Collections.singletonMap(testTableId, fullRingNormalized); + + KeyspaceMigrationInfo info = new KeyspaceMigrationInfo( + "test_ks", + pendingRangesPerTable, + epoch1 + ); + assertFalse(info.isComplete()); + + // Reverse without any progress + KeyspaceMigrationInfo reversed = info.withDirectionReversed(Collections.singletonList(testTableId), epoch2); + + assertEquals(epoch2, reversed.startedAtEpoch); + + // Should be complete, full ring was subtracted from full ring + assertTrue(reversed.isComplete()); + } + + @Test + public void testWithDirectionReversed_TableAddedDuringMigration() + { + List> ranges = createTestRanges(); + Epoch epoch1 = Epoch.create(1); + Epoch epoch2 = Epoch.create(2); + + // Start with full ring migrating + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + NormalizedRanges fullRingNormalized = NormalizedRanges.normalizedRanges(Collections.singleton(fullRing)); + Map> pendingRangesPerTable = Collections.singletonMap(testTableId, fullRingNormalized); + + KeyspaceMigrationInfo info = new KeyspaceMigrationInfo( + "test_ks", + pendingRangesPerTable, + epoch1 + ); + + // Repair one range + Range repairedRange = ranges.get(0); + KeyspaceMigrationInfo afterRepair = info.withRangesRepairedForTable(epoch1, testTableId, Collections.singleton(repairedRange)); + + // reverse migration direction with new table + TableId newTableId = TableId.generate(); + List allTables = Arrays.asList(testTableId, newTableId); + KeyspaceMigrationInfo reversed = afterRepair.withDirectionReversed(allTables, epoch2); + assertFalse(reversed.isComplete()); + + // The original table should pending ranges, since some ranges were migrated + assertFalse(reversed.getPendingRangesForTable(testTableId).isEmpty()); + assertEquals(NormalizedRanges.normalizedRanges(Collections.singleton(repairedRange)), reversed.getPendingRangesForTable(testTableId)); + + // New table should have a full ranges since it was created fully migrated + assertFalse(reversed.getPendingRangesForTable(newTableId).isEmpty()); + assertEquals(NormalizedRanges.normalizedRanges(Collections.singleton(fullRing)), reversed.getPendingRangesForTable(newTableId)); + } + + @Test + public void testWithTablesRemoved_SingleTable() + { + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + NormalizedRanges fullRingNormalized = NormalizedRanges.normalizedRanges(Collections.singleton(fullRing)); + + TableId table1 = TableId.generate(); + TableId table2 = TableId.generate(); + + Map> pendingRangesPerTable = new HashMap<>(); + pendingRangesPerTable.put(table1, fullRingNormalized); + pendingRangesPerTable.put(table2, fullRingNormalized); + + KeyspaceMigrationInfo info = new KeyspaceMigrationInfo( + "test_ks", + pendingRangesPerTable, + Epoch.create(1) + ); + + // Remove one table + KeyspaceMigrationInfo updated = info.withTablesRemoved(Collections.singleton(table1)); + + assertNotNull(updated); + assertFalse(updated.isComplete()); + assertNull(updated.pendingRangesPerTable.get(table1)); + assertNotNull(updated.pendingRangesPerTable.get(table2)); + assertEquals(1, updated.pendingRangesPerTable.size()); + } + + @Test + public void testWithTablesRemoved_AllTables() + { + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + NormalizedRanges fullRingNormalized = NormalizedRanges.normalizedRanges(Collections.singleton(fullRing)); + + TableId table1 = TableId.generate(); + + Map> pendingRangesPerTable = Collections.singletonMap(table1, fullRingNormalized); + + KeyspaceMigrationInfo info = new KeyspaceMigrationInfo( + "test_ks", + pendingRangesPerTable, + Epoch.create(1) + ); + + // Remove the only table - should return null (migration complete) + KeyspaceMigrationInfo updated = info.withTablesRemoved(Collections.singleton(table1)); + + assertNull(updated); + } + + @Test + public void testWithTablesRemoved_NonExistentTable() + { + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + NormalizedRanges fullRingNormalized = NormalizedRanges.normalizedRanges(Collections.singleton(fullRing)); + + TableId table1 = TableId.generate(); + TableId nonExistentTable = TableId.generate(); + + Map> pendingRangesPerTable = Collections.singletonMap(table1, fullRingNormalized); + + KeyspaceMigrationInfo info = new KeyspaceMigrationInfo( + "test_ks", + pendingRangesPerTable, + Epoch.create(1) + ); + + // Try to remove a table that doesn't exist - should return same instance + KeyspaceMigrationInfo updated = info.withTablesRemoved(Collections.singleton(nonExistentTable)); + + assertSame(info, updated); + } + + @Test + public void testWithTablesRemoved_EmptySet() + { + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + NormalizedRanges fullRingNormalized = NormalizedRanges.normalizedRanges(Collections.singleton(fullRing)); + + Map> pendingRangesPerTable = Collections.singletonMap(testTableId, fullRingNormalized); + + KeyspaceMigrationInfo info = new KeyspaceMigrationInfo( + "test_ks", + pendingRangesPerTable, + Epoch.create(1) + ); + + // Remove empty set - should return same instance + KeyspaceMigrationInfo updated = info.withTablesRemoved(Collections.emptySet()); + + assertSame(info, updated); + } + + @Test + public void testReadAndWriteRouting_ToTracked() + { + Token tokenInPending = partitioner.getTokenFactory().fromString("100"); + Token tokenOutsidePending = partitioner.getTokenFactory().fromString("500"); + + Token pendingStart = partitioner.getTokenFactory().fromString("-200"); + Token pendingEnd = partitioner.getTokenFactory().fromString("200"); + Range pendingRange = new Range<>(pendingStart, pendingEnd); + + NormalizedRanges normalizedRanges = NormalizedRanges.normalizedRanges(Collections.singleton(pendingRange)); + Map> pendingRangesPerTable = Collections.singletonMap(testTableId, normalizedRanges); + + KeyspaceMigrationInfo migrationInfo = new KeyspaceMigrationInfo( + "test_ks", + pendingRangesPerTable, + Epoch.create(1) + ); + + // all writes should be tracked + assertTrue(migrationInfo.shouldUseTrackedForWrites(true, testTableId, tokenInPending)); + assertTrue(migrationInfo.shouldUseTrackedForWrites(true, testTableId, tokenOutsidePending)); + + // Verify reads have different behavior + assertFalse(migrationInfo.shouldUseTrackedForReads(true, testTableId, tokenInPending)); + assertTrue(migrationInfo.shouldUseTrackedForReads(true, testTableId, tokenOutsidePending)); + } + + @Test + public void testReadAndWriteRouting_ToUntracked() + { + Token tokenInPending = partitioner.getTokenFactory().fromString("100"); + Token tokenOutsidePending = partitioner.getTokenFactory().fromString("500"); + + Token pendingStart = partitioner.getTokenFactory().fromString("-200"); + Token pendingEnd = partitioner.getTokenFactory().fromString("200"); + Range pendingRange = new Range<>(pendingStart, pendingEnd); + + NormalizedRanges normalizedRanges = NormalizedRanges.normalizedRanges(Collections.singleton(pendingRange)); + Map> pendingRangesPerTable = Collections.singletonMap(testTableId, normalizedRanges); + + KeyspaceMigrationInfo migrationInfo = new KeyspaceMigrationInfo( + "test_ks", + pendingRangesPerTable, + Epoch.create(1) + ); + + // only writes for pending ranges are tracked + assertTrue(migrationInfo.shouldUseTrackedForWrites(false, testTableId, tokenInPending)); + assertFalse(migrationInfo.shouldUseTrackedForWrites(false, testTableId, tokenOutsidePending)); + + // reads are always untracked + assertFalse(migrationInfo.shouldUseTrackedForReads(false, testTableId, tokenInPending)); + assertFalse(migrationInfo.shouldUseTrackedForReads(false, testTableId, tokenOutsidePending)); + } + + private List> createTestRanges() + { + Token t1 = partitioner.getTokenFactory().fromString("100"); + Token t2 = partitioner.getTokenFactory().fromString("200"); + Token t3 = partitioner.getTokenFactory().fromString("300"); + Token t4 = partitioner.getTokenFactory().fromString("400"); + + return Arrays.asList( + new Range<>(t1, t2), + new Range<>(t2, t3), + new Range<>(t3, t4) + ); + } +} diff --git a/test/unit/org/apache/cassandra/service/replication/migration/MigrationRouterTest.java b/test/unit/org/apache/cassandra/service/replication/migration/MigrationRouterTest.java new file mode 100644 index 000000000000..5855eab04627 --- /dev/null +++ b/test/unit/org/apache/cassandra/service/replication/migration/MigrationRouterTest.java @@ -0,0 +1,687 @@ +/* + * 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.replication.migration; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.ServerTestUtils; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.Mutation; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.PartitionRangeReadCommand; +import org.apache.cassandra.db.ReadCommand; +import org.apache.cassandra.db.Slices; +import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter; +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.UTF8Type; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Bounds; +import org.apache.cassandra.dht.ExcludingBounds; +import org.apache.cassandra.dht.IncludingExcludingBounds; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.NormalizedRanges; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.replication.MutationId; +import org.apache.cassandra.schema.DistributedSchema; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.ReplicationType; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.Tables; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.utils.Clock; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; + +public class MigrationRouterTest +{ + private static final String TEST_KEYSPACE = "test_ks"; + private static final String TEST_TABLE = "test_table"; + private static final String SYSTEM_KEYSPACE = SchemaConstants.SYSTEM_KEYSPACE_NAME; + private static TableMetadata systemTable; + private static Murmur3Partitioner partitioner; + + @BeforeClass + public static void setUpClass() + { + CassandraRelevantProperties.PARTITIONER.setString(Murmur3Partitioner.class.getName()); + DatabaseDescriptor.daemonInitialization(); + ServerTestUtils.prepareServer(); + partitioner = (Murmur3Partitioner) DatabaseDescriptor.getPartitioner(); + + systemTable = TableMetadata.builder(SYSTEM_KEYSPACE, "system_table") + .addPartitionKeyColumn("pk", UTF8Type.instance) + .addRegularColumn("value", UTF8Type.instance) + .partitioner(partitioner) + .build(); + } + + /** + * Helper method to create a PartitionRangeReadCommand for a specific token range. + */ + private PartitionRangeReadCommand createRangeCommand(TableMetadata table, Token start, Token end) + { + Range keyRange = new Range<>(start.minKeyBound(), end.maxKeyBound()); + DataRange dataRange = DataRange.forKeyRange(keyRange); + + return PartitionRangeReadCommand.create(table, + 0, // nowInSec + ColumnFilter.all(table), + RowFilter.none(), + DataLimits.NONE, + dataRange); + } + + private Token createToken(long value) + { + return new Murmur3Partitioner.LongToken(value); + } + + private enum BoundaryType + { + RANGE, + BOUNDS, + INCLUDING_EXCLUDING_BOUNDS, + EXCLUDING_BOUNDS + } + + private PartitionRangeReadCommand createRangeCommandWithBoundaryType(TableMetadata table, Token start, Token end, BoundaryType boundaryType) + { + AbstractBounds keyRange; + + switch (boundaryType) + { + case RANGE: + keyRange = new Range<>(start.maxKeyBound(), end.maxKeyBound()); + break; + case BOUNDS: + keyRange = new Bounds<>(start.minKeyBound(), end.maxKeyBound()); + break; + case INCLUDING_EXCLUDING_BOUNDS: + keyRange = new IncludingExcludingBounds<>(start.minKeyBound(), end.minKeyBound()); + break; + case EXCLUDING_BOUNDS: + keyRange = new ExcludingBounds<>(start.maxKeyBound(), end.minKeyBound()); + break; + default: + throw new IllegalArgumentException("Unknown boundary type: " + boundaryType); + } + + DataRange dataRange = new DataRange(keyRange, new ClusteringIndexSliceFilter(Slices.ALL, false)); + + return PartitionRangeReadCommand.create(table, + 0, + ColumnFilter.all(table), + RowFilter.none(), + DataLimits.NONE, + dataRange); + } + + private KeyspaceMetadata createKeyspaceMetadata(String keyspace, ReplicationType replicationType, String... tableNames) + { + TableMetadata[] tables = new TableMetadata[tableNames.length]; + for (int i=0; i replication = ImmutableMap.of("class", "SimpleStrategy", "replication_factor", "3"); + return KeyspaceMetadata.create(keyspace, KeyspaceParams.create(false, replication, replicationType), Tables.of(tables)); + } + + private ClusterMetadata withKeyspace(ClusterMetadata cm, KeyspaceMetadata ksm) + { + Assert.assertFalse(cm.schema.getKeyspaces().containsKeyspace(ksm.name)); + return cm.transformer().with(new DistributedSchema(cm.schema.getKeyspaces().withAddedOrUpdated(ksm))).build().metadata; + } + + private ClusterMetadata withMigrationInfo(ClusterMetadata cm, MutationTrackingMigrationState migrationState) + { + return cm.transformer().with(migrationState).build().metadata; + } + + private KeyspaceMigrationInfo createMigrationInfo(Collection tableIds, List> pendingRanges) + { + Map> pendingRangesPerTable; + + if (pendingRanges.isEmpty()) + { + pendingRangesPerTable = Collections.emptyMap(); + } + else + { + NormalizedRanges normalizedRanges = NormalizedRanges.normalizedRanges(pendingRanges); + pendingRangesPerTable = tableIds + .stream() + .collect(Collectors.toMap(tableId -> tableId, tableId -> normalizedRanges)); + } + + return new KeyspaceMigrationInfo( + TEST_KEYSPACE, + pendingRangesPerTable, + Epoch.create(1)); + } + + private KeyspaceMigrationInfo createMigrationInfo(KeyspaceMetadata ksm, List> pendingRanges) + { + return createMigrationInfo(ksm.tables.stream().map(t -> t.id).collect(Collectors.toList()), pendingRanges); + } + + private ClusterMetadata createMetadata(boolean isTracked, List> pendingRanges) + { + ClusterMetadata metadata = new ClusterMetadata(partitioner); + + ReplicationType replicationType = isTracked ? ReplicationType.tracked : ReplicationType.untracked; + KeyspaceMetadata ksm = createKeyspaceMetadata(TEST_KEYSPACE, replicationType, TEST_TABLE); + + metadata = withKeyspace(metadata, ksm); + + KeyspaceMigrationInfo migrationInfo = createMigrationInfo(ksm, pendingRanges); + MutationTrackingMigrationState migrationState = new MutationTrackingMigrationState(Epoch.create(1), Collections.singletonMap(TEST_KEYSPACE, migrationInfo)); + + return withMigrationInfo(metadata, migrationState); + } + + /** + * Confirm that range reads don't get split up when there's not an active migration for them + */ + @Test + public void testNoPendingRanges_NoSplit() + { + Token queryStart = createToken(-800L); + Token queryEnd = createToken(800L); + + ClusterMetadata metadata = createMetadata(true, Collections.emptyList()); + TableMetadata testTable = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + + PartitionRangeReadCommand command = createRangeCommand(testTable, queryStart, queryEnd); + + List splits = MigrationRouter.splitRangeRead(metadata, command); + + assertEquals(1, splits.size()); + + MigrationRouter.RangeReadWithReplication split = splits.get(0); + + assertTrue(split.useTracked); + + // Verify range covers entire query range + assertEquals(queryStart.minKeyBound(), split.read.dataRange().keyRange().left); + assertEquals(queryEnd.maxKeyBound(), split.read.dataRange().keyRange().right); + } + + @Test + public void testSinglePendingRangeInMiddle_Splits() + { + Token queryStart = createToken(-800L); + Token queryEnd = createToken(800L); + Token pendingStart = createToken(-400L); + Token pendingEnd = createToken(400L); + + Range pendingRange = new Range<>(pendingStart, pendingEnd); + + ClusterMetadata metadata = createMetadata(true, Collections.singletonList(pendingRange)); + TableMetadata testTable = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + + PartitionRangeReadCommand command = createRangeCommand(testTable, queryStart, queryEnd); + + List splits = MigrationRouter.splitRangeRead(metadata, command); + + // Should split into 3 parts: [-800..-400) uses tracked, [-400..400) uses untracked, [400..800] uses tracked + assertEquals(3, splits.size()); + + // First split: [-800..-400) - before pending range, uses tracked + MigrationRouter.RangeReadWithReplication split1 = splits.get(0); + assertTrue(split1.useTracked); + assertEquals(queryStart.minKeyBound(), split1.read.dataRange().keyRange().left); + + // Second split: [-400..400) - pending range, uses untracked + MigrationRouter.RangeReadWithReplication split2 = splits.get(1); + assertFalse(split2.useTracked); + + // Third split: [400..800] - after pending range, uses tracked + MigrationRouter.RangeReadWithReplication split3 = splits.get(2); + assertTrue(split3.useTracked); + assertEquals(queryEnd.maxKeyBound(), split3.read.dataRange().keyRange().right); + } + + @Test + public void testMultiplePendingRanges_Splits() + { + Token queryStart = createToken(-800L); + Token queryEnd = createToken(800L); + Token pending1Start = createToken(-400L); + Token pending1End = createToken(-200L); + Token pending2Start = createToken(0L); + Token pending2End = createToken(200L); + + List> pendingRanges = new ArrayList<>(); + pendingRanges.add(new Range<>(pending1Start, pending1End)); + pendingRanges.add(new Range<>(pending2Start, pending2End)); + + ClusterMetadata metadata = createMetadata(true, pendingRanges); + TableMetadata testTable = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + + PartitionRangeReadCommand command = createRangeCommand(testTable, queryStart, queryEnd); + + List splits = MigrationRouter.splitRangeRead(metadata, command); + + // Should split into 5 parts: + // [-800..-400) tracked, [-400..-200) untracked, [-200..0) tracked, [0..200) untracked, [200..800] tracked + assertEquals(5, splits.size()); + + assertTrue(splits.get(0).useTracked); + assertFalse(splits.get(1).useTracked); + assertTrue(splits.get(2).useTracked); + assertFalse(splits.get(3).useTracked); + assertTrue(splits.get(4).useTracked); + } + + @Test + public void testRangeBeforeAllPending_NoSplit() + { + Token queryStart = createToken(-800L); + Token queryEnd = createToken(-600L); + Token pendingStart = createToken(0L); + Token pendingEnd = createToken(200L); + + Range pendingRange = new Range<>(pendingStart, pendingEnd); + + ClusterMetadata metadata = createMetadata(true, Collections.singletonList(pendingRange)); + TableMetadata testTable = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + + PartitionRangeReadCommand command = createRangeCommand(testTable, queryStart, queryEnd); + + List splits = MigrationRouter.splitRangeRead(metadata, command); + + // Should return single tracked split since not in pending range + assertEquals(1, splits.size()); + assertTrue(splits.get(0).useTracked); + } + + @Test + public void testRangeAfterAllPending_NoSplit() + { + Token queryStart = createToken(600L); + Token queryEnd = createToken(800L); + Token pendingStart = createToken(-400L); + Token pendingEnd = createToken(400L); + + Range pendingRange = new Range<>(pendingStart, pendingEnd); + + ClusterMetadata metadata = createMetadata(true, Collections.singletonList(pendingRange)); + TableMetadata testTable = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + + PartitionRangeReadCommand command = createRangeCommand(testTable, queryStart, queryEnd); + + List splits = MigrationRouter.splitRangeRead(metadata, command); + + // Should return single untracked split since not in pending range + assertEquals(1, splits.size()); + assertTrue(splits.get(0).useTracked); + } + + @Test + public void testToTrackedDirection_CorrectProtocols() + { + Token queryStart = createToken(-800L); + Token queryEnd = createToken(800L); + Token pendingStart = createToken(0L); + Token pendingEnd = createToken(200L); + + Range pendingRange = new Range<>(pendingStart, pendingEnd); + + ClusterMetadata metadata = createMetadata(true, Collections.singletonList(pendingRange)); + TableMetadata testTable = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + + PartitionRangeReadCommand command = createRangeCommand(testTable, queryStart, queryEnd); + List splits = MigrationRouter.splitRangeRead(metadata, command); + + assertEquals(3, splits.size()); + + // Before pending: tracked + assertTrue(splits.get(0).useTracked); + + // Pending range: untracked + assertFalse(splits.get(1).useTracked); + + // After pending: tracked + assertTrue(splits.get(2).useTracked); + } + + @Test + public void testToUntrackedDirection_CorrectProtocols() + { + // For migration to untracked: all reads use untracked (no splitting needed) + // This matches single partition behavior: shouldUseTrackedForReads returns false for all ranges + Token queryStart = createToken(-800L); + Token queryEnd = createToken(800L); + Token pendingStart = createToken(0L); + Token pendingEnd = createToken(200L); + + Range pendingRange = new Range<>(pendingStart, pendingEnd); + + ClusterMetadata metadata = createMetadata(false, Collections.singletonList(pendingRange)); + TableMetadata testTable = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + + PartitionRangeReadCommand command = createRangeCommand(testTable, queryStart, queryEnd); + List splits = MigrationRouter.splitRangeRead(metadata, command); + + // Should have 1 split (no splitting needed - all reads use untracked) + assertEquals(1, splits.size()); + + // Entire range: untracked + assertFalse(splits.get(0).useTracked); + + // Verify it covers the entire query range + assertEquals(queryStart, splits.get(0).read.dataRange().startKey().getToken()); + assertEquals(queryEnd, splits.get(0).read.dataRange().stopKey().getToken()); + } + + @Test + public void testSplitsAreContiguousAndCoverEntireRange() + { + Token queryStart = createToken(-800L); + Token queryEnd = createToken(800L); + Token pending1Start = createToken(-400L); + Token pending1End = createToken(-200L); + Token pending2Start = createToken(0L); + Token pending2End = createToken(200L); + + List> pendingRanges = new ArrayList<>(); + pendingRanges.add(new Range<>(pending1Start, pending1End)); + pendingRanges.add(new Range<>(pending2Start, pending2End)); + + ClusterMetadata metadata = createMetadata(true, pendingRanges); + TableMetadata testTable = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + + PartitionRangeReadCommand command = createRangeCommand(testTable, queryStart, queryEnd); + List splits = MigrationRouter.splitRangeRead(metadata, command); + + // verify query bounds + assertEquals(queryStart.minKeyBound(), splits.get(0).read.dataRange().keyRange().left); + assertEquals(queryEnd.maxKeyBound(), splits.get(splits.size() - 1).read.dataRange().keyRange().right); + + // verify split to split continuity and tracked/untracked alternation + for (int i = 0; i < splits.size() - 1; i++) + { + PartitionPosition currentEnd = splits.get(i).read.dataRange().keyRange().right; + PartitionPosition nextStart = splits.get(i + 1).read.dataRange().keyRange().left; + + assertEquals(currentEnd, nextStart); + assertNotEquals(splits.get(i).useTracked, splits.get(i + 1).useTracked); + } + } + + @Test + public void testSystemKeyspacesAlwaysUntracked() + { + Token queryStart = createToken(-800L); + Token queryEnd = createToken(800L); + Token pendingStart = createToken(0L); + Token pendingEnd = createToken(200L); + + Range pendingRange = new Range<>(pendingStart, pendingEnd); + + ClusterMetadata metadata = createMetadata(true, Collections.singletonList(pendingRange)); + + PartitionRangeReadCommand command = createRangeCommand(systemTable, queryStart, queryEnd); + + List splits = MigrationRouter.splitRangeRead(metadata, command); + + assertEquals(1, splits.size()); + + // system keyspaces always use untracked path + assertFalse(splits.get(0).useTracked); + + assertEquals(queryStart.minKeyBound(), splits.get(0).read.dataRange().keyRange().left); + assertEquals(queryEnd.maxKeyBound(), splits.get(0).read.dataRange().keyRange().right); + } + + /** + * Helper method to test range splitting with a specific boundary type and expected split count. + * Migration boundary is at token 0: (minToken, 0] is pending/migrated. + */ + private void assertRangeSplit(BoundaryType boundaryType, long startToken, long endToken, int expectedSplits, String description) + { + // Setup: pending range is (minToken, 0] - meaning tokens <= 0 are migrating + Token splitPoint = createToken(0L); + Range pendingRange = new Range<>(partitioner.getMinimumToken(), splitPoint); + + ClusterMetadata metadata = createMetadata(true, Collections.singletonList(pendingRange)); + TableMetadata testTable = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + + Token start = createToken(startToken); + Token end = createToken(endToken); + + PartitionRangeReadCommand command = createRangeCommandWithBoundaryType(testTable, start, end, boundaryType); + List splits = MigrationRouter.splitRangeRead(metadata, command); + + String testDesc = String.format("%s with %s [%d, %d]", boundaryType, description, startToken, endToken); + + assertEquals(testDesc + " - wrong number of splits", expectedSplits, splits.size()); + + // Verify splits are contiguous and cover entire range + if (splits.size() > 0) + { + assertEquals(testDesc + " - first split doesn't start at query start", + command.dataRange().keyRange().left, + splits.get(0).read.dataRange().keyRange().left); + + assertEquals(testDesc + " - last split doesn't end at query end", + command.dataRange().keyRange().right, + splits.get(splits.size() - 1).read.dataRange().keyRange().right); + + for (int i = 0; i < splits.size() - 1; i++) + { + assertEquals(testDesc + " - splits not contiguous at index " + i, + splits.get(i).read.dataRange().keyRange().right, + splits.get(i + 1).read.dataRange().keyRange().left); + } + + if (splits.size() > 1) + { + for (int i = 0; i < splits.size() - 1; i++) + { + assertNotEquals(testDesc + " - tracked/untracked should alternate at index " + i, + splits.get(i).useTracked, + splits.get(i + 1).useTracked); + } + } + } + } + + /** + * Test range splitting correctness + */ + @Test + public void testRangeSplitWithAllBoundaryTypes() + { + // Range evenly crossing split point + assertRangeSplit(BoundaryType.RANGE, -100L, 100L, 2, "evenly crossing"); + assertRangeSplit(BoundaryType.BOUNDS, -100L, 100L, 2, "evenly crossing"); + assertRangeSplit(BoundaryType.INCLUDING_EXCLUDING_BOUNDS, -100L, 100L, 2, "evenly crossing"); + assertRangeSplit(BoundaryType.EXCLUDING_BOUNDS, -100L, 100L, 2, "evenly crossing"); + + // Range ending at split point + assertRangeSplit(BoundaryType.RANGE, -100L, 0L, 1, "ending at split"); // (start, 0] includes 0, all pending + assertRangeSplit(BoundaryType.BOUNDS, -100L, 0L, 1, "ending at split"); // [start, 0] includes 0, all pending + assertRangeSplit(BoundaryType.INCLUDING_EXCLUDING_BOUNDS, -100L, 0L, 1, "ending at split"); // [start, 0) excludes 0, all pending + assertRangeSplit(BoundaryType.EXCLUDING_BOUNDS, -100L, 0L, 1, "ending at split"); // (start, 0) excludes 0, all pending + + // Range starting at split point + assertRangeSplit(BoundaryType.RANGE, 0L, 100L, 1, "starting at split"); // (0, end] excludes 0, all non-pending + assertRangeSplit(BoundaryType.BOUNDS, 0L, 100L, 2, "starting at split"); // [0, end] includes 0, needs split + assertRangeSplit(BoundaryType.INCLUDING_EXCLUDING_BOUNDS, 0L, 100L, 2, "starting at split"); // [0, end) includes 0, needs split + assertRangeSplit(BoundaryType.EXCLUDING_BOUNDS, 0L, 100L, 1, "starting at split"); // (0, end) excludes 0, all non-pending + + // Range entirely before split + assertRangeSplit(BoundaryType.RANGE, -100L, -50L, 1, "entirely before"); + assertRangeSplit(BoundaryType.BOUNDS, -100L, -50L, 1, "entirely before"); + assertRangeSplit(BoundaryType.INCLUDING_EXCLUDING_BOUNDS, -100L, -50L, 1, "entirely before"); + assertRangeSplit(BoundaryType.EXCLUDING_BOUNDS, -100L, -50L, 1, "entirely before"); + + // Range entirely after split + assertRangeSplit(BoundaryType.RANGE, 50L, 100L, 1, "entirely after"); + assertRangeSplit(BoundaryType.BOUNDS, 50L, 100L, 1, "entirely after"); + assertRangeSplit(BoundaryType.INCLUDING_EXCLUDING_BOUNDS, 50L, 100L, 1, "entirely after"); + assertRangeSplit(BoundaryType.EXCLUDING_BOUNDS, 50L, 100L, 1, "entirely after"); + } + + /** + * Test write routing through MigrationRouter for migration to tracked replication. + * Writes always use tracked replication regardless of pendingRanges. + */ + @Test + public void testWriteRoutingToTracked_AlwaysTracked() + { + Token tokenInPending = createToken(0L); + Token tokenOutsidePending = createToken(500L); + + Range pendingRange = new Range<>(createToken(-200L), createToken(200L)); + + ClusterMetadata metadata = createMetadata(true, Collections.singletonList(pendingRange)); + TableMetadata testTable = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + + assertTrue(MigrationRouter.shouldUseTrackedForWrites(metadata, TEST_KEYSPACE, testTable.id, tokenInPending)); + assertTrue(MigrationRouter.shouldUseTrackedForWrites(metadata, TEST_KEYSPACE, testTable.id, tokenOutsidePending)); + } + + /** + * Test write routing through MigrationRouter for migration to untracked replication. + * Writes use tracked for tokens in pending ranges (still migrating),untracked for completed ranges. + */ + @Test + public void testWriteRoutingToUntracked_PerRangeRouting() + { + Token tokenInPending = createToken(0L); + Token tokenOutsidePending = createToken(500L); + + Range pendingRange = new Range<>(createToken(-200L), createToken(200L)); + + ClusterMetadata metadata = createMetadata(false, Collections.singletonList(pendingRange)); + TableMetadata testTable = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + + assertTrue(MigrationRouter.shouldUseTrackedForWrites(metadata, TEST_KEYSPACE, testTable.id, tokenInPending)); + assertFalse(MigrationRouter.shouldUseTrackedForWrites(metadata, TEST_KEYSPACE, testTable.id, tokenOutsidePending)); + } + + @Test + public void testMultiTableMutationRouting_ToUntracked() + { + ClusterMetadata metadata = new ClusterMetadata(partitioner); + + KeyspaceMetadata ksm = createKeyspaceMetadata(TEST_KEYSPACE, ReplicationType.untracked, "table1", "table2"); + metadata = withKeyspace(metadata, ksm); + + TableMetadata table1 = ksm.getTableNullable("table1"); + TableMetadata table2 = ksm.getTableNullable("table2"); + + ClusterMetadata.Transformer transformer = metadata.transformer(); + + // table1 migrating to untracked, table2 complete + MutationTrackingMigrationState migrationState = metadata.mutationTrackingMigrationState.withKeyspaceMigrating(ksm.name, Collections.singleton(table1.id), transformer.epoch()); + metadata = transformer.with(migrationState).build().metadata; + + // Create a mutation with both tables + DecoratedKey key = partitioner.decorateKey(UTF8Type.instance.decompose("key")); + PartitionUpdate update1 = PartitionUpdate.emptyUpdate(table1, key); + PartitionUpdate update2 = PartitionUpdate.emptyUpdate(table2, key); + + Mutation mutation = new Mutation(MutationId.none(), TEST_KEYSPACE, key, ImmutableMap.of(table1.id, update1, table2.id, update2), Clock.Global.nanoTime(), ReadCommand.PotentialTxnConflicts.ALLOW); + + MigrationRouter.RoutedMutations routed = MigrationRouter.routeMutations(metadata, Collections.singletonList(mutation)); + + Mutation trackedMutation = (Mutation) routed.trackedMutations.get(0); + Mutation untrackedMutation = (Mutation) routed.untrackedMutations.get(0); + + // table 1 is still migrating, so it should be in the tracked mutation + assertEquals(Collections.singleton(table1.id), trackedMutation.getTableIds()); + + // table 2 is done migrating, so it should appear in the untracked mutation + assertEquals(Collections.singleton(table2.id), untrackedMutation.getTableIds()); + } + + /** + * Test mutation routing with multiple tables - some tracked, some untracked. + * This verifies that routeMutations correctly filters mutations to separate tracked/untracked tables. + */ + @Test + public void testMultiTableMutationRouting_ToTracked() + { + ClusterMetadata metadata = new ClusterMetadata(partitioner); + + KeyspaceMetadata ksm = createKeyspaceMetadata(TEST_KEYSPACE, ReplicationType.tracked, "table1", "table2"); + metadata = withKeyspace(metadata, ksm); + + TableMetadata table1 = ksm.getTableNullable("table1"); + TableMetadata table2 = ksm.getTableNullable("table2"); + + ClusterMetadata.Transformer transformer = metadata.transformer(); + + // table1 migrating to untracked, table2 complete + MutationTrackingMigrationState migrationState = metadata.mutationTrackingMigrationState.withKeyspaceMigrating(ksm.name, Collections.singleton(table1.id), transformer.epoch()); + metadata = transformer.with(migrationState).build().metadata; + + // Create a mutation with both tables + DecoratedKey key = partitioner.decorateKey(UTF8Type.instance.decompose("key")); + PartitionUpdate update1 = PartitionUpdate.emptyUpdate(table1, key); + PartitionUpdate update2 = PartitionUpdate.emptyUpdate(table2, key); + + Mutation mutation = new Mutation(MutationId.none(), TEST_KEYSPACE, key, ImmutableMap.of(table1.id, update1, table2.id, update2), Clock.Global.nanoTime(), ReadCommand.PotentialTxnConflicts.ALLOW); + + MigrationRouter.RoutedMutations routed = MigrationRouter.routeMutations(metadata, Collections.singletonList(mutation)); + + Mutation trackedMutation = (Mutation) routed.trackedMutations.get(0); + + // since we're migrating to tracked replication, both updates should be tracked as well + assertEquals(Set.of(table1.id, table2.id), trackedMutation.getTableIds()); + + assertEquals(0, routed.untrackedMutations.size()); + } +} + diff --git a/test/unit/org/apache/cassandra/service/replication/migration/MutationTrackingMigrationStateTest.java b/test/unit/org/apache/cassandra/service/replication/migration/MutationTrackingMigrationStateTest.java new file mode 100644 index 000000000000..dc27a3792c6d --- /dev/null +++ b/test/unit/org/apache/cassandra/service/replication/migration/MutationTrackingMigrationStateTest.java @@ -0,0 +1,291 @@ +/* + * 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.replication.migration; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.ServerTestUtils; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.NormalizedRanges; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.tcm.membership.NodeVersion; + +import static org.junit.Assert.*; + +public class MutationTrackingMigrationStateTest +{ + private static Murmur3Partitioner partitioner; + private static TableId testTableId; + + @BeforeClass + public static void setup() throws Exception + { + CassandraRelevantProperties.PARTITIONER.setString(Murmur3Partitioner.class.getName()); + ServerTestUtils.prepareServerNoRegister(); + partitioner = Murmur3Partitioner.instance; + testTableId = TableId.generate(); + } + + @Test + public void testEmptyState() + { + MutationTrackingMigrationState state = MutationTrackingMigrationState.EMPTY; + assertNotNull(state); + assertEquals(Epoch.EMPTY, state.lastModified); + assertTrue(state.keyspaceInfo.isEmpty()); + assertFalse(state.hasMigratingKeyspaces()); + } + + @Test + public void testWithKeyspaceMigrating() + { + MutationTrackingMigrationState state = MutationTrackingMigrationState.EMPTY; + Epoch epoch = Epoch.create(1); + + MutationTrackingMigrationState updated = state.withKeyspaceMigrating( + "test_ks", + Collections.singletonList(testTableId), + epoch + ); + + assertNotSame(state, updated); + assertTrue(state.keyspaceInfo.isEmpty()); + + assertTrue(updated.hasMigratingKeyspaces()); + + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + KeyspaceMigrationInfo expected = createExpectedKeyspaceMigrationInfo( + "test_ks", + testTableId, + Collections.singleton(fullRing), + epoch + ); + + KeyspaceMigrationInfo actual = updated.getKeyspaceInfo("test_ks"); + + assertEquals(expected, actual); + } + + @Test + public void testStateTransitions() + { + MutationTrackingMigrationState state = MutationTrackingMigrationState.EMPTY; + List> ranges = createTestRanges(); + Epoch epoch1 = Epoch.create(1); + Epoch epoch2 = Epoch.create(2); + + // Start migration + state = state.withKeyspaceMigrating("test_ks", Collections.singletonList(testTableId), epoch1); + assertTrue(state.hasMigratingKeyspaces()); + + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + KeyspaceMigrationInfo expectedAfterStart = createExpectedKeyspaceMigrationInfo( + "test_ks", + testTableId, + Collections.singleton(fullRing), + epoch1 + ); + + assertEquals(expectedAfterStart, state.getKeyspaceInfo("test_ks")); + assertFalse(state.getKeyspaceInfo("test_ks").isComplete()); + + // Subtract migrated ranges + state = state.withRangesRepairedForTable("test_ks", testTableId, ranges, epoch2); + + Set> expectedRemaining = Range.subtract(Collections.singleton(fullRing), ranges); + + KeyspaceMigrationInfo expectedAfterRepair = createExpectedKeyspaceMigrationInfo( + "test_ks", + testTableId, + expectedRemaining, + epoch1 + ); + + assertEquals(expectedAfterRepair, state.getKeyspaceInfo("test_ks")); + } + + @Test + public void testWithMigrationsCompleted() + { + MutationTrackingMigrationState state = MutationTrackingMigrationState.EMPTY; + Epoch epoch1 = Epoch.create(1); + Epoch epoch2 = Epoch.create(2); + + // Start migration + state = state.withKeyspaceMigrating("test_ks", Collections.singletonList(testTableId), epoch1); + assertTrue(state.hasMigratingKeyspaces()); + + // Complete migration + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + MutationTrackingMigrationState completed = state.withRangesRepairedForTable("test_ks", testTableId, Collections.singleton(fullRing), epoch2); + + assertNotSame(state, completed); + + assertFalse(completed.hasMigratingKeyspaces()); + assertNull(completed.getKeyspaceInfo("test_ks")); + } + + @Test + public void testWithMigrationsRemoved() + { + MutationTrackingMigrationState state = MutationTrackingMigrationState.EMPTY; + Epoch epoch = Epoch.create(1); + + // Start migration + state = state.withKeyspaceMigrating("test_ks", Collections.singletonList(testTableId), epoch); + + // Remove migration + MutationTrackingMigrationState removed = state.dropKeyspaces(epoch, Collections.singleton("test_ks")); + + assertFalse(removed.hasMigratingKeyspaces()); + assertNull(removed.getKeyspaceInfo("test_ks")); + } + + @Test + public void testSerializationRoundtrip() throws IOException + { + List> ranges = createTestRanges(); + Epoch epoch1 = Epoch.create(1); + Epoch epoch2 = Epoch.create(2); + + MutationTrackingMigrationState original = MutationTrackingMigrationState.EMPTY + .withKeyspaceMigrating("test_ks", Collections.singletonList(testTableId), epoch1) + .withRangesRepairedForTable("test_ks", testTableId, ranges, epoch2); + + // Serialize + DataOutputBuffer out = new DataOutputBuffer(); + MutationTrackingMigrationState.serializer.serialize(original, out, NodeVersion.CURRENT_METADATA_VERSION); + + // Deserialize + DataInputBuffer in = new DataInputBuffer(out.unsafeGetBufferAndFlip(), false); + MutationTrackingMigrationState deserialized = MutationTrackingMigrationState.serializer.deserialize(in, NodeVersion.CURRENT_METADATA_VERSION); + + assertEquals(original.lastModified, deserialized.lastModified); + assertEquals(original.keyspaceInfo.size(), deserialized.keyspaceInfo.size()); + for (String ks : original.keyspaceInfo.keySet()) + { + KeyspaceMigrationInfo origInfo = original.keyspaceInfo.get(ks); + KeyspaceMigrationInfo deserInfo = deserialized.keyspaceInfo.get(ks); + assertNotNull(deserInfo); + assertEquals(origInfo, deserInfo); + } + } + + @Test + public void testWithLastModified() + { + Epoch epoch1 = Epoch.create(1); + Epoch epoch2 = Epoch.create(2); + + MutationTrackingMigrationState state = MutationTrackingMigrationState.EMPTY + .withKeyspaceMigrating("test_ks", Collections.singletonList(testTableId), epoch1); + + MutationTrackingMigrationState updated = state.withLastModified(epoch2); + + assertNotSame(state, updated); + + assertEquals(epoch1, state.lastModified); + assertEquals(epoch2, updated.lastModified); + } + + @Test + public void testMultipleKeyspaces() + { + MutationTrackingMigrationState state = MutationTrackingMigrationState.EMPTY; + Epoch epoch = Epoch.create(1); + TableId table2Id = TableId.generate(); + + // Start migrations for multiple keyspaces + state = state.withKeyspaceMigrating("ks1", Collections.singletonList(testTableId), epoch); + state = state.withKeyspaceMigrating("ks2", Collections.singletonList(table2Id), epoch); + + assertEquals(2, state.keyspaceInfo.size()); + + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + KeyspaceMigrationInfo expectedKs1 = createExpectedKeyspaceMigrationInfo( + "ks1", + testTableId, + Collections.singleton(fullRing), + epoch + ); + KeyspaceMigrationInfo expectedKs2 = createExpectedKeyspaceMigrationInfo( + "ks2", + table2Id, + Collections.singleton(fullRing), + epoch + ); + + assertEquals(expectedKs1, state.getKeyspaceInfo("ks1")); + assertEquals(expectedKs2, state.getKeyspaceInfo("ks2")); + + // Complete one keyspace + state = state.withRangesRepairedForTable("ks1", testTableId, Collections.singleton(fullRing), epoch); + + assertEquals(1, state.keyspaceInfo.size()); + assertNull(state.getKeyspaceInfo("ks1")); + + // ks2 should still have full ring pending + KeyspaceMigrationInfo expectedKs2AfterKs1Complete = createExpectedKeyspaceMigrationInfo( + "ks2", + table2Id, + Collections.singleton(fullRing), + epoch + ); + + assertEquals(expectedKs2AfterKs1Complete, state.getKeyspaceInfo("ks2")); + } + + private KeyspaceMigrationInfo createExpectedKeyspaceMigrationInfo( + String keyspace, + TableId tableId, + Collection> ranges, + Epoch startedAtEpoch) + { + Map> pendingRanges = + Collections.singletonMap(tableId, NormalizedRanges.normalizedRanges(ranges)); + return new KeyspaceMigrationInfo(keyspace, pendingRanges, startedAtEpoch); + } + + private List> createTestRanges() + { + Token t1 = partitioner.getTokenFactory().fromString("100"); + Token t2 = partitioner.getTokenFactory().fromString("200"); + Token t3 = partitioner.getTokenFactory().fromString("300"); + + return Arrays.asList( + new Range<>(t1, t2), + new Range<>(t2, t3) + ); + } +} diff --git a/test/unit/org/apache/cassandra/tcm/ClusterMetadataTransformationTest.java b/test/unit/org/apache/cassandra/tcm/ClusterMetadataTransformationTest.java index d3bb189c90fb..7ea0522e48f5 100644 --- a/test/unit/org/apache/cassandra/tcm/ClusterMetadataTransformationTest.java +++ b/test/unit/org/apache/cassandra/tcm/ClusterMetadataTransformationTest.java @@ -311,6 +311,8 @@ else if (key == CONSENSUS_MIGRATION_STATE) return metadata.consensusMigrationState; else if (key == ACCORD_STALE_REPLICAS) return metadata.accordStaleReplicas; + else if (key == MUTATION_TRACKING_MIGRATION_STATE) + return metadata.mutationTrackingMigrationState; throw new IllegalArgumentException("Unknown metadata key " + key); } diff --git a/test/unit/org/apache/cassandra/tcm/transformations/AdvanceMutationTrackingMigrationTest.java b/test/unit/org/apache/cassandra/tcm/transformations/AdvanceMutationTrackingMigrationTest.java new file mode 100644 index 000000000000..c0289e8fc27d --- /dev/null +++ b/test/unit/org/apache/cassandra/tcm/transformations/AdvanceMutationTrackingMigrationTest.java @@ -0,0 +1,325 @@ +/* + * 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.tcm.transformations; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; + +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.ServerTestUtils; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.dht.IPartitioner; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.NormalizedRanges; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.service.replication.migration.KeyspaceMigrationInfo; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationState; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.tcm.Transformation; + +import static org.junit.Assert.*; + +/** + * Tests for AdvanceMutationTrackingMigrationRanges transformation. + */ +public class AdvanceMutationTrackingMigrationTest +{ + private static IPartitioner partitioner; + private static TableId testTableId; + + @BeforeClass + public static void setup() throws Exception + { + CassandraRelevantProperties.PARTITIONER.setString(Murmur3Partitioner.class.getName()); + ServerTestUtils.prepareServerNoRegister(); + partitioner = DatabaseDescriptor.getPartitioner(); + assertTrue(partitioner instanceof Murmur3Partitioner); + testTableId = TableId.generate(); + } + + @Test + public void testAdvanceRangesForMigratingKeyspace() + { + Epoch epoch1 = Epoch.create(1); + + // Create initial state with migrating keyspace (TO_TRACKED) + MutationTrackingMigrationState initialState = MutationTrackingMigrationState.EMPTY + .withKeyspaceMigrating("test_ks", Collections.singletonList(testTableId), epoch1); + + ClusterMetadata metadata = new ClusterMetadata(partitioner); + ClusterMetadata prev = metadata.forceEpoch(epoch1).transformer() + .with(initialState) + .build().metadata; + + // Create ranges to mark as completed + Collection> completedRanges = createTestRanges(); + + // Apply transformation (with TableId) + AdvanceMutationTrackingMigration transformation = + new AdvanceMutationTrackingMigration("test_ks", testTableId, completedRanges); + + Transformation.Result result = transformation.execute(prev); + + // Verify success + assertTrue(result.isSuccess()); + + ClusterMetadata updated = result.success().metadata; + KeyspaceMigrationInfo actual = updated.mutationTrackingMigrationState.getKeyspaceInfo("test_ks"); + + Range fullRing = fullRing(); + Collection> expectedRemainingRanges = Range.subtract( + Collections.singleton(fullRing), + completedRanges + ); + + KeyspaceMigrationInfo expected = createExpectedInfo( + "test_ks", + testTableId, + expectedRemainingRanges, + epoch1 + ); + + assertEquals(expected, actual); + + assertFalse(actual.isComplete()); + assertTrue(updated.mutationTrackingMigrationState.hasMigratingKeyspaces()); + } + + @Test + public void testAdvanceRangesCompleteMigration() + { + Epoch epoch1 = Epoch.create(1); + + // Create initial state with migrating keyspace (TO_TRACKED) + MutationTrackingMigrationState initialState = MutationTrackingMigrationState.EMPTY + .withKeyspaceMigrating("test_ks", Collections.singletonList(testTableId), epoch1); + + ClusterMetadata metadata = new ClusterMetadata(partitioner); + ClusterMetadata prev = metadata.forceEpoch(epoch1).transformer() + .with(initialState) + .build().metadata; + + // Complete the full ring + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + Collection> completedRanges = Collections.singleton(fullRing); + + // Apply transformation (with TableId) + AdvanceMutationTrackingMigration transformation = + new AdvanceMutationTrackingMigration("test_ks", testTableId, completedRanges); + + Transformation.Result result = transformation.execute(prev); + + // Verify success + assertTrue(result.isSuccess()); + + ClusterMetadata updated = result.success().metadata; + + // Verify migration was auto-completed (keyspace removed from state) + assertFalse(updated.mutationTrackingMigrationState.hasMigratingKeyspaces()); + assertNull(updated.mutationTrackingMigrationState.getKeyspaceInfo("test_ks")); + } + + @Test + public void testAdvanceRangesForNonMigratingKeyspace() + { + Epoch epoch1 = Epoch.create(1); + + // Create state without any migrating keyspaces + MutationTrackingMigrationState initialState = MutationTrackingMigrationState.EMPTY; + + ClusterMetadata metadata = new ClusterMetadata(partitioner); + ClusterMetadata prev = metadata.forceEpoch(epoch1).transformer() + .with(initialState) + .build().metadata; + + // Try to advance ranges for non-migrating keyspace + Collection> completedRanges = createTestRanges(); + + AdvanceMutationTrackingMigration transformation = + new AdvanceMutationTrackingMigration("test_ks", testTableId, completedRanges); + + Transformation.Result result = transformation.execute(prev); + + // Verify rejection + assertTrue(result.isRejected()); + assertTrue(result.rejected().reason.contains("not migrating")); + } + + @Test + public void testAdvanceRangesForWrongTable() + { + Epoch epoch1 = Epoch.create(1); + + MutationTrackingMigrationState initialState = MutationTrackingMigrationState.EMPTY + .withKeyspaceMigrating("test_ks", Collections.singletonList(testTableId), epoch1); + + ClusterMetadata metadata = new ClusterMetadata(partitioner); + ClusterMetadata prev = metadata.forceEpoch(epoch1).transformer() + .with(initialState) + .build().metadata; + + // Try to advance ranges for a table ID not being migrated + TableId wrongTableId = TableId.generate(); + Collection> completedRanges = createTestRanges(); + + AdvanceMutationTrackingMigration transformation = + new AdvanceMutationTrackingMigration("test_ks", wrongTableId, completedRanges); + + Transformation.Result result = transformation.execute(prev); + + // confirm noop + assertTrue(result.isSuccess()); + ClusterMetadata updated = result.success().metadata; + + KeyspaceMigrationInfo expected = createExpectedInfo( + "test_ks", + testTableId, + Collections.singleton(fullRing()), + epoch1 + ); + + assertEquals(expected, updated.mutationTrackingMigrationState.getKeyspaceInfo("test_ks")); + } + + @Test + public void testAdvancePartialThenComplete() + { + Epoch epoch1 = Epoch.create(1); + + MutationTrackingMigrationState initialState = MutationTrackingMigrationState.EMPTY + .withKeyspaceMigrating("test_ks", Collections.singletonList(testTableId), epoch1); + + ClusterMetadata metadata = new ClusterMetadata(partitioner); + ClusterMetadata prev = metadata.forceEpoch(epoch1).transformer() + .with(initialState) + .build().metadata; + + // advance some ranges + Collection> partialRanges = createTestRanges(); + AdvanceMutationTrackingMigration partial = new AdvanceMutationTrackingMigration("test_ks", testTableId, partialRanges); + + Transformation.Result result1 = partial.execute(prev); + assertTrue(result1.isSuccess()); + ClusterMetadata afterPartial = result1.success().metadata; + + KeyspaceMigrationInfo afterPartialInfo = + afterPartial.mutationTrackingMigrationState.getKeyspaceInfo("test_ks"); + assertNotNull(afterPartialInfo); + assertFalse(afterPartialInfo.isComplete()); + + // advance the rest + Range fullRing = fullRing(); + AdvanceMutationTrackingMigration complete = + new AdvanceMutationTrackingMigration("test_ks", testTableId, + Collections.singleton(fullRing)); + + Transformation.Result result2 = complete.execute(afterPartial); + assertTrue(result2.isSuccess()); + ClusterMetadata afterComplete = result2.success().metadata; + + assertNull(afterComplete.mutationTrackingMigrationState.getKeyspaceInfo("test_ks")); + assertFalse(afterComplete.mutationTrackingMigrationState.hasMigratingKeyspaces()); + } + + @Test + public void testAdvanceMultipleTables() + { + Epoch epoch1 = Epoch.create(1); + TableId table2Id = TableId.generate(); + + // 2 tables migrating + MutationTrackingMigrationState initialState = MutationTrackingMigrationState.EMPTY + .withKeyspaceMigrating("test_ks", Arrays.asList(testTableId, table2Id), epoch1); + + ClusterMetadata metadata = new ClusterMetadata(partitioner); + ClusterMetadata prev = metadata.forceEpoch(epoch1).transformer() + .with(initialState) + .build().metadata; + + // Complete first table + Range fullRing = fullRing(); + AdvanceMutationTrackingMigration completeTable1 = + new AdvanceMutationTrackingMigration("test_ks", testTableId, + Collections.singleton(fullRing)); + + Transformation.Result result1 = completeTable1.execute(prev); + assertTrue(result1.isSuccess()); + ClusterMetadata afterTable1 = result1.success().metadata; + + KeyspaceMigrationInfo info = + afterTable1.mutationTrackingMigrationState.getKeyspaceInfo("test_ks"); + assertNotNull(info); + assertFalse(info.isComplete()); // Keyspace not complete yet + assertTrue(info.pendingRangesPerTable.containsKey(table2Id)); + assertFalse(info.pendingRangesPerTable.containsKey(testTableId)); // table1 removed + + // Complete second table + AdvanceMutationTrackingMigration completeTable2 = + new AdvanceMutationTrackingMigration("test_ks", table2Id, + Collections.singleton(fullRing)); + + Transformation.Result result2 = completeTable2.execute(afterTable1); + assertTrue(result2.isSuccess()); + ClusterMetadata afterTable2 = result2.success().metadata; + + assertNull(afterTable2.mutationTrackingMigrationState.getKeyspaceInfo("test_ks")); + assertFalse(afterTable2.mutationTrackingMigrationState.hasMigratingKeyspaces()); + } + + /** + * Helper to create expected KeyspaceMigrationInfo for assertions + */ + private KeyspaceMigrationInfo createExpectedInfo(String keyspace, + TableId tableId, + Collection> pendingRanges, + Epoch startedAtEpoch) + { + NormalizedRanges normalized = NormalizedRanges.normalizedRanges(pendingRanges); + Map> pendingRangesPerTable = + Collections.singletonMap(tableId, normalized); + return new KeyspaceMigrationInfo(keyspace, pendingRangesPerTable, startedAtEpoch); + } + + /** + * Helper to create full ring range + */ + private Range fullRing() + { + return new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + } + + private Collection> createTestRanges() + { + Token t1 = partitioner.getTokenFactory().fromString("100"); + Token t2 = partitioner.getTokenFactory().fromString("200"); + Token t3 = partitioner.getTokenFactory().fromString("300"); + + return Arrays.asList( + new Range<>(t1, t2), + new Range<>(t2, t3) + ); + } +} diff --git a/test/unit/org/apache/cassandra/tcm/transformations/AlterSchemaMutationTrackingTest.java b/test/unit/org/apache/cassandra/tcm/transformations/AlterSchemaMutationTrackingTest.java new file mode 100644 index 000000000000..12bd501a866c --- /dev/null +++ b/test/unit/org/apache/cassandra/tcm/transformations/AlterSchemaMutationTrackingTest.java @@ -0,0 +1,386 @@ +/* + * 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.tcm.transformations; + +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.collect.ImmutableMap; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.ServerTestUtils; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.NormalizedRanges; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.replication.MutationJournal; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.service.replication.migration.KeyspaceMigrationInfo; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationState; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.Epoch; + +import static org.apache.cassandra.cql3.CQLTester.schemaChange; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +/** + * Tests for AlterSchema auto-starting mutation tracking migration when replication type changes. + */ +public class AlterSchemaMutationTrackingTest +{ + private static final AtomicInteger ksCounter = new AtomicInteger(); + private static Murmur3Partitioner partitioner; + + @BeforeClass + public static void setUpClass() throws Exception + { + CassandraRelevantProperties.PARTITIONER.setString(Murmur3Partitioner.class.getName()); + ServerTestUtils.daemonInitialization(); + ServerTestUtils.prepareServer(); + MutationJournal.instance.start(); + partitioner = (Murmur3Partitioner) DatabaseDescriptor.getPartitioner(); + } + + private static String nextKsName() + { + return "ks" + ksCounter.incrementAndGet(); + } + + @Test + public void testAutoStartToTrackedMigration() throws Throwable + { + String ksName = nextKsName(); + // untracked replication + schemaChange( "CREATE KEYSPACE " + ksName + + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'} " + + "AND replication_type = 'untracked'" + ); + schemaChange(String.format("CREATE TABLE %s.tbl (pk int PRIMARY KEY, val int)", ksName)); + + ClusterMetadata metadata = ClusterMetadata.current(); + assertNull(metadata.mutationTrackingMigrationState.getKeyspaceInfo(ksName)); + + // Alter tracked replication + schemaChange(String.format("ALTER KEYSPACE %s WITH replication_type = 'tracked'", ksName)); + + metadata = ClusterMetadata.current(); + TableId tableId = metadata.schema.getKeyspaceMetadata(ksName).getTableOrViewNullable("tbl").id; + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + + MutationTrackingMigrationState actualState = metadata.mutationTrackingMigrationState; + KeyspaceMigrationInfo actualInfo = actualState.getKeyspaceInfo(ksName); + + MutationTrackingMigrationState expectedState = createExpectedState( + actualState.lastModified, + ksName, + tableId, + fullRing, + actualInfo.startedAtEpoch + ); + + assertStatesEqual(expectedState, actualState, ksName); + } + + @Test + public void testAutoStartToUntrackedMigration() throws Throwable + { + String ksName = nextKsName(); + // tracked replication + schemaChange("CREATE KEYSPACE " + ksName + + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'} " + + "AND replication_type = 'tracked'" + ); + + schemaChange(String.format("CREATE TABLE %s.tbl (pk int PRIMARY KEY, val int)", ksName)); + + ClusterMetadata metadata = ClusterMetadata.current(); + assertNull(metadata.mutationTrackingMigrationState.getKeyspaceInfo(ksName)); + + // Alter keyspace to untracked + schemaChange(String.format("ALTER KEYSPACE %s WITH replication_type = 'untracked'", ksName)); + + metadata = ClusterMetadata.current(); + TableId tableId = metadata.schema.getKeyspaceMetadata(ksName).getTableOrViewNullable("tbl").id; + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + + MutationTrackingMigrationState actualState = metadata.mutationTrackingMigrationState; + KeyspaceMigrationInfo actualInfo = actualState.getKeyspaceInfo(ksName); + + MutationTrackingMigrationState expectedState = createExpectedState( + actualState.lastModified, + ksName, + tableId, + fullRing, + actualInfo.startedAtEpoch + ); + + assertStatesEqual(expectedState, actualState, ksName); + } + + @Test + public void testNoMigrationWhenReplicationTypeUnchanged() throws Throwable + { + String ksName = nextKsName(); + // untracked replication + schemaChange("CREATE KEYSPACE " + ksName + + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'} " + + "AND replication_type = 'untracked'" + ); + schemaChange(String.format("CREATE TABLE %s.tbl (pk int PRIMARY KEY, val int)", ksName)); + assertNull(ClusterMetadata.current().mutationTrackingMigrationState.getKeyspaceInfo(ksName)); + + // Alter keyspace without changing replication type + schemaChange(String.format( + "ALTER KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '3'}", + ksName + )); + + // confirm no migrations were started + assertNull(ClusterMetadata.current().mutationTrackingMigrationState.getKeyspaceInfo(ksName)); + } + + @Test + public void testMultipleKeyspaceMigrations() throws Throwable + { + String ks1 = nextKsName(); + // untracked replication + schemaChange("CREATE KEYSPACE " + ks1 + + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'} " + + "AND replication_type = 'untracked'" + ); + schemaChange(String.format("CREATE TABLE %s.tbl (pk int PRIMARY KEY, val int)", ks1)); + + String ks2 = nextKsName(); + // tracked replication + schemaChange("CREATE KEYSPACE " + ks2 + + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'} " + + "AND replication_type = 'tracked'" + ); + schemaChange(String.format("CREATE TABLE %s.tbl (pk int PRIMARY KEY, val int)", ks2)); + + // Alter to tracked + schemaChange(String.format("ALTER KEYSPACE %s WITH replication_type = 'tracked'", ks1)); + + // Alter to untracked + schemaChange(String.format("ALTER KEYSPACE %s WITH replication_type = 'untracked'", ks2)); + + ClusterMetadata metadata = ClusterMetadata.current(); + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + + TableId table1Id = metadata.schema.getKeyspaceMetadata(ks1).getTableOrViewNullable("tbl").id; + TableId table2Id = metadata.schema.getKeyspaceMetadata(ks2).getTableOrViewNullable("tbl").id; + + MutationTrackingMigrationState actualState = metadata.mutationTrackingMigrationState; + KeyspaceMigrationInfo actual1 = actualState.getKeyspaceInfo(ks1); + KeyspaceMigrationInfo actual2 = actualState.getKeyspaceInfo(ks2); + + ImmutableMap.Builder expectedKeyspaces = ImmutableMap.builder(); + expectedKeyspaces.put(ks1, createKeyspaceMigrationInfo(ks1, table1Id, fullRing, actual1.startedAtEpoch)); + expectedKeyspaces.put(ks2, createKeyspaceMigrationInfo(ks2, table2Id, fullRing, actual2.startedAtEpoch)); + + MutationTrackingMigrationState expectedState = new MutationTrackingMigrationState( + actualState.lastModified, + expectedKeyspaces.build() + ); + + assertStatesEqual(expectedState, actualState, ks1, ks2); + } + + @Test + public void testReverseMigrationDirection() throws Throwable + { + String ksName = nextKsName(); + // untracked replication + schemaChange("CREATE KEYSPACE " + ksName + + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'} " + + "AND replication_type = 'untracked'" + ); + + schemaChange(String.format("CREATE TABLE %s.tbl (pk int PRIMARY KEY, val int)", ksName)); + + ClusterMetadata metadata = ClusterMetadata.current(); + assertNull("Should have no migration before first alter for " + ksName, + metadata.mutationTrackingMigrationState.getKeyspaceInfo(ksName)); + + // Alter to tracked (untracked → tracked) + schemaChange(String.format("ALTER KEYSPACE %s WITH replication_type = 'tracked'", ksName)); + + metadata = ClusterMetadata.current(); + TableId tableId = metadata.schema.getKeyspaceMetadata(ksName).getTableOrViewNullable("tbl").id; + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + + MutationTrackingMigrationState actualAfterFirstAlter = metadata.mutationTrackingMigrationState; + KeyspaceMigrationInfo actualInfo1 = actualAfterFirstAlter.getKeyspaceInfo(ksName); + + MutationTrackingMigrationState expectedAfterFirstAlter = createExpectedState( + actualAfterFirstAlter.lastModified, + ksName, + tableId, + fullRing, + actualInfo1.startedAtEpoch + ); + + assertStatesEqual(expectedAfterFirstAlter, actualAfterFirstAlter, ksName); + + // Alter back to untracked + schemaChange(String.format("ALTER KEYSPACE %s WITH replication_type = 'untracked'", ksName)); + + // this should auto-complete the migration, since none of the ranges from the initial alter completed migration + metadata = ClusterMetadata.current(); + assertNull(metadata.mutationTrackingMigrationState.getKeyspaceInfo(ksName)); + + // Alter back to tracked again + schemaChange(String.format("ALTER KEYSPACE %s WITH replication_type = 'tracked'", ksName)); + metadata = ClusterMetadata.current(); + + MutationTrackingMigrationState actualAfterThirdAlter = metadata.mutationTrackingMigrationState; + KeyspaceMigrationInfo actualInfo3 = actualAfterThirdAlter.getKeyspaceInfo(ksName); + + MutationTrackingMigrationState expectedAfterThirdAlter = createExpectedState( + actualAfterThirdAlter.lastModified, + ksName, + tableId, + fullRing, + actualInfo3.startedAtEpoch + ); + + assertStatesEqual(expectedAfterThirdAlter, actualAfterThirdAlter, ksName); + } + + @Test + public void testDropKeyspaceDuringMigration() throws Throwable + { + String ksName = nextKsName(); + schemaChange("CREATE KEYSPACE " + ksName + + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'} " + + "AND replication_type = 'untracked'" + ); + schemaChange(String.format("CREATE TABLE %s.tbl (pk int PRIMARY KEY, val int)", ksName)); + + // Alter to tracked + schemaChange(String.format("ALTER KEYSPACE %s WITH replication_type = 'tracked'", ksName)); + + ClusterMetadata metadata = ClusterMetadata.current(); + assertNotNull(metadata.mutationTrackingMigrationState.getKeyspaceInfo(ksName)); + + // Drop the keyspace & confirm migration is also removed + schemaChange(String.format("DROP KEYSPACE %s", ksName)); + + ClusterMetadata afterDrop = ClusterMetadata.current(); + assertNull(afterDrop.mutationTrackingMigrationState.getKeyspaceInfo(ksName)); + } + + @Test + public void testDropTableDuringMigration() throws Throwable + { + String ksName = nextKsName(); + schemaChange("CREATE KEYSPACE " + ksName + + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'} " + + "AND replication_type = 'untracked'" + ); + schemaChange(String.format("CREATE TABLE %s.tbl1 (pk int PRIMARY KEY, val int)", ksName)); + schemaChange(String.format("CREATE TABLE %s.tbl2 (pk int PRIMARY KEY, val int)", ksName)); + + // Alter to tracked + schemaChange(String.format("ALTER KEYSPACE %s WITH replication_type = 'tracked'", ksName)); + + ClusterMetadata metadata = ClusterMetadata.current(); + TableId table1Id = metadata.schema.getKeyspaceMetadata(ksName).getTableOrViewNullable("tbl1").id; + TableId table2Id = metadata.schema.getKeyspaceMetadata(ksName).getTableOrViewNullable("tbl2").id; + Range fullRing = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()); + + MutationTrackingMigrationState actualWithBothTables = metadata.mutationTrackingMigrationState; + KeyspaceMigrationInfo actualInfo = actualWithBothTables.getKeyspaceInfo(ksName); + + ImmutableMap.Builder> pendingRangesBuilder = ImmutableMap.builder(); + pendingRangesBuilder.put(table1Id, NormalizedRanges.normalizedRanges(Collections.singleton(fullRing))); + pendingRangesBuilder.put(table2Id, NormalizedRanges.normalizedRanges(Collections.singleton(fullRing))); + + KeyspaceMigrationInfo expectedInfo = new KeyspaceMigrationInfo( + ksName, + pendingRangesBuilder.build(), + actualInfo.startedAtEpoch + ); + + MutationTrackingMigrationState expectedWithBothTables = new MutationTrackingMigrationState( + actualWithBothTables.lastModified, + ImmutableMap.of(ksName, expectedInfo) + ); + + assertStatesEqual(expectedWithBothTables, actualWithBothTables, ksName); + + schemaChange(String.format("DROP TABLE %s.tbl2", ksName)); + + metadata = ClusterMetadata.current(); + + MutationTrackingMigrationState actualWithOnlyTable1 = metadata.mutationTrackingMigrationState; + + MutationTrackingMigrationState expectedWithOnlyTable1 = createExpectedState( + actualWithOnlyTable1.lastModified, + ksName, + table1Id, + fullRing, + actualInfo.startedAtEpoch + ); + + assertStatesEqual(expectedWithOnlyTable1, actualWithOnlyTable1, ksName); + } + + private MutationTrackingMigrationState createExpectedState(Epoch lastModified, + String keyspace, + TableId tableId, + Range fullRing, + Epoch startedAtEpoch) + { + KeyspaceMigrationInfo info = createKeyspaceMigrationInfo(keyspace, tableId, fullRing, startedAtEpoch); + return new MutationTrackingMigrationState(lastModified, ImmutableMap.of(keyspace, info)); + } + + private KeyspaceMigrationInfo createKeyspaceMigrationInfo(String keyspace, + TableId tableId, + Range fullRing, + Epoch startedAtEpoch) + { + Map> pendingRanges = + ImmutableMap.of(tableId, NormalizedRanges.normalizedRanges(Collections.singleton(fullRing))); + return new KeyspaceMigrationInfo(keyspace, pendingRanges, startedAtEpoch); + } + + /** + * Assert two MutationTrackingMigrationState objects are equal for the given keyspaces + */ + private void assertStatesEqual(MutationTrackingMigrationState expected, MutationTrackingMigrationState actual, String... keyspaces) + { + assertFalse(keyspaces.length == 0); + assertEquals(expected.lastModified, actual.lastModified); + + for (String keyspace : keyspaces) + { + KeyspaceMigrationInfo expectedInfo = expected.getKeyspaceInfo(keyspace); + KeyspaceMigrationInfo actualInfo = actual.getKeyspaceInfo(keyspace); + assertEquals(expectedInfo, actualInfo); + } + } +} diff --git a/test/unit/org/apache/cassandra/utils/CassandraGenerators.java b/test/unit/org/apache/cassandra/utils/CassandraGenerators.java index 6f96be2a01a7..e72439dcc1d7 100644 --- a/test/unit/org/apache/cassandra/utils/CassandraGenerators.java +++ b/test/unit/org/apache/cassandra/utils/CassandraGenerators.java @@ -133,6 +133,7 @@ import org.apache.cassandra.service.accord.fastpath.SimpleFastPathStrategy; import org.apache.cassandra.service.consensus.TransactionalMode; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationState; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.extensions.ExtensionKey; @@ -1975,9 +1976,10 @@ public Gen build() LockedRanges lockedRanges = LockedRanges.EMPTY; InProgressSequences inProgressSequences = InProgressSequences.EMPTY; ConsensusMigrationState consensusMigrationState = ConsensusMigrationState.EMPTY; + MutationTrackingMigrationState mutationTrackingMigrationState = MutationTrackingMigrationState.EMPTY; Map, ExtensionValue> extensions = ImmutableMap.of(); AccordStaleReplicas accordStaleReplicas = accordStaleReplicasGen.generate(rnd); - return new ClusterMetadata(epoch, partitioner, schema, directory, tokenMap, placements, accordFastPath, lockedRanges, inProgressSequences, consensusMigrationState, extensions, accordStaleReplicas); + return new ClusterMetadata(epoch, partitioner, schema, directory, tokenMap, placements, accordFastPath, lockedRanges, inProgressSequences, consensusMigrationState, mutationTrackingMigrationState, extensions, accordStaleReplicas); }; } }