diff --git a/src/java/org/apache/cassandra/exceptions/CoordinatorBehindException.java b/src/java/org/apache/cassandra/exceptions/CoordinatorBehindException.java index 3ca1b51e23f7..92b91bd42536 100644 --- a/src/java/org/apache/cassandra/exceptions/CoordinatorBehindException.java +++ b/src/java/org/apache/cassandra/exceptions/CoordinatorBehindException.java @@ -25,7 +25,7 @@ public CoordinatorBehindException(String msg) super(msg); } - public CoordinatorBehindException(String msg, UnknownTableException cause) + public CoordinatorBehindException(String msg, Throwable cause) { super(msg, cause); } diff --git a/src/java/org/apache/cassandra/exceptions/WriteTimeoutException.java b/src/java/org/apache/cassandra/exceptions/WriteTimeoutException.java index f516a707cb99..2f8ea4c657b9 100644 --- a/src/java/org/apache/cassandra/exceptions/WriteTimeoutException.java +++ b/src/java/org/apache/cassandra/exceptions/WriteTimeoutException.java @@ -42,6 +42,12 @@ public WriteTimeoutException(WriteType writeType, ConsistencyLevel consistency, this.writeType = writeType; } + public WriteTimeoutException(WriteType writeType, ConsistencyLevel consistency, int received, int blockFor, Throwable cause) + { + super(ExceptionCode.WRITE_TIMEOUT, consistency, received, blockFor, cause); + this.writeType = writeType; + } + @Override protected void serializeSpecificFields(DataOutputPlus out, int version) throws IOException { diff --git a/src/java/org/apache/cassandra/hints/HintsService.java b/src/java/org/apache/cassandra/hints/HintsService.java index 4db17c516f36..ae620754e225 100644 --- a/src/java/org/apache/cassandra/hints/HintsService.java +++ b/src/java/org/apache/cassandra/hints/HintsService.java @@ -73,6 +73,14 @@ */ public final class HintsService implements HintsServiceMBean { + private static long REJECT_HINTS_BEFORE_NANOS; + + @VisibleForTesting + public static void setRejectHintsBeforeNanos(long nanos) + { + REJECT_HINTS_BEFORE_NANOS = nanos; + } + // Dummy address to use for storing metrics for hints that will be retried on a different transaction system // and aren't being sent to a specific node public static final InetAddressAndPort RETRY_ON_DIFFERENT_SYSTEM_ADDRESS; @@ -189,6 +197,9 @@ public void write(Collection hostIds, Hint hint) if (isShutDown) throw new IllegalStateException("HintsService is shut down and can't accept new hints"); + if (hint.mutation.getApproxCreatedAtNanos() < REJECT_HINTS_BEFORE_NANOS) + return; + // we have to make sure that the HintsStore instances get properly initialized - otherwise dispatch will not trigger catalog.maybeLoadStores(hostIds); diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java index 4a29d8716532..ce6fee3b1813 100644 --- a/src/java/org/apache/cassandra/service/StorageProxy.java +++ b/src/java/org/apache/cassandra/service/StorageProxy.java @@ -111,6 +111,7 @@ import org.apache.cassandra.exceptions.ReadTimeoutException; import org.apache.cassandra.exceptions.RequestFailure; import org.apache.cassandra.exceptions.RequestFailureException; +import org.apache.cassandra.exceptions.RequestFailureReason; import org.apache.cassandra.exceptions.RequestTimeoutException; import org.apache.cassandra.exceptions.RetryOnDifferentSystemException; import org.apache.cassandra.exceptions.UnavailableException; @@ -140,11 +141,11 @@ import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.NoPayload; import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.net.RequestCallbackWithFailure; import org.apache.cassandra.net.Verb; import org.apache.cassandra.replication.MutationId; import org.apache.cassandra.replication.MutationTrackingService; import org.apache.cassandra.replication.TrackedWriteRequest; -import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.PartitionDenylist; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.SchemaConstants; @@ -189,6 +190,7 @@ 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.replication.migration.MigrationRouter; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.membership.NodeState; import org.apache.cassandra.tcm.ownership.VersionedEndpoints; @@ -800,7 +802,7 @@ private static PaxosBallotAndContention beginAndRepairPaxos(Dispatcher.RequestTi { Tracing.trace("Finishing incomplete paxos round {}", inProgress); casMetrics.unfinishedCommit.inc(); - Commit refreshedInProgress = Commit.newProposal(ballot, inProgress.update); + Commit refreshedInProgress = Commit.newProposal(ballot, inProgress.mutation); if (proposePaxos(refreshedInProgress, paxosPlan, false, requestTime)) { commitPaxos(refreshedInProgress, consistencyForCommit, false, requestTime); @@ -844,16 +846,29 @@ private static PaxosBallotAndContention beginAndRepairPaxos(Dispatcher.RequestTi } /** - * Unlike commitPaxos, this does not wait for replies. - * For tracked keyspaces, ensures proper mutation ID generation and tracking. + * Unlike commitPaxos, this does not wait for commit application replies. + * For tracked keyspaces where the local node is not a replica, waits for the + * forwarding coordinator to acknowledge dispatch. */ private static void sendCommit(Commit commit, Iterable targetReplicas, ReplicaPlan.ForPaxosWrite replicaPlan) { String ksName = commit.metadata().keyspace; - KeyspaceMetadata ksMetadata = Schema.instance.getKeyspaceMetadata(ksName); + ClusterMetadata cm = ClusterMetadata.current(); + boolean shouldBeTracked = MigrationRouter.shouldUseTrackedForWrites(cm, ksName, commit.metadata().id, commit.partitionKey().getToken()); + + // Reconcile the mutation's ID with the current migration state. + // The commit may have been saved to system.paxos under a different replication type. + if (!shouldBeTracked && !commit.mutation.id().isNone()) + { + logger.warn("Stripping mutation ID {} from V2 paxos sendCommit for {}.{} partition {} - keyspace migrated to untracked", + commit.mutation.id(), ksName, commit.metadata().name, commit.partitionKey()); + Tracing.trace("Stripping mutation ID {} from V2 paxos sendCommit for {}.{} partition {} - keyspace migrated to untracked", + commit.mutation.id(), ksName, commit.metadata().name, commit.partitionKey()); + commit = commit.withMutationId(MutationId.none()); + } // Non-tracked keyspaces OR already has mutation ID: fire and forget to target replicas - if (ksMetadata == null || !ksMetadata.params.replicationType.isTracked() || !commit.mutation.id().isNone()) + if (!shouldBeTracked || !commit.mutation.id().isNone()) { Message message = Message.out(PAXOS_COMMIT_REQ, commit); for (InetAddressAndPort target : targetReplicas) @@ -887,30 +902,23 @@ private static void sendCommit(Commit commit, Iterable targe private static void forwardPaxosCommit(Commit commit, ReplicaPlan.ForPaxosWrite replicaPlan) { InetAddressAndPort localEndpoint = FBUtilities.getBroadcastAddressAndPort(); - - // Get live replicas (excluding local node) to find a coordinator EndpointsForToken liveReplicas = replicaPlan.live().filter(replica -> !replica.endpoint().equals(localEndpoint)); - if (liveReplicas.isEmpty()) { - logger.warn("No live replicas available to forward Paxos commit for tracked keyspace"); + logger.warn("No live replicas available to forward Paxos commit for {}.{} partition {}", + commit.metadata().keyspace, commit.metadata().name, commit.partitionKey()); return; } - // Sort by proximity and select the closest as coordinator EndpointsForToken sortedReplicas = DatabaseDescriptor.getNodeProximity().sortedByProximity(localEndpoint, liveReplicas); InetAddressAndPort replicaCoordinator = sortedReplicas.get(0).endpoint(); - Tracing.trace("Forwarding Paxos commit to replica coordinator {}", replicaCoordinator); - // Use respondAfterSend=true so coordinator responds after sending commits (not waiting for application) + // respondAfterSend=true so coordinator responds after sending commits (not waiting for application) PaxosCommitForwardRequest forwardRequest = new PaxosCommitForwardRequest(commit, replicaPlan.consistencyLevel(), true); Message message = Message.out(PAXOS_COMMIT_FORWARD_REQ, forwardRequest); - - // Wait for coordinator to confirm commits were sent before returning Promise promise = new AsyncPromise<>(); - - RequestCallback callback = new RequestCallback() + RequestCallbackWithFailure callback = new RequestCallbackWithFailure() { @Override public void onResponse(Message response) @@ -921,21 +929,27 @@ public void onResponse(Message response) @Override public void onFailure(InetAddressAndPort from, RequestFailure reason) { - promise.setFailure(new RuntimeException("Failed to forward Paxos commit to " + from + ": " + reason)); + if (reason.reason == RequestFailureReason.COORDINATOR_BEHIND) + promise.setFailure(new CoordinatorBehindException("Failed to forward Paxos commit to " + from + ": " + reason, reason.failure)); + else + promise.setFailure(new RuntimeException("Failed to forward Paxos commit to " + from + ": " + reason, reason.failure)); } }; MessagingService.instance().sendWithCallback(message, replicaCoordinator, callback); - try { - // Wait for coordinator to confirm commits were sent promise.get(DatabaseDescriptor.getWriteRpcTimeout(MILLISECONDS), MILLISECONDS); } catch (TimeoutException e) { logger.warn("Timeout waiting for forwarded Paxos commit response from {}", replicaCoordinator); } + catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e); + } catch (Exception e) { logger.warn("Error waiting for forwarded Paxos commit response from {}", replicaCoordinator, e); @@ -1026,35 +1040,56 @@ private static boolean proposePaxos(Commit proposal, ReplicaPlan.ForPaxosWrite r private static void commitPaxos(Commit proposal, ConsistencyLevel consistencyLevel, boolean allowHints, Dispatcher.RequestTime requestTime) throws WriteTimeoutException { checkArgument(!proposal.isEmpty()); - // Check if this is a tracked keyspace String keyspaceName = proposal.metadata().keyspace; Keyspace keyspace = Keyspace.openIfExists(keyspaceName); if (keyspace == null) throw new KeyspaceNotDefinedException("Keyspace " + keyspaceName + " does not exist"); - KeyspaceMetadata ksMetadata = keyspace.getMetadata(); - - if (ksMetadata.params.replicationType.isTracked()) - { - // For tracked keyspaces, check if we need to forward or execute directly - Token tk = proposal.partitionKey().getToken(); - ReplicaPlan.ForWrite replicaPlan = ReplicaPlans.forWrite(keyspace, consistencyLevel, tk, ReplicaPlans.writeAll); - - if (isTrackedKeyspaceRequiringPaxosCommitForwarding(ksMetadata, proposal, replicaPlan.liveAndDown())) + + long deadline = requestTime.computeDeadline(DatabaseDescriptor.getWriteRpcTimeout(NANOSECONDS)); + while (nanoTime() < deadline) + { + try { - // Forward to a replica coordinator - forwardPaxosCommit(proposal, consistencyLevel, replicaPlan); + Token tk = proposal.partitionKey().getToken(); + ClusterMetadata cm = ClusterMetadata.current(); + boolean shouldBeTracked = MigrationRouter.shouldUseTrackedForWrites(cm, keyspaceName, proposal.metadata().id, tk); + + // Reconcile the mutation's ID with the current migration state. + Commit reconciled = proposal; + if (!shouldBeTracked && !proposal.mutation.id().isNone()) + { + logger.warn("Stripping mutation ID {} from V1 paxos commit for {}.{} partition {} - keyspace migrated to untracked", + proposal.mutation.id(), keyspaceName, proposal.metadata().name, proposal.partitionKey()); + Tracing.trace("Stripping mutation ID {} from V1 paxos commit for {}.{} partition {} - keyspace migrated to untracked", + proposal.mutation.id(), keyspaceName, proposal.metadata().name, proposal.partitionKey()); + reconciled = proposal.withMutationId(MutationId.none()); + } + + if (shouldBeTracked) + { + ReplicaPlan.ForWrite replicaPlan = ReplicaPlans.forWrite(keyspace, consistencyLevel, tk, ReplicaPlans.writeAll); + + if (requiresPaxosCommitForwarding(replicaPlan.liveAndDown())) + forwardPaxosCommit(reconciled, consistencyLevel, replicaPlan); + else + commitPaxosTracked(keyspace, reconciled, consistencyLevel, requestTime); + } + else + { + commitPaxosUntracked(keyspace, reconciled, consistencyLevel, allowHints, requestTime); + } + return; } - else + catch (CoordinatorBehindException e) { - // Execute directly using tracked logic - commitPaxosTracked(keyspace, proposal, consistencyLevel, requestTime); + casWriteMetrics.retryCoordinatorBehind.mark(); + logger.warn("Retrying V1 Paxos commit after COORDINATOR_BEHIND for {}.{} partition {}", + keyspaceName, proposal.metadata().name, proposal.partitionKey()); + Tracing.trace("Retrying V1 Paxos commit after COORDINATOR_BEHIND for {}.{} partition {}", + keyspaceName, proposal.metadata().name, proposal.partitionKey()); } } - else - { - // For untracked keyspaces, use existing logic - commitPaxosUntracked(keyspace, proposal, consistencyLevel, allowHints, requestTime); - } + throw new WriteTimeoutException(WriteType.CAS, consistencyLevel, 0, consistencyLevel.blockFor(keyspace.getReplicationStrategy())); } public static void commitPaxosTracked(Keyspace keyspace, Commit proposal, ConsistencyLevel consistencyLevel, Dispatcher.RequestTime requestTime) throws WriteTimeoutException @@ -1152,7 +1187,7 @@ public static void commitPaxosTracked(Keyspace keyspace, Commit proposal, Consis responseHandler.get(); } - private static void commitPaxosUntracked(Keyspace keyspace, Commit proposal, ConsistencyLevel consistencyLevel, boolean allowHints, Dispatcher.RequestTime requestTime) throws WriteTimeoutException + public static void commitPaxosUntracked(Keyspace keyspace, Commit proposal, ConsistencyLevel consistencyLevel, boolean allowHints, Dispatcher.RequestTime requestTime) throws WriteTimeoutException { boolean shouldBlock = consistencyLevel != ConsistencyLevel.ANY; PartitionUpdate update = proposal.update; @@ -1245,48 +1280,35 @@ protected Verb verb() } /** - * Checks if this commit needs to be forwarded to a replica coordinator for tracked keyspace support. + * Returns true if the local node is not a participant and the operation must be forwarded to a replica coordinator. */ - private static boolean isTrackedKeyspaceRequiringPaxosCommitForwarding(KeyspaceMetadata ksMetadata, Commit proposal, EndpointsForToken participants) + private static boolean requiresPaxosCommitForwarding(EndpointsForToken participants) { - if (!ksMetadata.params.replicationType.isTracked()) - return false; - - // Check if current coordinator is not a replica InetAddressAndPort localEndpoint = FBUtilities.getBroadcastAddressAndPort(); - boolean isLocalReplica = participants.endpoints().contains(localEndpoint); - return !isLocalReplica; + return !participants.endpoints().contains(localEndpoint); } /** * Forwards a Paxos V1 commit operation to a replica coordinator for tracked keyspaces. * Uses the replica plan to select the best live, non-local replica based on proximity. */ - private static void forwardPaxosCommit(Commit proposal, ConsistencyLevel consistencyLevel, ReplicaPlan.ForWrite replicaPlan) throws WriteTimeoutException + private static void forwardPaxosCommit(Commit proposal, + ConsistencyLevel consistencyLevel, + ReplicaPlan.ForWrite replicaPlan) + throws WriteTimeoutException, CoordinatorBehindException { InetAddressAndPort localEndpoint = FBUtilities.getBroadcastAddressAndPort(); - - // Get live replicas and filter out local node EndpointsForToken liveReplicas = replicaPlan.live().filter(replica -> !replica.endpoint().equals(localEndpoint)); - if (liveReplicas.isEmpty()) - { - // No live replica available, throw exception throw new WriteTimeoutException(WriteType.CAS, consistencyLevel, 0, consistencyLevel.blockFor(replicaPlan.replicationStrategy())); - } - - // Sort by proximity and select the best coordinator + EndpointsForToken sortedReplicas = DatabaseDescriptor.getNodeProximity().sortedByProximity(localEndpoint, liveReplicas); InetAddressAndPort replicaCoordinator = sortedReplicas.get(0).endpoint(); - - // Create forward request with participant list PaxosCommitForwardRequest forwardRequest = new PaxosCommitForwardRequest(proposal, consistencyLevel); Message message = Message.out(PAXOS_COMMIT_FORWARD_REQ, forwardRequest); - - // Use AsyncPromise for proper callback handling Promise promise = new AsyncPromise<>(); - - RequestCallback callback = new RequestCallback() + + RequestCallbackWithFailure callback = new RequestCallbackWithFailure() { @Override public void onResponse(Message response) @@ -1297,27 +1319,36 @@ public void onResponse(Message response) @Override public void onFailure(InetAddressAndPort from, RequestFailure reason) { - promise.setFailure(new WriteTimeoutException(WriteType.CAS, consistencyLevel, 0, consistencyLevel.blockFor(replicaPlan.replicationStrategy()))); + if (reason.reason == RequestFailureReason.COORDINATOR_BEHIND) + promise.setFailure(new CoordinatorBehindException("Forwarded Paxos commit rejected: handler says coordinator is behind", reason.failure)); + else + promise.setFailure(new WriteTimeoutException(WriteType.CAS, consistencyLevel, 0, consistencyLevel.blockFor(replicaPlan.replicationStrategy()), reason.failure)); } }; - + try { MessagingService.instance().sendWithCallback(message, replicaCoordinator, callback); - - // Wait for response with timeout - promise.get(DatabaseDescriptor.getWriteRpcTimeout(java.util.concurrent.TimeUnit.MILLISECONDS), java.util.concurrent.TimeUnit.MILLISECONDS); + promise.get(DatabaseDescriptor.getWriteRpcTimeout(MILLISECONDS), MILLISECONDS); } catch (TimeoutException e) { throw new WriteTimeoutException(WriteType.CAS, consistencyLevel, 0, consistencyLevel.blockFor(replicaPlan.replicationStrategy())); } + catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e); + } catch (Exception e) { - if (e instanceof WriteTimeoutException) - throw (WriteTimeoutException) e; - - throw new WriteTimeoutException(WriteType.CAS, consistencyLevel, 0, consistencyLevel.blockFor(replicaPlan.replicationStrategy())); + Throwable cause = e.getCause(); + if (cause instanceof WriteTimeoutException) + throw (WriteTimeoutException) cause; + if (cause instanceof CoordinatorBehindException) + throw (CoordinatorBehindException) cause; + + throw new WriteTimeoutException(WriteType.CAS, consistencyLevel, 0, consistencyLevel.blockFor(replicaPlan.replicationStrategy()), e); } } @@ -4349,13 +4380,13 @@ private static RowIterator checkAndForwardCasIfNeeded(String keyspaceName, boolean alreadyForwarded) throws UnavailableException, RequestFailureException, RequestTimeoutException { - // Get keyspace metadata to check if it's tracked Keyspace keyspace = Keyspace.openIfExists(keyspaceName); if (keyspace == null) throw new KeyspaceNotDefinedException("Keyspace " + keyspaceName + " does not exist"); - KeyspaceMetadata ksMetadata = keyspace.getMetadata(); - if (!ksMetadata.params.replicationType.isTracked()) + ClusterMetadata cm = ClusterMetadata.current(); + TableMetadata tableMetadata = cm.schema.getTableMetadata(keyspaceName, cfName); + if (tableMetadata == null || !MigrationRouter.shouldUseTrackedForWrites(cm, keyspaceName, tableMetadata.id, key.getToken())) return null; // Not tracked, no forwarding needed // Property to disable top-level forwarding for testing @@ -4364,7 +4395,7 @@ private static RowIterator checkAndForwardCasIfNeeded(String keyspaceName, // Check if current coordinator is not a replica Token tk = key.getToken(); - EndpointsForToken allReplicas = ReplicaLayout.forTokenWriteLiveAndDown(ClusterMetadata.current(), keyspace, tk) + EndpointsForToken allReplicas = ReplicaLayout.forTokenWriteLiveAndDown(cm, keyspace, tk) .all(); EndpointsForToken liveReplicas = allReplicas.filter(FailureDetector.isReplicaAlive); @@ -4445,13 +4476,12 @@ private static PartitionIterator checkAndForwardConsensusReadIfNeeded(SinglePart SinglePartitionReadCommand firstCommand = group.queries.get(0); String keyspaceName = firstCommand.metadata().keyspace; - // Get keyspace metadata to check if it's tracked Keyspace keyspace = Keyspace.openIfExists(keyspaceName); if (keyspace == null) throw new KeyspaceNotDefinedException("Keyspace " + keyspaceName + " does not exist"); - KeyspaceMetadata ksMetadata = keyspace.getMetadata(); - if (!ksMetadata.params.replicationType.isTracked()) + ClusterMetadata cm = ClusterMetadata.current(); + if (!MigrationRouter.shouldUseTracked(cm, firstCommand)) return null; // Not tracked, no forwarding needed // Property to disable top-level forwarding for testing @@ -4460,7 +4490,7 @@ private static PartitionIterator checkAndForwardConsensusReadIfNeeded(SinglePart // Check if current coordinator is not a replica Token tk = firstCommand.partitionKey().getToken(); - EndpointsForToken allReplicas = ReplicaLayout.forTokenWriteLiveAndDown(ClusterMetadata.current(), keyspace, tk) + EndpointsForToken allReplicas = ReplicaLayout.forTokenWriteLiveAndDown(cm, keyspace, tk) .all(); EndpointsForToken liveReplicas = allReplicas.filter(FailureDetector.isReplicaAlive); diff --git a/src/java/org/apache/cassandra/service/paxos/CasForwardHandler.java b/src/java/org/apache/cassandra/service/paxos/CasForwardHandler.java index b550b5d07f78..3de80c251a3a 100644 --- a/src/java/org/apache/cassandra/service/paxos/CasForwardHandler.java +++ b/src/java/org/apache/cassandra/service/paxos/CasForwardHandler.java @@ -40,7 +40,7 @@ /** * Handler for forwarded CAS (Compare-And-Set) operations. * Executes the CAS operation on behalf of the original coordinator, - * ensuring that MutationId generation happens on a replica coordinator for tracked keyspaces. + * ensuring that MutationId generation happens on a replica coordinator. * * TODO (expected): more comprehensive testing */ @@ -60,7 +60,6 @@ public void doVerb(Message message) ClientWarn.instance.captureWarnings(); try { - // Validate keyspace exists and is tracked KeyspaceMetadata ksMetadata = Schema.instance.getKeyspaceMetadata(request.keyspaceName); if (ksMetadata == null) { @@ -69,13 +68,6 @@ public void doVerb(Message message) return; } - if (!ksMetadata.params.replicationType.isTracked()) - { - MessagingService.instance().respondWithFailure(RequestFailureReason.INCOMPATIBLE_SCHEMA, message); - logger.error("Asked to perform forwarded CAS operation, but keyspace {} is not tracked", request.keyspaceName); - return; - } - // Execute the forwarded CAS operation logger.debug("Executing CAS operation for table {}.{} with key {}", request.keyspaceName, request.cfName, request.key); diff --git a/src/java/org/apache/cassandra/service/paxos/Commit.java b/src/java/org/apache/cassandra/service/paxos/Commit.java index 43ab2103e17c..dc21020e9e72 100644 --- a/src/java/org/apache/cassandra/service/paxos/Commit.java +++ b/src/java/org/apache/cassandra/service/paxos/Commit.java @@ -180,6 +180,12 @@ public AcceptedWithTTL(Ballot ballot, PartitionUpdate update, long localDeletion this.localDeletionTime = localDeletionTime; } + public AcceptedWithTTL(Ballot ballot, Mutation mutation, long localDeletionTime) + { + super(ballot, mutation); + this.localDeletionTime = localDeletionTime; + } + boolean isExpired(long nowInSec) { return nowInSec >= localDeletionTime; @@ -194,7 +200,7 @@ Accepted lastDeleted(Accepted b) @Override public AcceptedWithTTL withMutationId(MutationId mutationId) { - return new AcceptedWithTTL(ballot, makeMutation(mutationId).getOnlyUpdate(), localDeletionTime); + return new AcceptedWithTTL(ballot, makeMutation(mutationId), localDeletionTime); } } @@ -385,6 +391,12 @@ public static Commit newProposal(Ballot ballot, PartitionUpdate update) return new Commit(ballot, update); } + public static Commit newProposal(Ballot ballot, Mutation mutation) + { + PartitionUpdate update = withTimestamp(mutation.getOnlyUpdate(), ballot.unixMicros()); + return new Commit(ballot, new Mutation(mutation.id(), update, mutation.potentialTxnConflicts())); + } + public boolean isAfter(Commit other) { return other == null || ballot.uuidTimestamp() > other.ballot.uuidTimestamp(); @@ -651,7 +663,7 @@ public T deserialize(DataInputPlus in, int version) throws IOException if (version >= MessagingService.VERSION_61) { // New format: deserialize Mutation - Mutation mutation = org.apache.cassandra.db.Mutation.serializer.deserialize(in, version); + Mutation mutation = Mutation.serializer.deserialize(in, version); return mutationConstructor.apply(ballot, mutation); } else diff --git a/src/java/org/apache/cassandra/service/paxos/ConsensusReadForwardHandler.java b/src/java/org/apache/cassandra/service/paxos/ConsensusReadForwardHandler.java index fdae1f9a1ad8..847be3df75f7 100644 --- a/src/java/org/apache/cassandra/service/paxos/ConsensusReadForwardHandler.java +++ b/src/java/org/apache/cassandra/service/paxos/ConsensusReadForwardHandler.java @@ -40,8 +40,7 @@ /** * Handler for forwarded consensus read operations. - * Executes the consensus read operation on behalf of the original coordinator, - * ensuring proper coordination for tracked keyspaces on a replica coordinator. + * Executes the consensus read operation on behalf of the original coordinator. * * TODO (expected): more comprehensive testing */ @@ -55,13 +54,13 @@ public void doVerb(Message message) { ConsensusReadForwardRequest request = message.payload; SinglePartitionReadCommand command = request.command; + Tracing.trace("Executing forwarded consensus read operation for {}", command.partitionKey()); // Start capturing client warnings for the forwarded operation ClientWarn.instance.captureWarnings(); try { - // Validate keyspace exists and is tracked String keyspaceName = command.metadata().keyspace; KeyspaceMetadata ksMetadata = Schema.instance.getKeyspaceMetadata(keyspaceName); if (ksMetadata == null) @@ -71,13 +70,6 @@ public void doVerb(Message message) return; } - if (!ksMetadata.params.replicationType.isTracked()) - { - MessagingService.instance().respondWithFailure(RequestFailureReason.INCOMPATIBLE_SCHEMA, message); - logger.error("Asked to perform forwarded consensus read operation, but keyspace {} is not tracked", keyspaceName); - return; - } - // Create a Group from the single command for reading SinglePartitionReadCommand.Group group = SinglePartitionReadCommand.Group.one(command); @@ -85,7 +77,7 @@ public void doVerb(Message message) // 1. Check forwarding (returns null since we're on a replica) // 2. Execute the consensus read with the appropriate protocol logger.debug("Executing consensus read operation for table {}.{} with key {}", - keyspaceName, command.metadata().name, command.partitionKey()); + command.metadata().keyspace, command.metadata().name, command.partitionKey()); Dispatcher.RequestTime requestTime = Dispatcher.RequestTime.forImmediateExecution(); PartitionIterator result = StorageProxy.readWithConsensusForwarded(group, request.consistencyLevel, requestTime); diff --git a/src/java/org/apache/cassandra/service/paxos/Paxos.java b/src/java/org/apache/cassandra/service/paxos/Paxos.java index f2f60f27d34f..dd88bae1bd43 100644 --- a/src/java/org/apache/cassandra/service/paxos/Paxos.java +++ b/src/java/org/apache/cassandra/service/paxos/Paxos.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Collection; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -94,6 +95,7 @@ import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.FailureRecordingCallback.AsMap; import org.apache.cassandra.service.consensus.migration.ConsensusRequestRouter; +import org.apache.cassandra.service.paxos.Commit.Agreed; import org.apache.cassandra.service.paxos.Commit.Proposal; import org.apache.cassandra.service.paxos.PaxosPrepare.FoundIncompleteAccepted; import org.apache.cassandra.service.paxos.PaxosPrepare.FoundIncompleteCommitted; @@ -740,6 +742,8 @@ public static ConsensusAttemptResult cas(DecoratedKey partitionKey, try (PaxosOperationLock lock = PaxosState.lock(partitionKey, metadata, proposeDeadline, consistencyForConsensus, true)) { Paxos.Async commit = null; + Agreed agreed = null; + Participants commitParticipants = null; done: while (true) { // read the current values and check they validate the conditions @@ -839,7 +843,11 @@ else if (begin.isPromised) // 1) reached a majority, in which case it was agreed, had no effect and we can do nothing; or // 2) did not reach a majority, was not agreed, and was not user visible as a result so we can ignore it if (!proposal.isEmpty()) - commit = commit(proposal.agreed(), participants, consistencyForConsensus, consistencyForCommit, true); + { + agreed = proposal.agreed(); + commitParticipants = participants; + commit = commit(agreed, participants, consistencyForConsensus, consistencyForCommit, true); + } break done; } @@ -872,8 +880,26 @@ else if (begin.isPromised) if (commit != null) { PaxosCommit.Status result = commit.awaitUntil(commitDeadline); - if (!result.isSuccess()) - throw result.maybeFailure().markAndThrowAsTimeoutOrFailure(true, consistencyForCommit, failedAttemptsDueToContention); + while (!result.isSuccess()) + { + Paxos.MaybeFailure failure = result.maybeFailure(); + long coordinatorBehindCount = Collections.frequency(failure.failures.values(), + RequestFailureReason.COORDINATOR_BEHIND); + if (coordinatorBehindCount == 0 + || failure.successes + coordinatorBehindCount < failure.required + || nanoTime() >= commitDeadline) + { + throw failure.markAndThrowAsTimeoutOrFailure(true, consistencyForCommit, failedAttemptsDueToContention); + } + + casWriteMetrics.retryCoordinatorBehind.mark(); + Tracing.trace("Retrying V2 Paxos commit after COORDINATOR_BEHIND for {}.{} partition {}, {} behind replicas out of {} required", + metadata.keyspace, metadata.name, partitionKey, coordinatorBehindCount, failure.required); + logger.warn("Retrying V2 Paxos commit after COORDINATOR_BEHIND for {}.{} partition {}, {} behind replicas out of {} required", + metadata.keyspace, metadata.name, partitionKey, coordinatorBehindCount, failure.required); + commit = commit(agreed, commitParticipants, consistencyForConsensus, consistencyForCommit, true); + result = commit.awaitUntil(commitDeadline); + } } Tracing.trace("CAS successful"); return casResult((RowIterator)null); @@ -1103,7 +1129,7 @@ private static BeginResult begin(long deadline, // is equal to the latest commit (even if the ballots aren't) we're done and can abort earlier, // and in fact it's possible for a CAS to sometimes determine if side effects occurred by reading // the underlying data and not witnessing the timestamp of its ballot (or any newer for the relevant data). - Proposal repropose = new Proposal(inProgress.ballot, inProgress.accepted.update); + Proposal repropose = new Proposal(inProgress.ballot, inProgress.accepted.mutation); PaxosPropose.Status proposeResult = propose(repropose, inProgress.participants, false).awaitUntil(deadline); switch (proposeResult.outcome) { diff --git a/src/java/org/apache/cassandra/service/paxos/Paxos2CommitForwardHandler.java b/src/java/org/apache/cassandra/service/paxos/Paxos2CommitForwardHandler.java index fac9043bfb1d..f2f0473a5fcf 100644 --- a/src/java/org/apache/cassandra/service/paxos/Paxos2CommitForwardHandler.java +++ b/src/java/org/apache/cassandra/service/paxos/Paxos2CommitForwardHandler.java @@ -29,8 +29,8 @@ import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.NoPayload; -import org.apache.cassandra.schema.KeyspaceMetadata; -import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.service.replication.migration.MigrationRouter; +import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.concurrent.ConditionAsConsumer; @@ -39,13 +39,11 @@ /** * Handler for forwarded Paxos V2 commit requests. - * Executes the commit operation on behalf of the original coordinator, - * ensuring that MutationId generation happens on a replica coordinator. - * - * The PaxosCommit constructor handles mutation ID generation, so this handler - * simply delegates to PaxosCommit.commit() with the original commit. + * Delegates to PaxosCommit.commit() which handles mutation ID generation + * in its constructor. * * TODO (expected): more comprehensive testing + * TODO: should loop on CoordinatorBehindException rather than propagating failure to the forwarding coordinator */ public class Paxos2CommitForwardHandler implements IVerbHandler { @@ -55,27 +53,38 @@ public class Paxos2CommitForwardHandler implements IVerbHandler message) { - // Ensure we have up-to-date cluster metadata before executing the forwarded commit - ClusterMetadataService.instance().fetchLogFromPeerOrCMS(message.from(), message.header.epoch); Paxos2CommitForwardRequest request = message.payload; + Commit.Agreed commit = request.commit; - Tracing.trace("Executing forwarded Paxos V2 commit for {}", request.commit.partitionKey()); + Tracing.trace("Executing forwarded Paxos V2 commit for {}.{} partition {}", + commit.metadata().keyspace, commit.metadata().name, commit.partitionKey()); try { - String ksName = request.commit.metadata().keyspace; - KeyspaceMetadata ksMetadata = Schema.instance.getKeyspaceMetadata(ksName); - if (ksMetadata == null) + String ksName = commit.metadata().keyspace; + ClusterMetadata metadata = ClusterMetadata.current(); + boolean shouldBeTracked = MigrationRouter.shouldUseTrackedForWrites(metadata, + ksName, + commit.metadata().id, + commit.partitionKey().getToken()); + + if (!shouldBeTracked && message.epoch().isAfter(metadata.epoch)) { - MessagingService.instance().respondWithFailure(RequestFailureReason.INCOMPATIBLE_SCHEMA, message); - logger.error("Failed to forward paxos commit for non-existent keyspace {}", ksName); - return; + metadata = ClusterMetadataService.instance().fetchLogFromPeerOrCMS(metadata, message.from(), message.epoch()); + // shouldBeTracked isn't used after this, but is kept up to date just in case + shouldBeTracked = MigrationRouter.shouldUseTrackedForWrites(metadata, + ksName, + commit.metadata().id, + commit.partitionKey().getToken()); } - if (!ksMetadata.params.replicationType.isTracked()) + if (metadata.schema.getKeyspaces().getNullable(ksName) == null) { MessagingService.instance().respondWithFailure(RequestFailureReason.INCOMPATIBLE_SCHEMA, message); - logger.error("Asked to perform forwarded paxos commit, but keyspace {} is not tracked", ksName); + logger.error("Failed to forward paxos commit for non-existent keyspace {}.{} partition {}", + ksName, commit.metadata().name, commit.partitionKey()); + Tracing.trace("Failed to forward paxos commit for non-existent keyspace {}.{} partition {}", + ksName, commit.metadata().name, commit.partitionKey()); return; } @@ -112,8 +121,21 @@ public void doVerb(Message message) } else { - MessagingService.instance().respondWithFailure(RequestFailureReason.UNKNOWN, message); + RequestFailureReason reason = RequestFailureReason.UNKNOWN; + if (status != null && status.maybeFailure() != null) + { + for (RequestFailureReason r : status.maybeFailure().failures.values()) + { + if (r == RequestFailureReason.COORDINATOR_BEHIND) + { + reason = RequestFailureReason.COORDINATOR_BEHIND; + break; + } + } + } + MessagingService.instance().respondWithFailure(reason, message); logger.error("Forwarded Paxos V2 commit failed with status: {}", status); + Tracing.trace("Forwarded Paxos V2 commit failed with status: {}", status); } } catch (InterruptedException e) @@ -121,6 +143,7 @@ public void doVerb(Message message) Thread.currentThread().interrupt(); MessagingService.instance().respondWithFailure(RequestFailure.forException(e), message); logger.error("Forwarded Paxos V2 commit interrupted", e); + Tracing.trace("Forwarded Paxos V2 commit interrupted"); } } catch (Exception e) diff --git a/src/java/org/apache/cassandra/service/paxos/PaxosCommit.java b/src/java/org/apache/cassandra/service/paxos/PaxosCommit.java index 03208491fc9d..4f0214191df2 100644 --- a/src/java/org/apache/cassandra/service/paxos/PaxosCommit.java +++ b/src/java/org/apache/cassandra/service/paxos/PaxosCommit.java @@ -18,6 +18,7 @@ package org.apache.cassandra.service.paxos; +import java.util.Collections; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.function.Consumer; @@ -43,11 +44,12 @@ import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.NoPayload; -import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.net.RequestCallbackWithFailure; import org.apache.cassandra.net.Verb; import org.apache.cassandra.replication.MutationId; import org.apache.cassandra.replication.MutationTrackingService; import org.apache.cassandra.service.paxos.Paxos.Participants; +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.FBUtilities; @@ -112,6 +114,7 @@ static class Status final EndpointsForToken replicas; final int required; final OnDone onDone; + final boolean tracked; @Nullable final IntHashSet remoteReplicas; @@ -128,8 +131,7 @@ static class Status public PaxosCommit(Agreed commit, boolean allowHints, ConsistencyLevel consistencyForConsensus, ConsistencyLevel consistencyForCommit, EndpointsForToken replicas, int required, OnDone onDone) { - // Check if this is a tracked keyspace - boolean isTracked = commit.metadata().replicationType().isTracked(); + boolean isTracked = MigrationRouter.shouldUseTrackedForWrites(commit.metadata().keyspace, commit.metadata().id, commit.partitionKey().getToken()); Agreed commitToUse = commit; IntHashSet remoteReplicas = null; @@ -162,7 +164,18 @@ public PaxosCommit(Agreed commit, boolean allowHints, ConsistencyLevel consisten remoteReplicas.add(metadata.directory.peerId(replica.endpoint()).id()); } } + else if (!commit.mutation.id().isNone()) + { + // Keyspace is now untracked but the commit has an ID (from system.paxos or a previous + // round when tracking was active). Strip it to avoid misrouting on replicas. + logger.warn("Stripping mutation ID {} from PaxosCommit for {}.{} partition {} - keyspace migrated to untracked", + commit.mutation.id(), commit.metadata().keyspace, commit.metadata().name, commit.partitionKey()); + Tracing.trace("Stripping mutation ID {} from PaxosCommit for {}.{} partition {} - keyspace migrated to untracked", + commit.mutation.id(), commit.metadata().keyspace, commit.metadata().name, commit.partitionKey()); + commitToUse = commit.withMutationId(MutationId.none()); + } + this.tracked = isTracked; this.commit = commitToUse; this.allowHints = allowHints; this.consistencyForConsensus = consistencyForConsensus; @@ -181,10 +194,8 @@ public PaxosCommit(Agreed commit, boolean allowHints, ConsistencyLevel consisten */ static Paxos.Async commit(Agreed commit, EndpointsForToken all, EndpointsForToken allLive, EndpointsForToken allDown, int required, boolean isUrgent, ConsistencyLevel consistencyForConsensus, ConsistencyLevel consistencyForCommit, /** @deprecated See CASSANDRA-17164 */ @Deprecated(since = "4.1") boolean allowHints) { - // Check if this is a tracked keyspace requiring forwarding to a replica coordinator if (isTrackedKeyspaceRequiringForwarding(commit, all)) { - // For async version, create a wrapper that handles forwarding Status[] statusHolder = new Status[1]; ConditionAsConsumer condition = newConditionAsConsumer(); Consumer statusCapture = status -> { @@ -246,7 +257,6 @@ public Status awaitUntil(long deadline) */ static > T commit(Agreed commit, EndpointsForToken all, EndpointsForToken allLive, EndpointsForToken allDown, int required, boolean isUrgent, ConsistencyLevel consistencyForConsensus, ConsistencyLevel consistencyForCommit, /** @deprecated See CASSANDRA-17164 */ @Deprecated(since = "4.1") boolean allowHints, T onDone) { - // Check if this is a tracked keyspace requiring forwarding to a replica coordinator if (isTrackedKeyspaceRequiringForwarding(commit, all)) { forwardPaxos2Commit(commit, all, allLive, allDown, required, isUrgent, consistencyForConsensus, consistencyForCommit, onDone); @@ -328,13 +338,11 @@ void start(EndpointsForToken allLive, EndpointsForToken allDown, boolean isUrgen } } - // Now send to remote replicas (and record local execution for non-tracked keyspaces) boolean executeOnSelf = false; for (int i = 0, mi = allLive.size(); i < mi ; ++i) { InetAddressAndPort endpoint = allLive.endpoint(i); - // Skip self if we already executed synchronously for tracked keyspace. - // Use direct comparison instead of shouldExecuteOnSelf to avoid dependence on USE_SELF_EXECUTION. + // Skip self if already executed synchronously (avoid shouldExecuteOnSelf to not depend on USE_SELF_EXECUTION) if (localExecutedSynchronously && endpoint.equals(localEndpoint)) continue; executeOnSelf |= isSelfOrSend(commitMessage, mutationMessage, endpoint); @@ -343,17 +351,14 @@ void start(EndpointsForToken allLive, EndpointsForToken allDown, boolean isUrgen for (int i = 0, mi = allDown.size(); i < mi ; ++i) { InetAddressAndPort endpoint = allDown.endpoint(i); - // Skip self if we already executed synchronously for tracked keyspace. - // We can't "retry" to self via network anyway, and we've already written to the journal. + // Skip self — already handled above if (localExecutedSynchronously && endpoint.equals(localEndpoint)) continue; onFailure(endpoint, RequestFailure.NODE_DOWN); } - // Tracked if remoteReplicas != null, register write request with tracking service for tracked keyspaces - if (remoteReplicas != null) + if (isTrackedKeyspace && !remoteReplicas.isEmpty()) { - checkState(!remoteReplicas.isEmpty()); MutationTrackingService.instance().sentWriteRequest(commit.makeMutation(), remoteReplicas); } @@ -390,7 +395,7 @@ private static boolean isInLocalDc(InetAddressAndPort destination) private boolean isTracked() { - return !commit.mutation.id().equals(MutationId.none()); + return tracked; } /** @@ -399,17 +404,18 @@ private boolean isTracked() @Override public void onFailure(InetAddressAndPort from, RequestFailure reason) { + super.onFailure(from, reason); // Populates failureResponses for failureReasonsAsMap() + if (logger.isTraceEnabled()) logger.trace("{} {} from {}", commit, reason, from); - // Track failed response for tracked keyspaces if (isTracked()) MutationTrackingService.instance().retryFailedWrite(commit.mutation.id(), from, reason); response(false, from); Replica replica = replicas.lookup(from); - if (allowHints && shouldHint(replica)) + if (allowHints && shouldHint(replica) && !isTracked()) submitHint(commit.makeMutation(), replica, null); } @@ -420,8 +426,7 @@ public void onResponse(Message response) { logger.trace("{} Success from {}", commit, response.from()); - // Track successful response for tracked keyspaces - // (Local mutations are witnessed from Keyspace.applyInternalTracked) + // Local responses are handled via Keyspace.applyInternalTracked if (isTracked()) MutationTrackingService.instance().receivedWriteResponse(commit.mutation.id(), response.from()); @@ -458,7 +463,6 @@ public void executeOnSelf() @Override public void onResponse(NoPayload response, InetAddressAndPort from) { - // Track successful response for tracked keyspaces if (isTracked()) { if (response != null) @@ -519,6 +523,15 @@ public static class RequestHandler implements IVerbHandler @Override public void doVerb(Message message) { + Agreed agreed = message.payload; + ClusterMetadata metadata = ClusterMetadata.current(); + + boolean coordinatorSaysTracked = !agreed.mutation.id().isNone(); + MigrationRouter.checkPaxosCommitMigration(metadata, message, message.from(), + agreed.metadata().keyspace, agreed.metadata().id, + agreed.partitionKey().getToken(), + coordinatorSaysTracked); + NoPayload response = execute(message.payload); // NOTE: for correctness, this must be our last action, so that we cannot throw an error and send both a response and a failure response if (response == null) @@ -543,7 +556,7 @@ private static NoPayload execute(Agreed agreed) */ private static boolean isTrackedKeyspaceRequiringForwarding(Agreed commit, EndpointsForToken all) { - if (!commit.metadata().replicationType().isTracked()) + if (!MigrationRouter.shouldUseTrackedForWrites(commit.metadata().keyspace, commit.metadata().id, commit.partitionKey().getToken())) return false; // Check if current coordinator is not a replica @@ -592,8 +605,7 @@ private static > void forwardPaxos2Commit(Agreed comm required, isUrgent); Message message = Message.out(Verb.PAXOS2_COMMIT_FORWARD_REQ, forwardRequest); - // Create callback to handle forwarding response - RequestCallback callback = new RequestCallback() + RequestCallbackWithFailure callback = new RequestCallbackWithFailure() { @Override public void onResponse(Message response) @@ -607,9 +619,9 @@ public void onFailure(InetAddressAndPort from, RequestFailure failure) { logger.debug("Forwarded Paxos V2 commit to {} failed: {}", from, failure); Tracing.trace("Forwarded Paxos V2 commit to {} failed: {}", from, failure); - // Populate the failure map with the actual failure reason; contacted=1, required=1 for forwarded request + // contacted=1, required=1 because this is a single forwarded request onDone.accept(new Status(new Paxos.MaybeFailure(true, 1, 1, 0, - java.util.Collections.singletonMap(from, failure.reason)))); + Collections.singletonMap(from, failure.reason)))); } }; @@ -622,7 +634,7 @@ public void onFailure(InetAddressAndPort from, RequestFailure failure) logger.debug("Failed to send forwarded Paxos V2 commit to {}: {}", replicaCoordinator, e.getMessage()); Tracing.trace("Failed to send forwarded Paxos V2 commit: {}", e.getMessage()); onDone.accept(new Status(new Paxos.MaybeFailure(true, 1, 1, 0, - java.util.Collections.singletonMap(replicaCoordinator, UNKNOWN)))); + Collections.singletonMap(replicaCoordinator, UNKNOWN)))); } } diff --git a/src/java/org/apache/cassandra/service/paxos/PaxosCommitAndPrepare.java b/src/java/org/apache/cassandra/service/paxos/PaxosCommitAndPrepare.java index 73fff932642a..053f2fdf3bea 100644 --- a/src/java/org/apache/cassandra/service/paxos/PaxosCommitAndPrepare.java +++ b/src/java/org/apache/cassandra/service/paxos/PaxosCommitAndPrepare.java @@ -20,6 +20,9 @@ import java.io.IOException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.EmbeddableSinglePartitionReadCommand; @@ -29,6 +32,7 @@ import org.apache.cassandra.net.IVerbHandler; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.replication.MutationId; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.consensus.migration.ConsensusKeyMigrationState.KeyMigrationState; import org.apache.cassandra.service.paxos.Commit.Agreed; @@ -36,8 +40,8 @@ import org.apache.cassandra.service.paxos.PaxosPrepare.Response; import org.apache.cassandra.service.reads.tracked.TrackedRead; import org.apache.cassandra.service.reads.tracked.TrackedRead.Id; +import org.apache.cassandra.service.replication.migration.MigrationRouter; import org.apache.cassandra.tcm.ClusterMetadata; -import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.transport.Dispatcher.RequestTime; import org.apache.cassandra.utils.concurrent.Future; @@ -52,6 +56,8 @@ public class PaxosCommitAndPrepare { + private static final Logger logger = LoggerFactory.getLogger(PaxosCommitAndPrepare.class); + public static final RequestSerializer requestSerializer = new RequestSerializer(); public static final RequestHandler requestHandler = new RequestHandler(); @@ -68,7 +74,22 @@ static PaxosPrepare commitAndPrepare(Agreed commit, Paxos.Participants participa * * All these things are tractable to do better, but for now doing something simple and correct. */ - if (readCommand.metadata().replicationType().isTracked()) + boolean shouldBeTracked = MigrationRouter.shouldUseTrackedForWrites(commit.metadata().keyspace, + commit.metadata().id, + commit.partitionKey().getToken()); + + // Reconcile the mutation's ID with the current migration state. + // The commit may have been saved to system.paxos under a different replication type. + if (!shouldBeTracked && !commit.mutation.id().isNone()) + { + logger.warn("Stripping mutation ID {} from PaxosCommitAndPrepare for {}.{} partition {} - keyspace migrated to untracked", + commit.mutation.id(), commit.metadata().keyspace, commit.metadata().name, commit.partitionKey()); + Tracing.trace("Stripping mutation ID {} from PaxosCommitAndPrepare for {}.{} partition {} - keyspace migrated to untracked", + commit.mutation.id(), commit.metadata().keyspace, commit.metadata().name, commit.partitionKey()); + commit = commit.withMutationId(MutationId.none()); + } + + if (shouldBeTracked) { /* * Consistency for consensus is tricky to pick here. The goal of sending this commit is to unblock the prepare @@ -169,7 +190,17 @@ public static class RequestHandler implements IVerbHandler @Override public void doVerb(Message message) { - ClusterMetadataService.instance().fetchLogFromPeerOrCMS(ClusterMetadata.current(), message.from(), message.epoch()); + ClusterMetadata metadata = ClusterMetadata.current(); + + Agreed commit = message.payload.commit; + boolean coordinatorSaysTracked = !commit.mutation.id().isNone(); + metadata = MigrationRouter.checkPaxosCommitMigration(metadata, message, message.from(), + commit.metadata().keyspace, commit.metadata().id, + commit.partitionKey().getToken(), + coordinatorSaysTracked); + + if (message.payload.read != null) + MigrationRouter.checkPaxosPrepareReadMigration(metadata, message, message.from(), message.payload.read); Future response = execute(message.payload, new RequestTime(message.createdAtNanos())); if (response == null) diff --git a/src/java/org/apache/cassandra/service/paxos/PaxosCommitForwardHandler.java b/src/java/org/apache/cassandra/service/paxos/PaxosCommitForwardHandler.java index 728b9b78a573..ee1e35cc8d6d 100644 --- a/src/java/org/apache/cassandra/service/paxos/PaxosCommitForwardHandler.java +++ b/src/java/org/apache/cassandra/service/paxos/PaxosCommitForwardHandler.java @@ -28,17 +28,21 @@ import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.NoPayload; +import org.apache.cassandra.replication.MutationId; import org.apache.cassandra.service.StorageProxy; +import org.apache.cassandra.service.replication.migration.MigrationRouter; +import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.transport.Dispatcher; /** * Handler for forwarded Paxos V1 commit requests. - * Executes the commit operation on behalf of the original coordinator, - * ensuring that MutationId generation happens on a replica coordinator. + * Routes the commit to the tracked or untracked path based on the current + * migration state of the keyspace for the affected partition. * * TODO (expected): more comprehensive testing + * TODO: should loop on CoordinatorBehindException rather than propagating failure to the forwarding coordinator */ public class PaxosCommitForwardHandler implements IVerbHandler { @@ -48,35 +52,59 @@ public class PaxosCommitForwardHandler implements IVerbHandler message) { - // PaxosV1 when doing commit picks whatever the current replicas are to send the commits to - // so make sure we at least match what they would have picked - ClusterMetadataService.instance().fetchLogFromPeerOrCMS(message.from(), message.header.epoch); PaxosCommitForwardRequest request = message.payload; + Commit proposal = request.proposal; - Tracing.trace("Executing forwarded Paxos commit for {}", request.proposal.partitionKey()); + Tracing.trace("Executing forwarded Paxos commit for {}", proposal.partitionKey()); try { - String ksName = request.proposal.metadata().keyspace; + String ksName = proposal.metadata().keyspace; Keyspace keyspace = Keyspace.openIfExists(ksName); if (keyspace == null) { MessagingService.instance().respondWithFailure(RequestFailureReason.INCOMPATIBLE_SCHEMA, message); - logger.error("Failed to forward paxos commit for non-existent keyspace {}", ksName); + logger.error("Failed to forward paxos commit for non-existent keyspace {}.{} partition {}", + ksName, proposal.metadata().name, proposal.partitionKey()); return; } - if (!keyspace.getMetadata().params.replicationType.isTracked()) + ClusterMetadata metadata = ClusterMetadata.current(); + boolean shouldBeTracked = MigrationRouter.shouldUseTrackedForWrites(metadata, + ksName, + proposal.metadata().id, + proposal.partitionKey().getToken()); + + if (!shouldBeTracked && message.epoch().isAfter(metadata.epoch)) { - MessagingService.instance().respondWithFailure(RequestFailureReason.INCOMPATIBLE_SCHEMA, message); - logger.error("Asked to perform forwarded paxos commit, but keyspace {} is not tracked", ksName); - return; + metadata = ClusterMetadataService.instance().fetchLogFromPeerOrCMS(metadata, message.from(), message.epoch()); + shouldBeTracked = MigrationRouter.shouldUseTrackedForWrites(metadata, + ksName, + proposal.metadata().id, + proposal.partitionKey().getToken()); } - // Call commitPaxosTracked which handles mutation ID generation, sending to all replicas, - // and tracking. The respondAfterSend flag determines if we wait for application. - StorageProxy.commitPaxosTracked(keyspace, request.proposal, request.consistencyLevel, - Dispatcher.RequestTime.forImmediateExecution(), request.respondAfterSend); + if (shouldBeTracked) + { + StorageProxy.commitPaxosTracked(keyspace, proposal, request.consistencyLevel, + Dispatcher.RequestTime.forImmediateExecution(), request.respondAfterSend); + } + else + { + // respondAfterSend is not propagated here — commitPaxosUntracked always blocks. + // During migration races this adds latency but doesn't affect correctness. + Commit reconciled = proposal; + if (!proposal.mutation.id().isNone()) + { + logger.warn("Stripping mutation ID {} from forwarded PaxosCommit for {}.{} partition {} - keyspace is untracked at handler", + proposal.mutation.id(), ksName, proposal.metadata().name, proposal.partitionKey()); + Tracing.trace("Stripping mutation ID {} from forwarded PaxosCommit for {}.{} partition {} - keyspace is untracked at handler", + proposal.mutation.id(), ksName, proposal.metadata().name, proposal.partitionKey()); + reconciled = proposal.withMutationId(MutationId.none()); + } + StorageProxy.commitPaxosUntracked(keyspace, reconciled, request.consistencyLevel, + true, Dispatcher.RequestTime.forImmediateExecution()); + } MessagingService.instance().respond(NoPayload.noPayload, message); } diff --git a/src/java/org/apache/cassandra/service/paxos/PaxosPrepare.java b/src/java/org/apache/cassandra/service/paxos/PaxosPrepare.java index 7f0a38c8a8f6..97de0841e3f5 100644 --- a/src/java/org/apache/cassandra/service/paxos/PaxosPrepare.java +++ b/src/java/org/apache/cassandra/service/paxos/PaxosPrepare.java @@ -71,6 +71,7 @@ import org.apache.cassandra.service.reads.tracked.TrackedRead.DataRequest; import org.apache.cassandra.service.reads.tracked.TrackedRead.Id; import org.apache.cassandra.service.reads.tracked.TrackedRead.SummaryRequest; +import org.apache.cassandra.service.replication.migration.MigrationRouter; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.Epoch; @@ -346,6 +347,7 @@ private RetryDifferentSystem(Participants participants) private final List> readResponses; private boolean haveReadResponseWithLatest; private boolean haveTrackedDataResponseIfNeeded; + private boolean isTrackedRead; private boolean haveQuorumOfPermissions; // permissions => SUCCESS or READ_SUCCESS private @Nonnull List withLatest; // promised and have latest commit private @Nullable List needLatest; // promised without having witnessed latest commit, nor yet been refreshed by us @@ -430,7 +432,9 @@ private static PaxosPrepare prepareWithBallotInternal(Participants participants, */ static > void start(PaxosPrepare prepare, Participants participants, Message send, BiFunction> selfHandler) { - if (send.payload.read != null && send.payload.read.metadata().replicationType().isTracked()) + boolean tracked = send.payload.read != null && MigrationRouter.shouldUseTracked(send.payload.read); + prepare.isTrackedRead = tracked; + if (tracked) startTracked(prepare, participants, send, selfHandler); else startUntracked(prepare, participants, send, selfHandler); @@ -554,7 +558,7 @@ public synchronized Status awaitUntil(long deadline) private boolean isTracked() { - return request.read != null && request.read.metadata().replicationType().isTracked(); + return isTrackedRead; } private boolean isDone() @@ -1226,7 +1230,12 @@ public static class RequestHandler implements IVerbHandler @Override public void doVerb(Message message) { - ClusterMetadataService.instance().fetchLogFromPeerOrCMS(ClusterMetadata.current(), message.from(), message.epoch()); + ClusterMetadata metadata = ClusterMetadataService.instance().fetchLogFromPeerOrCMS(ClusterMetadata.current(), + message.from(), + message.epoch()); + + if (message.payload.read != null) + MigrationRouter.checkPaxosPrepareReadMigration(metadata, message, message.from(), message.payload.read); try { diff --git a/src/java/org/apache/cassandra/service/paxos/PaxosPrepareRefresh.java b/src/java/org/apache/cassandra/service/paxos/PaxosPrepareRefresh.java index 08a2d84e26fe..a29c09f15957 100644 --- a/src/java/org/apache/cassandra/service/paxos/PaxosPrepareRefresh.java +++ b/src/java/org/apache/cassandra/service/paxos/PaxosPrepareRefresh.java @@ -21,18 +21,20 @@ import java.io.IOException; import java.util.List; +import com.google.common.collect.ImmutableList; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.db.Mutation; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.exceptions.RequestFailure; import org.apache.cassandra.exceptions.RequestFailureReason; import org.apache.cassandra.exceptions.RetryOnDifferentSystemException; import org.apache.cassandra.exceptions.WriteTimeoutException; -import org.apache.cassandra.gms.FailureDetector; import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.locator.EndpointsForToken; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.net.IVerbHandler; @@ -44,6 +46,7 @@ import org.apache.cassandra.replication.MutationTrackingService; import org.apache.cassandra.service.paxos.Commit.Agreed; import org.apache.cassandra.service.paxos.Commit.Committed; +import org.apache.cassandra.service.replication.migration.MigrationRouter; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tracing.Tracing; @@ -78,10 +81,11 @@ interface Callbacks void onRefreshSuccess(Ballot isSupersededBy, InetAddressAndPort from); } - private Message send; + private volatile Message send; private final Callbacks callbacks; private final Paxos.Participants participants; private final boolean isUrgent; + private boolean selfCallbackDelivered; public PaxosPrepareRefresh(Ballot prepared, Paxos.Participants participants, Committed latestCommitted, Callbacks callbacks) { @@ -97,145 +101,146 @@ public PaxosPrepareRefresh(Ballot prepared, Paxos.Participants participants, Com */ void refresh(List refresh) { - // Check if forwarding is needed for tracked keyspaces + selfCallbackDelivered = false; Committed commit = send.payload.missingCommit; + boolean tracked = MigrationRouter.shouldUseTrackedForWrites(commit.metadata().keyspace, + commit.metadata().id, + commit.partitionKey().getToken()); - if (commit.metadata().replicationType().isTracked() && commit.mutation.id().isNone()) + if (tracked && commit.mutation.id().isNone()) { - // Check if we can generate mutation ID locally (are we a replica?) Replica localReplica = participants.all.byEndpoint().get(getBroadcastAddressAndPort()); - - if (localReplica != null) + if (localReplica == null) { - // We ARE a replica - generate mutation ID and update the commit - String keyspaceName = commit.metadata().keyspace; - MutationId mutationId = MutationTrackingService.instance().nextMutationId(keyspaceName, commit.partitionKey().getToken()); - Mutation mutationWithId = commit.makeMutation(mutationId); - Committed commitWithId = new Commit.Committed(commit.ballot, mutationWithId); - - // Update the message payload with the new commit - this.send = Message.out(PAXOS2_PREPARE_REFRESH_REQ, - new Request(send.payload.promised, commitWithId), - isUrgent); - - // For tracked keyspaces, we MUST ALWAYS write to the local journal since we generated the mutation ID. - // This is required for retry purposes: if a remote target fails, the ActiveLogReconciler will try - // to look up the mutation in the local journal. The node that generated the mutation ID is the "owner" - // and must have the mutation available for retries. - // - // We do this BEFORE the main refresh loop to ensure the mutation is in the journal before any - // failure callback can trigger reconciliation. - Response localResponse = null; - try - { - localResponse = RequestHandler.execute(this.send.payload, getBroadcastAddressAndPort()); - if (localResponse == null) - logger.warn("Local execution failed for tracked mutation {}", mutationId); - } - catch (Exception e) - { - logger.warn("Exception writing tracked mutation {} locally", mutationId, e); - } - - // If self is in the refresh list, report the local execution result to callbacks - // We need to do this because the main loop will skip self for tracked keyspaces - for (int i = 0, size = refresh.size(); i < size; ++i) - { - if (shouldExecuteOnSelf(refresh.get(i))) - { - if (localResponse != null) - callbacks.onRefreshSuccess(localResponse.isSupersededBy, getBroadcastAddressAndPort()); - else - callbacks.onRefreshFailure(getBroadcastAddressAndPort(), RequestFailure.UNKNOWN); - break; - } - } - } - else - { - // We're NOT a replica - forward to a replica forwardRefresh(refresh); return; } + if (!generateMutationIdAndPersistLocally(commit, refresh)) + return; + } + else if (!tracked && !commit.mutation.id().isNone()) + { + logger.warn("Stripping mutation ID {} from PaxosPrepareRefresh for {}.{} partition {} - keyspace migrated to untracked", + commit.mutation.id(), commit.metadata().keyspace, commit.metadata().name, commit.partitionKey()); + Tracing.trace("Stripping mutation ID {} from PaxosPrepareRefresh for {}.{} partition {} - keyspace migrated to untracked", + commit.mutation.id(), commit.metadata().keyspace, commit.metadata().name, commit.partitionKey()); + updateSendMessage(commit.withMutationId(MutationId.none())); } - // For tracked keyspaces where we generated the ID above, we already wrote locally. - // For tracked keyspaces where the ID was already present, we still need to ensure local execution. - boolean isTracked = !send.payload.missingCommit.mutation.id().isNone(); + dispatchRefresh(refresh, tracked); + } - // If we just generated the ID above, we already wrote locally - check by examining the original commit - boolean alreadyWroteLocally = commit.metadata().replicationType().isTracked() - && commit.mutation.id().isNone() - && participants.all.byEndpoint().get(getBroadcastAddressAndPort()) != null; - boolean localExecutedSync = alreadyWroteLocally; + /** + * Generates a mutation ID as the local replica, updates the send message, executes locally + * to persist to the journal (for ID ownership), and reports self's callback if self is a target. + * + * @return true if local write succeeded, false if it failed (all target callbacks reported as failure) + */ + private boolean generateMutationIdAndPersistLocally(Committed commit, List refresh) + { + String keyspaceName = commit.metadata().keyspace; + MutationId mutationId = MutationTrackingService.instance().nextMutationId(keyspaceName, commit.partitionKey().getToken()); + updateSendMessage(commit.withMutationId(mutationId)); + selfCallbackDelivered = true; - // For tracked keyspaces where we DIDN'T generate the ID (it was already present), we still need to - // execute locally BEFORE sending to remotes if self is in the refresh list. - if (isTracked && !alreadyWroteLocally) + Response localResponse = null; + try { - for (int i = 0, size = refresh.size(); i < size; ++i) + localResponse = RequestHandler.execute(this.send.payload, getBroadcastAddressAndPort()); + if (localResponse == null) + logger.warn("Local execution failed for tracked mutation {}", mutationId); + } + catch (Exception e) + { + logger.warn("Exception writing tracked mutation {} locally", mutationId, e); + } + + if (localResponse == null) + { + for (InetAddressAndPort target : refresh) + callbacks.onRefreshFailure(target, RequestFailure.UNKNOWN); + return false; + } + + for (int i = 0, size = refresh.size(); i < size; ++i) + { + if (shouldExecuteOnSelf(refresh.get(i))) { - if (shouldExecuteOnSelf(refresh.get(i))) - { - executeOnSelf(); // SYNCHRONOUS - journal write completes here - localExecutedSync = true; - break; - } + callbacks.onRefreshSuccess(localResponse.isSupersededBy, getBroadcastAddressAndPort()); + break; } } + return true; + } - // Now send to remote nodes (and record local execution for non-tracked keyspaces) - boolean executeOnSelf = false; - for (int i = 0, size = refresh.size(); i < size ; ++i) + private void updateSendMessage(Committed commit) + { + this.send = Message.out(PAXOS2_PREPARE_REFRESH_REQ, new Request(send.payload.promised, commit), isUrgent); + } + + /** + * Dispatches refresh to all targets. For tracked keyspaces, self is executed synchronously + * before remotes (unless already handled during ID generation). For untracked keyspaces, + * self is scheduled for async execution after remotes. + */ + private void dispatchRefresh(List refresh, boolean tracked) + { + if (tracked && !selfCallbackDelivered) + executeSelfSynchronously(refresh); + + boolean selfInList = false; + for (int i = 0, size = refresh.size(); i < size; ++i) { InetAddressAndPort destination = refresh.get(i); + if (shouldExecuteOnSelf(destination)) + { + selfInList = true; + continue; + } + if (logger.isTraceEnabled()) logger.trace("Refresh {} and Confirm {} to {}", send.payload.missingCommit, Ballot.toString(send.payload.promised, "Promise"), destination); if (Tracing.isTracing()) Tracing.trace("Refresh {} and Confirm {} to {}", send.payload.missingCommit.ballot, send.payload.promised, destination); - if (shouldExecuteOnSelf(destination)) - { - // For tracked keyspaces, skip self - already executed synchronously above - if (!localExecutedSync) - executeOnSelf = true; - } - else - { - MessagingService.instance().sendWithCallback(send, destination, this); - } + MessagingService.instance().sendWithCallback(send, destination, this); } - // Async local execution only for non-tracked keyspaces - if (executeOnSelf) + if (!tracked && selfInList) PAXOS2_PREPARE_REFRESH_REQ.stage.execute(this::executeOnSelf); } - /** - * Forward the refresh operation to a replica coordinator. - * The replica will generate the mutation ID and send to all refresh nodes. - */ - private void forwardRefresh(List refreshTargets) + private void executeSelfSynchronously(List refresh) { - // Find a live replica to forward to (that's not us) - InetAddressAndPort targetReplica = null; - for (Replica replica : participants.all) + for (int i = 0, size = refresh.size(); i < size; ++i) { - if (!replica.endpoint().equals(getBroadcastAddressAndPort()) && - FailureDetector.instance.isAlive(replica.endpoint())) + if (shouldExecuteOnSelf(refresh.get(i))) { - targetReplica = replica.endpoint(); + executeOnSelf(); break; } } + } + + /** + * Forward the refresh operation to a replica coordinator. + * The replica will generate (or find) the mutation ID and send the refresh to all target nodes. + */ + private void forwardRefresh(List refreshTargets) + { + InetAddressAndPort localEndpoint = getBroadcastAddressAndPort(); + EndpointsForToken liveExcludingSelf = participants.allLive.filter(replica -> !replica.endpoint().equals(localEndpoint)); + InetAddressAndPort targetReplica = liveExcludingSelf.isEmpty() + ? null + : DatabaseDescriptor.getNodeProximity().sortedByProximity(localEndpoint, liveExcludingSelf).get(0).endpoint(); if (targetReplica == null) { - logger.error("No live replica available to forward PaxosPrepareRefresh for {}", + logger.error("No live replica available to forward PaxosPrepareRefresh for {}.{} partition {}", + send.payload.missingCommit.metadata().keyspace, send.payload.missingCommit.metadata().name, send.payload.missingCommit.partitionKey()); - // Report failure for all refresh targets for (InetAddressAndPort target : refreshTargets) callbacks.onRefreshFailure(target, RequestFailure.UNKNOWN); return; @@ -244,54 +249,68 @@ private void forwardRefresh(List refreshTargets) logger.debug("Forwarding PaxosPrepareRefresh to replica {} for mutation ID generation", targetReplica); Tracing.trace("Forwarding PaxosPrepareRefresh to replica {}", targetReplica); - // Create forward request with refresh targets - PrepareRefreshForwardRequest forwardRequest = new PrepareRefreshForwardRequest( - send.payload.promised, - send.payload.missingCommit, - refreshTargets, - isUrgent - ); + ImmutableList immutableRefreshTargets = ImmutableList.copyOf(refreshTargets); + PrepareRefreshForwardRequest forwardRequest = new PrepareRefreshForwardRequest(send.payload.promised, + send.payload.missingCommit, + immutableRefreshTargets, + isUrgent); - Message message = Message.out( - Verb.PAXOS_PREPARE_REFRESH_FORWARD_REQ, forwardRequest, isUrgent); + Message message = Message.out(Verb.PAXOS_PREPARE_REFRESH_FORWARD_REQ, + forwardRequest, isUrgent); - // Send and handle response MessagingService.instance().sendWithCallback(message, targetReplica, - new ForwardCallback(refreshTargets)); + new ForwardCallback(immutableRefreshTargets)); } /** * Callback for forwarded refresh operations. - * Translates forward response to individual refresh callbacks. + * Receives multiple onResponse() calls (non-final + final) as the forward handler + * streams per-target results back incrementally. + * Caches the mutation ID from the first response so subsequent refresh() calls + * can dispatch directly without forwarding. */ private class ForwardCallback implements RequestCallbackWithFailure { private final List refreshTargets; + private final boolean[] reported; ForwardCallback(List refreshTargets) { this.refreshTargets = refreshTargets; + this.reported = new boolean[refreshTargets.size()]; } @Override - public void onResponse(Message message) + public synchronized void onResponse(Message message) { PrepareRefreshForwardResponse response = message.payload; - // Report results for each target - for (int i = 0; i < refreshTargets.size(); i++) + + Message currentSend = send; + if (!response.mutationId.isNone() && currentSend.payload.missingCommit.mutation.id().isNone()) + updateSendMessage(currentSend.payload.missingCommit.withMutationId(response.mutationId)); + + if (response.targetIndex != null && !reported[response.targetIndex]) { - InetAddressAndPort target = refreshTargets.get(i); - Ballot supersededBy = response.supersededBy.get(i); - callbacks.onRefreshSuccess(supersededBy, target); + reported[response.targetIndex] = true; + InetAddressAndPort target = refreshTargets.get(response.targetIndex); + if (PrepareRefreshForwardHandler.FAILED_SENTINEL.equals(response.supersededBy)) + callbacks.onRefreshFailure(target, RequestFailure.UNKNOWN); + else + callbacks.onRefreshSuccess(response.supersededBy, target); } } @Override - public void onFailure(InetAddressAndPort from, RequestFailure reason) + public synchronized void onFailure(InetAddressAndPort from, RequestFailure reason) { - // Report failure for all targets - for (InetAddressAndPort target : refreshTargets) - callbacks.onRefreshFailure(target, reason); + for (int i = 0; i < refreshTargets.size(); i++) + { + if (!reported[i]) + { + reported[i] = true; + callbacks.onRefreshFailure(refreshTargets.get(i), reason); + } + } } } @@ -315,7 +334,10 @@ private void executeOnSelf() { response = RequestHandler.execute(send.payload, getBroadcastAddressAndPort()); if (response == null) + { + onFailure(getBroadcastAddressAndPort(), RequestFailure.UNKNOWN); return; + } } catch (RetryOnDifferentSystemException e) { @@ -365,7 +387,16 @@ public static class RequestHandler implements IVerbHandler @Override public void doVerb(Message message) { - ClusterMetadataService.instance().fetchLogFromPeerOrCMS(ClusterMetadata.current(), message.from(), message.epoch()); + ClusterMetadata metadata = ClusterMetadataService.instance().fetchLogFromPeerOrCMS(ClusterMetadata.current(), + message.from(), + message.epoch()); + + Committed commit = message.payload.missingCommit; + MigrationRouter.checkPaxosCommitMigration(metadata, message, message.from(), + commit.metadata().keyspace, commit.metadata().id, + commit.partitionKey().getToken(), + !commit.mutation.id().isNone()); + try { Response response = execute(message.payload, message.from()); diff --git a/src/java/org/apache/cassandra/service/paxos/PaxosRepair.java b/src/java/org/apache/cassandra/service/paxos/PaxosRepair.java index a68ac4e54bcd..1f25dcad80dd 100644 --- a/src/java/org/apache/cassandra/service/paxos/PaxosRepair.java +++ b/src/java/org/apache/cassandra/service/paxos/PaxosRepair.java @@ -343,7 +343,7 @@ public State execute(Status input) throws Throwable // with a newer ballot) FoundIncompleteAccepted incomplete = input.incompleteAccepted(); - Proposal propose = new Proposal(incomplete.ballot, incomplete.accepted.update); + Proposal propose = new Proposal(incomplete.ballot, incomplete.accepted.mutation); logger.trace("PaxosRepair of {} found incomplete {}", partitionKey(), incomplete.accepted); return PaxosPropose.propose(propose, participants, false, new ProposingRepair(propose)); // we don't know if we're done, so we must restart diff --git a/src/java/org/apache/cassandra/service/paxos/PaxosState.java b/src/java/org/apache/cassandra/service/paxos/PaxosState.java index 6076df00225b..1bb1a4ff4c20 100644 --- a/src/java/org/apache/cassandra/service/paxos/PaxosState.java +++ b/src/java/org/apache/cassandra/service/paxos/PaxosState.java @@ -59,6 +59,7 @@ import org.apache.cassandra.service.paxos.uncommitted.PaxosBallotTracker; import org.apache.cassandra.service.paxos.uncommitted.PaxosStateTracker; import org.apache.cassandra.service.paxos.uncommitted.PaxosUncommittedTracker; +import org.apache.cassandra.service.replication.migration.MigrationRouter; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.Nemesis; @@ -729,18 +730,28 @@ private static void applyCommit(Commit commit, PaxosState state, BiConsumer { public static final PrepareRefreshForwardHandler instance = new PrepareRefreshForwardHandler(); private static final Logger logger = LoggerFactory.getLogger(PrepareRefreshForwardHandler.class); + static final Ballot FAILED_SENTINEL = Ballot.none(); @Override public void doVerb(Message message) { - ClusterMetadataService.instance().fetchLogFromPeerOrCMS(message.from(), message.header.epoch); + ClusterMetadata metadata = ClusterMetadata.current(); PrepareRefreshForwardRequest request = message.payload; - Tracing.trace("Executing forwarded PaxosPrepareRefresh for {}", request.commit.partitionKey()); try { String ksName = request.commit.metadata().keyspace; - KeyspaceMetadata ksMetadata = Schema.instance.getKeyspaceMetadata(ksName); - if (ksMetadata == null) - { - MessagingService.instance().respondWithFailure(RequestFailureReason.INCOMPATIBLE_SCHEMA, message); - logger.error("Failed to forward paxos prepare refresh for non-existent keyspace {}", ksName); - return; - } + Token token = request.commit.partitionKey().getToken(); - if (!ksMetadata.params.replicationType.isTracked()) + if (metadata.schema.getKeyspaces().getNullable(ksName) == null) { MessagingService.instance().respondWithFailure(RequestFailureReason.INCOMPATIBLE_SCHEMA, message); - logger.error("Asked to perform forwarded prepare refresh, but keyspace {} is not tracked", ksName); + logger.error("Failed to forward prepare refresh for non-existent keyspace {}.{} partition {}", + ksName, request.commit.metadata().name, request.commit.partitionKey()); return; } - Token token = request.commit.partitionKey().getToken(); + MigrationRouter.checkPaxosCommitMigration(metadata, message, message.from(), + ksName, request.commit.metadata().id, token, true); + MutationId mutationId = MutationTrackingService.instance().nextMutationId(ksName, token); Mutation mutationWithId = request.commit.makeMutation(mutationId); Committed commitWithId = new Commit.Committed(request.commit.ballot, mutationWithId); - // Now send the refresh to all targets and collect responses - List targets = request.refreshTargets; - List supersededBy = Collections.synchronizedList(new ArrayList<>(Collections.nCopies(targets.size(), null))); - CountDownLatch latch = CountDownLatch.newCountDownLatch(targets.size()); - - Message refreshMsg = Message.out( - PAXOS2_PREPARE_REFRESH_REQ, - new PaxosPrepareRefresh.Request(request.promised, commitWithId), - request.isUrgent - ); - // For tracked keyspaces, we MUST ALWAYS write to the local journal since we generated the mutation ID. // This is required for retry purposes: if a remote target fails, the ActiveLogReconciler will try // to look up the mutation in the local journal. The node that generated the mutation ID is the "owner" // and must have the mutation available for retries. - // - // This is different from checking if self is in targets - even if we're not in targets, - // we're still the ID generator and need the mutation locally. + PaxosPrepareRefresh.Response localResponse = null; try { - PaxosPrepareRefresh.RequestHandler.execute( - new PaxosPrepareRefresh.Request(request.promised, commitWithId), FBUtilities.getBroadcastAddressAndPort()); - // Note: we don't use the response since this node may not be in targets + localResponse = PaxosPrepareRefresh.RequestHandler.execute(new PaxosPrepareRefresh.Request(request.promised, commitWithId), + FBUtilities.getBroadcastAddressAndPort()); } catch (Exception e) { - // Log but continue - we still need to send to targets logger.warn("Failed to execute local commit for tracked keyspace mutation {}", mutationId, e); } - // Now send to remote targets + if (localResponse == null) + { + MessagingService.instance().respondWithFailure(RequestFailureReason.UNKNOWN, message); + logger.error("Aborting forwarded PaxosPrepareRefresh: local journal write failed for mutation {}", mutationId); + return; + } + + List targets = request.refreshTargets; + int[] remaining = new int[] { targets.size() }; + + Message refreshMsg = Message.out(PAXOS2_PREPARE_REFRESH_REQ, + new PaxosPrepareRefresh.Request(request.promised, commitWithId), + request.isUrgent); + for (int i = 0; i < targets.size(); i++) { final int targetIndex = i; InetAddressAndPort target = targets.get(i); - // Check if self is in targets for response tracking (separate from the local write above) - // We need to decrement the latch for the local target since we already executed above if (shouldExecuteOnSelf(target)) { - latch.decrement(); + respond(message, mutationId, targetIndex, localResponse.isSupersededBy, remaining); continue; } @@ -144,36 +131,18 @@ public void doVerb(Message message) @Override public void onResponse(Message response) { - supersededBy.set(targetIndex, response.payload.isSupersededBy); - latch.decrement(); + respond(message, mutationId, targetIndex, response.payload.isSupersededBy, remaining); } @Override public void onFailure(InetAddressAndPort from, RequestFailure reason) { - // Leave null to indicate we didn't get a definitive answer - latch.decrement(); + respond(message, mutationId, targetIndex, FAILED_SENTINEL, remaining); } }; MessagingService.instance().sendWithCallback(refreshMsg, target, callback); } - - // Wait for all responses with timeout - long timeoutNanos = message.expiresAtNanos() - Clock.Global.nanoTime(); - boolean completed = latch.await(Math.max(0, timeoutNanos), TimeUnit.NANOSECONDS); - - if (!completed) - logger.warn("Forwarded PaxosPrepareRefresh timed out waiting for responses"); - - // Send aggregated response back to original coordinator - MessagingService.instance().respond(new PrepareRefreshForwardResponse(supersededBy), message); - } - catch (InterruptedException e) - { - Thread.currentThread().interrupt(); - MessagingService.instance().respondWithFailure(RequestFailure.forException(e), message); - logger.error("Forwarded PaxosPrepareRefresh interrupted", e); } catch (Exception e) { @@ -181,4 +150,19 @@ public void onFailure(InetAddressAndPort from, RequestFailure reason) logger.error("Failed to execute forwarded PaxosPrepareRefresh for {}", request.commit, e); } } + + private void respond(Message request, MutationId mutationId, int targetIndex, + @Nullable Ballot supersededBy, int[] remaining) + { + PrepareRefreshForwardResponse payload = new PrepareRefreshForwardResponse(mutationId, targetIndex, supersededBy); + Message response = request.responseWith(payload); + //noinspection SynchronizationOnLocalVariableOrMethodParameter + synchronized (remaining) + { + boolean isFinal = --remaining[0] == 0; + if (!isFinal) + response = response.withFlag(MessageFlag.NOT_FINAL); + MessagingService.instance().send(response, request.respondTo()); + } + } } diff --git a/src/java/org/apache/cassandra/service/paxos/PrepareRefreshForwardResponse.java b/src/java/org/apache/cassandra/service/paxos/PrepareRefreshForwardResponse.java index ec6a38b87c64..e914ac856ee4 100644 --- a/src/java/org/apache/cassandra/service/paxos/PrepareRefreshForwardResponse.java +++ b/src/java/org/apache/cassandra/service/paxos/PrepareRefreshForwardResponse.java @@ -19,30 +19,44 @@ package org.apache.cassandra.service.paxos; import java.io.IOException; -import java.util.List; +import javax.annotation.Nullable; + +import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.utils.CollectionSerializers; +import org.apache.cassandra.replication.MutationId; import org.apache.cassandra.utils.NullableSerializer; /** * Response from a forwarded PaxosPrepareRefresh operation. - * Contains the superseding ballot for each refresh target (null if confirmed). + * Each response carries a single target result (targetIndex + supersededBy), + * or just the mutation ID if targetIndex is null. + * Multiple non-final responses are sent incrementally as targets respond, + * followed by a final response for the last target. */ public class PrepareRefreshForwardResponse { public static final Serializer serializer = new Serializer(); - /** - * List of superseding ballots, one per refresh target. - * Null entry means the promise was confirmed for that target. - */ - public final List supersededBy; + public final MutationId mutationId; + @Nullable + public final Integer targetIndex; + @Nullable + public final Ballot supersededBy; + + public PrepareRefreshForwardResponse(MutationId mutationId) + { + this.mutationId = mutationId; + this.targetIndex = null; + this.supersededBy = null; + } - public PrepareRefreshForwardResponse(List supersededBy) + public PrepareRefreshForwardResponse(MutationId mutationId, int targetIndex, @Nullable Ballot supersededBy) { + this.mutationId = mutationId; + this.targetIndex = targetIndex; this.supersededBy = supersededBy; } @@ -53,20 +67,39 @@ public static class Serializer implements IVersionedSerializer supersededBy = CollectionSerializers.deserializeList(in, version, NULLABLE_BALLOT_SERIALIZER); - return new PrepareRefreshForwardResponse(supersededBy); + MutationId mutationId = MutationId.serializer.deserialize(in, version); + boolean hasTarget = in.readBoolean(); + if (!hasTarget) + return new PrepareRefreshForwardResponse(mutationId); + int targetIndex = in.readUnsignedVInt32(); + Ballot supersededBy = NULLABLE_BALLOT_SERIALIZER.deserialize(in, version); + return new PrepareRefreshForwardResponse(mutationId, targetIndex, supersededBy); } @Override public long serializedSize(PrepareRefreshForwardResponse response, int version) { - return CollectionSerializers.serializedListSize(response.supersededBy, version, NULLABLE_BALLOT_SERIALIZER); + long size = MutationId.serializer.serializedSize(response.mutationId, version); + size += 1; // hasTarget boolean + if (response.targetIndex != null) + { + size += TypeSizes.sizeofUnsignedVInt(response.targetIndex); + size += NULLABLE_BALLOT_SERIALIZER.serializedSize(response.supersededBy, version); + } + return size; } } } diff --git a/src/java/org/apache/cassandra/service/replication/migration/MigrationRouter.java b/src/java/org/apache/cassandra/service/replication/migration/MigrationRouter.java index bd699710e914..c500755cd99c 100644 --- a/src/java/org/apache/cassandra/service/replication/migration/MigrationRouter.java +++ b/src/java/org/apache/cassandra/service/replication/migration/MigrationRouter.java @@ -25,19 +25,28 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.EmbeddableSinglePartitionReadCommand; import org.apache.cassandra.db.IMutation; import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.PartitionRangeReadCommand; -import org.apache.cassandra.db.SinglePartitionReadCommand; 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.locator.InetAddressAndPort; +import org.apache.cassandra.metrics.TCMMetrics; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.utils.Pair; import static com.google.common.base.Preconditions.checkState; @@ -49,23 +58,14 @@ */ public class MigrationRouter { - public static boolean shouldUseTracked(SinglePartitionReadCommand command) + private static final Logger logger = LoggerFactory.getLogger(MigrationRouter.class); + + public static boolean shouldUseTracked(EmbeddableSinglePartitionReadCommand 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); + return shouldUseTracked(ClusterMetadata.current(), command); } /** @@ -215,13 +215,18 @@ public static List splitRangeRead(ClusterMetadata meta PartitionRangeReadCommand command) { // System keyspaces never use tracked replication - if (SchemaConstants.isSystemKeyspace(command.metadata().keyspace)) + String keyspace = command.metadata().keyspace; + if (SchemaConstants.isSystemKeyspace(keyspace) || command.metadata().kind != TableMetadata.Kind.REGULAR) + return ImmutableList.of(new RangeReadWithReplication(command, false)); + + KeyspaceMetadata ksm = metadata.schema.maybeGetKeyspaceMetadata(keyspace).orElse(null); + if (ksm == null) return ImmutableList.of(new RangeReadWithReplication(command, false)); KeyspaceMigrationInfo migrationInfo = metadata.mutationTrackingMigrationState - .getKeyspaceInfo(command.metadata().keyspace); + .getKeyspaceInfo(keyspace); - boolean isTracked = command.metadata().replicationType().isTracked(); + boolean isTracked = ksm.params.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 @@ -254,16 +259,26 @@ public static boolean shouldUseTrackedForWrites(ClusterMetadata metadata, String if (SchemaConstants.isSystemKeyspace(keyspace)) return false; - KeyspaceMigrationInfo migrationInfo = metadata.mutationTrackingMigrationState - .getKeyspaceInfo(keyspace); + KeyspaceMigrationInfo migrationInfo = metadata.mutationTrackingMigrationState.getKeyspaceInfo(keyspace); if (migrationInfo == null) - return metadata.schema.getKeyspaceMetadata(keyspace).params.replicationType.isTracked(); + { + KeyspaceMetadata ksm = metadata.schema.maybeGetKeyspaceMetadata(keyspace).orElse(null); + return ksm != null && ksm.params.replicationType.isTracked(); + } - boolean isTracked = metadata.schema.getKeyspaceMetadata(keyspace).params.replicationType.isTracked(); + KeyspaceMetadata ksm = metadata.schema.maybeGetKeyspaceMetadata(keyspace).orElse(null); + if (ksm == null) + return false; + boolean isTracked = ksm.params.replicationType.isTracked(); return migrationInfo.shouldUseTrackedForWrites(isTracked, tableId, token); } + public static boolean shouldUseTrackedForWrites(String keyspace, TableId tableId, Token token) + { + return shouldUseTrackedForWrites(ClusterMetadata.current(), keyspace, tableId, token); + } + public static class RoutedMutations { public final List trackedMutations; @@ -392,4 +407,116 @@ public static void validateUntrackedMutation(IMutation mutation) { validateMutationReplication(mutation, MutationRouting.UNTRACKED); } + + /** + * Validate that coordinator and handler agree on tracked/untracked routing for a Paxos commit. + * Fetches log from coordinator only on mismatch when coordinator epoch is ahead. + * + * @return updated ClusterMetadata after any fetch + */ + public static ClusterMetadata checkPaxosCommitMigration(ClusterMetadata metadata, + Message message, + InetAddressAndPort respondTo, + String keyspace, + TableId tableId, + Token token, + boolean coordinatorSaysTracked) + { + boolean handlerSaysTracked = shouldUseTrackedForWrites(metadata, keyspace, tableId, token); + if (coordinatorSaysTracked == handlerSaysTracked) + return metadata; + + if (message.epoch().isAfter(metadata.epoch)) + { + metadata = ClusterMetadataService.instance().fetchLogFromPeerOrCMS(metadata, respondTo, message.epoch()); + handlerSaysTracked = shouldUseTrackedForWrites(metadata, keyspace, tableId, token); + if (coordinatorSaysTracked == handlerSaysTracked) + return metadata; + } + + if (message.epoch().isBefore(metadata.epoch)) + { + TCMMetrics.instance.coordinatorBehindReplication.mark(); + logger.warn("COORDINATOR_BEHIND: Paxos commit migration mismatch for keyspace {} table {} token {}, coordinator {} at epoch {} is behind our epoch {}", + keyspace, tableId, token, respondTo, message.epoch(), metadata.epoch); + throw new CoordinatorBehindException(String.format("Paxos commit migration mismatch for keyspace: %s token %s, coordinator: %s is behind, our epoch = %s, their epoch = %s", + keyspace, token, respondTo, metadata.epoch, message.epoch())); + } + else + { + logger.error("Inconsistent Paxos commit routing at same epoch {} for keyspace {} table {} token {} - coordinator says tracked={}, handler says tracked={}", + metadata.epoch, keyspace, tableId, token, coordinatorSaysTracked, handlerSaysTracked); + throw new IllegalStateException(String.format("Inconsistent Paxos commit routing at epoch = %s. Keyspace: %s token: %s", + metadata.epoch, keyspace, token)); + } + } + + /** + * Validate that a Paxos prepare read type matches the handler's migration state. + * Uses the conditional-fetch pattern: check first, fetch only on mismatch when coordinator is ahead. + * + * Both directions are validated: + * - Tracked read → untracked handler: CRITICAL — tracked reads lose monotonicity when writes are untracked. + * - Untracked read → tracked handler: less bad (untracked quorum reads are self-consistent) but still + * indicates epoch disagreement that should be resolved. + * + * @return updated ClusterMetadata after any fetch + */ + public static ClusterMetadata checkPaxosPrepareReadMigration(ClusterMetadata metadata, + Message message, + InetAddressAndPort respondTo, + EmbeddableSinglePartitionReadCommand read) + { + boolean coordinatorSaysTracked = read.isTracked(); + boolean handlerSaysTracked = shouldUseTracked(metadata, read); + if (coordinatorSaysTracked == handlerSaysTracked) + return metadata; + + if (message.epoch().isAfter(metadata.epoch)) + { + metadata = ClusterMetadataService.instance().fetchLogFromPeerOrCMS(metadata, respondTo, message.epoch()); + handlerSaysTracked = shouldUseTracked(metadata, read); + if (coordinatorSaysTracked == handlerSaysTracked) + return metadata; + } + + if (message.epoch().isBefore(metadata.epoch)) + { + TCMMetrics.instance.coordinatorBehindReplication.mark(); + logger.warn("COORDINATOR_BEHIND: Paxos prepare read migration mismatch for {}.{} partition {}, coordinator {} at epoch {} is behind our epoch {}", + read.metadata().keyspace, read.metadata().name, read.partitionKey(), respondTo, message.epoch(), metadata.epoch); + throw new CoordinatorBehindException(String.format("Paxos prepare read migration mismatch for keyspace: %s token %s, coordinator: %s is behind, our epoch = %s, their epoch = %s", + read.metadata().keyspace, read.partitionKey().getToken(), respondTo, metadata.epoch, message.epoch())); + } + else + { + logger.error("Inconsistent Paxos prepare read routing at same epoch {} for {}.{} partition {} - coordinator says tracked={}, handler says tracked={}", + metadata.epoch, read.metadata().keyspace, read.metadata().name, read.partitionKey(), coordinatorSaysTracked, handlerSaysTracked); + throw new IllegalStateException(String.format("Inconsistent Paxos prepare read routing at epoch = %s. Keyspace: %s token: %s", + metadata.epoch, read.metadata().keyspace, read.partitionKey().getToken())); + } + } + + public static boolean shouldUseTracked(ClusterMetadata metadata, EmbeddableSinglePartitionReadCommand command) + { + if (command.metadata().kind != TableMetadata.Kind.REGULAR) + return false; + + String keyspace = command.metadata().keyspace; + if (SchemaConstants.isSystemKeyspace(keyspace)) + return false; + + KeyspaceMetadata ksm = metadata.schema.maybeGetKeyspaceMetadata(keyspace).orElse(null); + if (ksm == null) + return false; + + KeyspaceMigrationInfo migrationInfo = metadata.mutationTrackingMigrationState.getKeyspaceInfo(keyspace); + + boolean isTracked = ksm.params.replicationType.isTracked(); + if (migrationInfo == null) + return isTracked; + + Token token = command.partitionKey().getToken(); + return migrationInfo.shouldUseTrackedForReads(isTracked, command.metadata().id(), token); + } } diff --git a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java index 41cb2884597f..6ec149970d0f 100644 --- a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java +++ b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java @@ -1773,4 +1773,47 @@ public static Integer instanceId(IInvokableInstance instance) return null; }); } + + /** + * Wait for all nodes in the cluster to reach at least the highest TCM epoch currently observed. + * Useful after tests that block TCM propagation to specific nodes. + */ + public static void awaitTCMCatchUp(ICluster cluster) + { + long maxEpoch = 0; + boolean foundRunning = false; + for (int i = 1; i <= cluster.size(); i++) + { + IInvokableInstance inst = (IInvokableInstance) cluster.get(i); + if (inst.isShutdown()) + continue; + foundRunning = true; + maxEpoch = Math.max(maxEpoch, inst.callOnInstance(() -> ClusterMetadata.current().epoch.getEpoch())); + } + if (!foundRunning) + throw new AssertionError("No running nodes to await TCM catch-up on"); + + long target = maxEpoch; + long deadline = System.nanoTime() + TimeUnit.SECONDS.toNanos(30); + List lagging = new ArrayList<>(); + for (int i = 1; i <= cluster.size(); i++) + { + IInvokableInstance inst = (IInvokableInstance) cluster.get(i); + if (inst.isShutdown()) + continue; + int node = i; + while (System.nanoTime() < deadline) + { + long nodeEpoch = inst.callOnInstance(() -> ClusterMetadata.current().epoch.getEpoch()); + if (nodeEpoch >= target) + break; + sleepUninterruptibly(100, TimeUnit.MILLISECONDS); + } + long finalEpoch = inst.callOnInstance(() -> ClusterMetadata.current().epoch.getEpoch()); + if (finalEpoch < target) + lagging.add(String.format("node%d(epoch=%d)", node, finalEpoch)); + } + if (!lagging.isEmpty()) + throw new AssertionError(String.format("Timed out waiting for nodes to reach epoch %d; lagging: %s", target, lagging)); + } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/tracking/AssertingLatch.java b/test/distributed/org/apache/cassandra/distributed/test/tracking/AssertingLatch.java new file mode 100644 index 000000000000..312db061141c --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/tracking/AssertingLatch.java @@ -0,0 +1,83 @@ +/* + * 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.tracking; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +/** + * A CountDownLatch wrapper that throws on timeout. Interruption during shutdown is benign. + */ +public class AssertingLatch +{ + private static final long DEFAULT_TIMEOUT = 30; + private static final TimeUnit DEFAULT_UNIT = TimeUnit.SECONDS; + + private final CountDownLatch latch; + private final String description; + private final long timeout; + private final TimeUnit unit; + + public AssertingLatch(String description) + { + this(1, DEFAULT_TIMEOUT, DEFAULT_UNIT, description); + } + + public AssertingLatch(int count, String description) + { + this(count, DEFAULT_TIMEOUT, DEFAULT_UNIT, description); + } + + public AssertingLatch(int count, long timeout, TimeUnit unit, String description) + { + this.latch = new CountDownLatch(count); + this.description = description; + this.timeout = timeout; + this.unit = unit; + } + + public void await() + { + try + { + if (!latch.await(timeout, unit)) + throw new AssertionError("Timed out after " + timeout + " " + unit + " waiting for: " + description); + } + catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + } + } + + public void countDown() + { + latch.countDown(); + } + + public void release() + { + while (latch.getCount() > 0) + latch.countDown(); + } + + public long getCount() + { + return latch.getCount(); + } +} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMigrationTestUtils.java b/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMigrationTestUtils.java new file mode 100644 index 000000000000..39bef320e2b5 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMigrationTestUtils.java @@ -0,0 +1,669 @@ +/* + * 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.tracking; + +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.cassandra.Util; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.Mutation; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.dht.Token; +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.IMessage; +import org.apache.cassandra.distributed.api.IMessageFilters; +import org.apache.cassandra.distributed.impl.Instance; +import org.apache.cassandra.exceptions.CasWriteTimeoutException; +import org.apache.cassandra.exceptions.CasWriteUnknownResultException; +import org.apache.cassandra.exceptions.RequestFailureReason; +import org.apache.cassandra.exceptions.WriteFailureException; +import org.apache.cassandra.exceptions.WriteTimeoutException; +import org.apache.cassandra.hints.HintsService; +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.MutationTrackingService; +import org.apache.cassandra.schema.ReplicationType; +import org.apache.cassandra.service.paxos.Commit; +import org.apache.cassandra.tcm.ClusterMetadata; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Utility methods for Paxos mutation tracking migration tests. + * + * Reduces boilerplate around deserializing messages, extracting MutationIds, + * generating synthetic failure responses, stranding nodes at old TCM epochs, and + * composing common cluster-level filter-and-count spy patterns. + */ +public class PaxosMigrationTestUtils +{ + private static final AtomicInteger KEYSPACE_COUNTER = new AtomicInteger(); + + private PaxosMigrationTestUtils() + { + } + + // Must be called inside callsOnInstance() or runOnInstance() on the receiving node + public static boolean messageHasMutationId(IMessage msg) + { + Message deserialized = Instance.deserializeMessage(msg); + return payloadHasMutationId(deserialized.payload); + } + + // Must be called inside runOnInstance() on the receiving node + public static void respondWithTimeout(IMessage msg) + { + Message deserialized = Instance.deserializeMessage(msg); + MessagingService.instance().respondWithFailure(RequestFailureReason.TIMEOUT, deserialized); + } + + private static MutationId extractMutationIdFromPayload(Object payload) + { + if (payload instanceof Commit) + return ((Commit) payload).mutation.id(); + + if (payload instanceof Mutation) + return ((Mutation) payload).id(); + + try + { + for (Field f : payload.getClass().getDeclaredFields()) + { + if (f.getName().equals("missingCommit") || f.getName().equals("mutation") || f.getName().equals("commit")) + { + f.setAccessible(true); + Object val = f.get(payload); + if (val instanceof Commit) + return ((Commit) val).mutation.id(); + if (val instanceof Mutation) + return ((Mutation) val).id(); + } + } + } + catch (IllegalAccessException e) + { + throw new RuntimeException(e); + } + + return MutationId.none(); + } + + private static boolean payloadHasMutationId(Object payload) + { + return !extractMutationIdFromPayload(payload).isNone(); + } + + public static void awaitReplicationType(Cluster cluster, String keyspace, ReplicationType expected, int... nodes) + { + boolean expectTracked = expected == ReplicationType.tracked; + long deadline = System.nanoTime() + TimeUnit.SECONDS.toNanos(30); + for (int node : nodes) + { + while (System.nanoTime() < deadline) + { + boolean isTracked = cluster.get(node).callOnInstance(() -> + ClusterMetadata.current().schema.getKeyspaceMetadata(keyspace).params.replicationType.isTracked()); + if (isTracked == expectTracked) + break; + Thread.yield(); + } + boolean isTracked = cluster.get(node).callOnInstance(() -> + ClusterMetadata.current().schema.getKeyspaceMetadata(keyspace).params.replicationType.isTracked()); + if (isTracked != expectTracked) + throw new AssertionError("Node " + node + " did not see replicationType " + + expected + " for " + keyspace + " within 30s"); + } + } + + // --- Cluster builder helpers --- + + /** + * Build a Cluster.Builder preconfigured for Paxos migration tests: NETWORK + GOSSIP, given + * paxos_variant, long (10s) write/CAS contention timeouts. The caller is responsible for + * calling .start() and wrapping in init() (which requires TestBaseImpl access), and for + * invoking {@link #pauseHintsAndReconciler(Cluster)} after the cluster is live. + */ + public static Cluster.Builder buildPaxosCluster(int nodes, String paxosVariant) + { + return Cluster.build(nodes) + .withConfig(cfg -> cfg.with(Feature.NETWORK) + .with(Feature.GOSSIP) + .set("paxos_variant", paxosVariant) + .set("write_request_timeout", "10000ms") + .set("cas_contention_timeout", "10000ms")); + } + + /** + * Pause hint delivery and the regular-priority mutation tracking reconciler on every node in + * the cluster. Intended to be called once in @BeforeClass after the cluster is started so that + * hints and reconciliation traffic don't bleed into tests. + */ + public static void pauseHintsAndReconciler(Cluster cluster) + { + cluster.forEach(instance -> instance.runOnInstance(() -> HintsService.instance.pauseDispatch())); + cluster.forEach(instance -> instance.runOnInstance(() -> MutationTrackingService.instance().pauseActiveReconcilerRegularPriority())); + } + + // --- Keyspace / schema helpers --- + + /** + * Creates a fresh keyspace with unique name (prefix_N) using SimpleStrategy at the requested + * RF and replication_type, plus a standard table {@code tbl (k int PRIMARY KEY, v int, v2 int)} + * covering every field used across the paxos-migration test suite. + * + * @return the generated keyspace name. + */ + public static String createKeyspace(Cluster cluster, String prefix, String replicationType, int rf) + { + String ks = prefix + '_' + KEYSPACE_COUNTER.incrementAndGet(); + cluster.schemaChange("CREATE KEYSPACE " + ks + " WITH replication = " + + "{'class': 'SimpleStrategy', 'replication_factor': " + rf + "} " + + "AND replication_type='" + replicationType + "'"); + cluster.schemaChange("CREATE TABLE " + ks + ".tbl (k int PRIMARY KEY, v int, v2 int)"); + return ks; + } + + /** Convenience overload defaulting to RF=3. */ + public static String createKeyspace(Cluster cluster, String prefix, String replicationType) + { + return createKeyspace(cluster, prefix, replicationType, 3); + } + + public static void awaitMigrationComplete(Cluster cluster, String keyspace) + { + String ks = keyspace; + long deadline = System.nanoTime() + TimeUnit.SECONDS.toNanos(30); + while (true) + { + boolean migrating = cluster.get(1).callOnInstance(() -> + ClusterMetadata.current().mutationTrackingMigrationState.isMigrating(ks)); + if (!migrating) + return; + if (System.nanoTime() >= deadline) + throw new AssertionError("Migration did not complete within 30s for keyspace " + keyspace); + try { Thread.sleep(100); } catch (InterruptedException e) { Thread.currentThread().interrupt(); return; } + } + } + + public static void alterReplicationType(Cluster cluster, String keyspace, String replicationType) + { + cluster.schemaChange("ALTER KEYSPACE " + keyspace + " WITH replication = " + + "{'class': 'SimpleStrategy', 'replication_factor': 3} " + + "AND replication_type='" + replicationType + "'"); + } + + public static void alterReplicationTypeFrom(Cluster cluster, + int coordinator, + String keyspace, + String replicationType, + ConsistencyLevel cl) + { + cluster.coordinator(coordinator).execute("ALTER KEYSPACE " + keyspace + " WITH replication = " + + "{'class': 'SimpleStrategy', 'replication_factor': 3} " + + "AND replication_type='" + replicationType + "'", + cl); + } + + // --- Assertion helpers --- + + public static void assertReplicasHaveValue(Cluster cluster, String keyspace, int key, Object value, int... nodes) + { + for (int node : nodes) + { + Util.spinAssertEquals(value, + () -> { + Object[][] r = cluster.get(node).executeInternal("SELECT v FROM " + keyspace + ".tbl WHERE k = " + key); + return r.length == 1 ? r[0][0] : null; + }); + } + } + + public static void assertReplicaHasNoRow(Cluster cluster, String keyspace, int key, int node) + { + Object[][] nodeResult = cluster.get(node).executeInternal("SELECT v FROM " + keyspace + ".tbl WHERE k = " + key); + assertEquals("Node " + node + " should have no row for k=" + key, 0, nodeResult.length); + } + + public static void assertAllNodesSee(Cluster cluster, String keyspace, ReplicationType expected) + { + for (int i = 1; i <= cluster.size(); i++) + assertNodeSees(cluster, i, keyspace, expected); + } + + public static void assertNodeSees(Cluster cluster, int node, String keyspace, ReplicationType expected) + { + boolean expectTracked = expected == ReplicationType.tracked; + boolean isTracked = cluster.get(node).callOnInstance(() -> + ClusterMetadata.current().schema.getKeyspaceMetadata(keyspace).params.replicationType.isTracked()); + assertEquals("Node " + node + " should see replicationType " + expected + " for " + keyspace, + expectTracked, isTracked); + } + + /** + * Fail-fast assertion: if the partitioner or token allocation changes such that {@code key} + * no longer maps to the expected replica set, every test that depends on that placement is + * silently invalid. Validate explicitly up-front so a placement regression fails loudly + * rather than via vacuous message counts. + */ + public static void assertReplicasAreExactly(Cluster cluster, String keyspace, int key, int[] expectedReplicas) + { + String[] actualReplicas = cluster.get(1).applyOnInstance((String ks, Integer k) -> { + Keyspace keyspaceObj = Keyspace.open(ks); + Token token = keyspaceObj.getColumnFamilyStore("tbl").getPartitioner() + .getToken(Int32Type.instance.decompose(k)); + return keyspaceObj.getReplicationStrategy() + .calculateNaturalReplicas(token, ClusterMetadata.current()) + .endpoints().stream() + .map(ep -> ep.getAddress().getHostAddress() + ":" + ep.getPort()) + .toArray(String[]::new); + }, keyspace, key); + + String[] expected = new String[expectedReplicas.length]; + for (int i = 0; i < expectedReplicas.length; i++) + expected[i] = cluster.get(expectedReplicas[i]).broadcastAddress().getAddress().getHostAddress() + + ":" + cluster.get(expectedReplicas[i]).broadcastAddress().getPort(); + + Set actualSet = new HashSet<>(Arrays.asList(actualReplicas)); + Set expectedSet = new HashSet<>(Arrays.asList(expected)); + assertEquals("KEY=" + key + " placement assumption violated. Actual replicas: " + actualSet + + " Expected: " + expectedSet + " — tests that depend on this placement are invalid.", + expectedSet, actualSet); + } + + /** + * Recognize a CAS-related write exception by simple-name string comparison. Uses string + * comparison because the exception may have been deserialized in a different classloader + * (in-JVM dtest nodes use isolated classloaders) and {@code instanceof} would return false. + */ + public static void assertCasException(Exception e) + { + String actual = e.getClass().getName(); + boolean recognized = actual.equals(CasWriteTimeoutException.class.getName()) + || actual.equals(CasWriteUnknownResultException.class.getName()) + || actual.equals(WriteTimeoutException.class.getName()) + || actual.equals(WriteFailureException.class.getName()); + assertTrue("Expected a CAS-related write exception but got " + actual + ": " + e.getMessage(), + recognized); + } + + public static void assertCasApplied(Object[][] result) + { + assertNotNull("CAS should return a result", result); + assertEquals(1, result.length); + assertTrue("CAS should be applied", (boolean) result[0][0]); + } + + public static void assertCasNotApplied(Object[][] result) + { + assertNotNull("CAS should return a result", result); + assertEquals(1, result.length); + assertFalse("CAS should NOT be applied", (boolean) result[0][0]); + } + + // --- Async CAS helpers --- + + /** + * Execute a CAS on the given coordinator at SERIAL/QUORUM asynchronously. The returned + * future completes with the result or completes exceptionally. + */ + public static CompletableFuture casAsync(Cluster cluster, int coordinator, String cql) + { + return CompletableFuture.supplyAsync(() -> + cluster.coordinator(coordinator).execute(cql, ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM)); + } + + /** + * Execute a CAS expected to fail. The returned future completes with the raised Throwable + * (or null if the CAS unexpectedly succeeded). + */ + public static CompletableFuture casAsyncExpectingFailure(Cluster cluster, int coordinator, String cql) + { + return CompletableFuture.supplyAsync(() -> { + try + { + cluster.coordinator(coordinator).execute(cql, ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + return null; + } + catch (Throwable t) + { + return t; + } + }); + } + + // --- EpochPin: hold TCM traffic so a node stays at its old epoch --- + + /** + * Install filters that pin a node at its current TCM epoch by holding inbound + * TCM_REPLICATION / TCM_NOTIFY_REQ messages targeted at the node, and outbound + * TCM_FETCH_CMS_LOG_REQ / TCM_FETCH_PEER_LOG_REQ messages originating from it, until + * {@link EpochPin#close()} releases the hold latch. Messages are held rather than + * dropped so they eventually flow and the node catches up before subsequent tests — an + * indefinite fetchLogFromPeerOrCMS stall would bleed into later tests. + */ + public static EpochPin epochPin(Cluster cluster, int node) + { + return new EpochPin(cluster, node); + } + + public static final class EpochPin implements AutoCloseable + { + private final AssertingLatch releaseTcm; + private final IMessageFilters.Filter inboundFilter; + private final IMessageFilters.Filter outboundFilter; + + private EpochPin(Cluster cluster, int node) + { + this.releaseTcm = new AssertingLatch("EpochPin release for node " + node); + this.inboundFilter = cluster.filters() + .inbound(true) + .verbs(Verb.TCM_REPLICATION.id, Verb.TCM_NOTIFY_REQ.id) + .to(node) + .messagesMatching((from, to, msg) -> { + releaseTcm.await(); + return false; + }).drop(); + this.outboundFilter = cluster.filters() + .inbound(true) + .verbs(Verb.TCM_FETCH_CMS_LOG_REQ.id, Verb.TCM_FETCH_PEER_LOG_REQ.id) + .from(node) + .messagesMatching((from, to, msg) -> { + releaseTcm.await(); + return false; + }).drop(); + } + + @Override + public void close() + { + // Release the hold so held messages flow through the filter. We deliberately do NOT + // call .off() here; the filter remains installed so any late messages still flow + // rather than being silently dropped. Callers typically reset filters via cluster.filters().reset() + // in @After. + releaseTcm.countDown(); + } + + public IMessageFilters.Filter inboundFilter() + { + return inboundFilter; + } + + public IMessageFilters.Filter outboundFilter() + { + return outboundFilter; + } + } + + // --- MessageSpy: fluent wrapper over cluster filter + counter + latch boilerplate --- + + /** + * Create a {@link Builder} that configures a spy (or dropper) over one or more message verbs + * on the given cluster. Default direction is inbound. Returns a {@link MessageSpy} once + * {@link Builder#start()} is called. + */ + public static Builder on(Cluster cluster, Verb... verbs) + { + return new Builder(cluster, verbs); + } + + /** + * Fluent builder for {@link MessageSpy}. Configures direction, source/destination nodes, + * mutation-id checking, expected message count, hold/release behavior, and whether messages + * should be dropped or allowed through. + */ + public static final class Builder + { + private final Cluster cluster; + private final int[] verbIds; + private int[] fromNodes; + private int[] toNodes; + private boolean inbound = true; + private boolean checkMutationId = false; + private int expect = 0; + private boolean holdAll = false; + private int holdFirst = 0; + private boolean drop = false; + private final List observers = new ArrayList<>(); + + private Builder(Cluster cluster, Verb... verbs) + { + this.cluster = cluster; + this.verbIds = new int[verbs.length]; + for (int i = 0; i < verbs.length; i++) + this.verbIds[i] = verbs[i].id; + } + + public Builder from(int... nodes) + { + this.fromNodes = nodes; + return this; + } + + public Builder to(int... nodes) + { + this.toNodes = nodes; + return this; + } + + /** Set the filter to apply on inbound (the default) or outbound traffic. */ + public Builder inbound() + { + this.inbound = true; + return this; + } + + public Builder inbound(boolean inbound) + { + this.inbound = inbound; + return this; + } + + /** Deserialize each matching message on its target node and count those carrying a mutation id. */ + public Builder checkMutationId() + { + this.checkMutationId = true; + return this; + } + + /** Arm an internal {@link AssertingLatch} so {@link MessageSpy#await()} waits for N matching messages. */ + public Builder expect(int count) + { + this.expect = count; + return this; + } + + /** + * Hold every matching message on an internal latch until {@link MessageSpy#release()} is + * called. Messages are still counted and (if configured) checked for a mutation id before + * blocking. + */ + public Builder holdAll() + { + this.holdAll = true; + return this; + } + + /** Hold only the first {@code n} matching messages; the rest flow through immediately. */ + public Builder holdFirst(int n) + { + this.holdFirst = n; + return this; + } + + /** Drop matching messages (don't deliver). Default is spy-only — messages flow through. */ + public Builder drop() + { + this.drop = true; + return this; + } + + /** Register an extra observer invoked on every matching message (after counting / holding). */ + public Builder onEach(IMessageFilters.Matcher observer) + { + this.observers.add(observer); + return this; + } + + public MessageSpy start() + { + return new MessageSpy(this); + } + } + + /** + * A running message spy — counts matching messages, optionally holds them, optionally checks + * for mutation ids, and fires an expectation latch once the target count is reached. + * + * Closing via {@link #close()} turns the underlying filter off and releases any held messages. + */ + public static final class MessageSpy implements AutoCloseable + { + private final Cluster cluster; + private final AtomicInteger total = new AtomicInteger(); + private final AtomicInteger withMutationId = new AtomicInteger(); + private final AtomicInteger held = new AtomicInteger(); + private final AtomicInteger passedThrough = new AtomicInteger(); + private final AssertingLatch deliveryLatch; + private final AssertingLatch firstArrivalLatch; + private final AssertingLatch holdLatch; + private final IMessageFilters.Filter filter; + private final boolean drop; + private final int holdFirst; + private final boolean holdAll; + + private MessageSpy(Builder b) + { + this.cluster = b.cluster; + this.drop = b.drop; + this.holdFirst = b.holdFirst; + this.holdAll = b.holdAll; + this.deliveryLatch = b.expect > 0 ? new AssertingLatch(b.expect, "MessageSpy delivery (expect=" + b.expect + ")") + : null; + this.firstArrivalLatch = new AssertingLatch("MessageSpy first arrival"); + this.holdLatch = (b.holdAll || b.holdFirst > 0) ? new AssertingLatch("MessageSpy hold release") : null; + + IMessageFilters.Builder fb = cluster.filters().inbound(b.inbound).verbs(b.verbIds); + if (b.fromNodes != null) + fb = fb.from(b.fromNodes); + if (b.toNodes != null) + fb = fb.to(b.toNodes); + + final boolean checkId = b.checkMutationId; + final List observers = b.observers; + this.filter = fb.messagesMatching((from, to, msg) -> { + total.incrementAndGet(); + firstArrivalLatch.countDown(); + + if (checkId) + { + boolean hasId = cluster.get(to).callsOnInstance(() -> messageHasMutationId(msg)).call(); + if (hasId) + withMutationId.incrementAndGet(); + } + + for (IMessageFilters.Matcher observer : observers) + observer.matches(from, to, msg); + + // Hold before signaling deliveryLatch so the held-message-count stays accurate + // while the hold is still active. + boolean shouldHold = holdAll || (holdFirst > 0 && total.get() <= holdFirst); + if (shouldHold) + { + held.incrementAndGet(); + holdLatch.await(); + } + else + { + passedThrough.incrementAndGet(); + } + + if (deliveryLatch != null) + deliveryLatch.countDown(); + + return drop; + }).drop(); + } + + public int total() + { + return total.get(); + } + + public int withMutationId() + { + return withMutationId.get(); + } + + public int held() + { + return held.get(); + } + + public int passedThrough() + { + return passedThrough.get(); + } + + /** Block until the {@code expect(N)} count is reached. Throws if no expectation was set. */ + public void await() + { + if (deliveryLatch == null) + throw new IllegalStateException("await() requires expect(N) to have been set at builder time"); + deliveryLatch.await(); + } + + /** Block until at least one matching message has arrived. */ + public void awaitFirstArrival() + { + firstArrivalLatch.await(); + } + + /** Release messages blocked by {@code holdAll()} / {@code holdFirst(N)}. */ + public void release() + { + if (holdLatch == null) + throw new IllegalStateException("release() requires holdAll() or holdFirst(N) to have been set at builder time"); + holdLatch.countDown(); + } + + @Override + public void close() + { + // Release any held messages so filter threads don't leak if the test fails early. + if (holdLatch != null) + holdLatch.countDown(); + filter.off(); + } + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMutationTrackingForwardingV1Test.java b/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMutationTrackingForwardingV1Test.java new file mode 100644 index 000000000000..02ff3fee4318 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMutationTrackingForwardingV1Test.java @@ -0,0 +1,290 @@ +/* + * 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.tracking; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.shared.ClusterUtils; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.MessageSpy; +import org.apache.cassandra.net.Verb; + +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.alterReplicationType; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertCasApplied; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertReplicasAreExactly; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertReplicasHaveValue; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.buildPaxosCluster; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.casAsync; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.createKeyspace; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.messageHasMutationId; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.on; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.pauseHintsAndReconciler; +import static org.junit.Assert.assertEquals; + +/** + * Tests for V1 Paxos handler-level forwarding paths with CAS-level forwarding disabled. + * + * Tests the V1 commitPaxos forwarding path (StorageProxy.forwardPaxosCommit) which is invoked when + * the coordinator is not a replica for a tracked keyspace. + * + * KEY=5 maps to replicas on nodes 1,2,3 (node 4 excluded) with Murmur3Partitioner RF=3. + */ +public class PaxosMutationTrackingForwardingV1Test extends TestBaseImpl +{ + private static Cluster cluster; + private static final int KEY = 5; + + @BeforeClass + public static void setup() throws Throwable + { + CassandraRelevantProperties.DISABLE_CONSENSUS_REQUEST_FORWARDING.setBoolean(true); + cluster = init(buildPaxosCluster(4, "v1").start()); + pauseHintsAndReconciler(cluster); + } + + @AfterClass + public static void teardown() + { + if (cluster != null) + cluster.close(); + CassandraRelevantProperties.DISABLE_CONSENSUS_REQUEST_FORWARDING.reset(); + } + + @After + public void resetFilters() + { + cluster.filters().reset(); + ClusterUtils.awaitTCMCatchUp(cluster); + } + + private String newKeyspace(String replicationType) + { + String ks = createKeyspace(cluster, "pmt_fwd_v1", replicationType); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + return ks; + } + + /* + * V1 commitPaxos forwarding from non-replica coordinator + * + * Covers: StorageProxy.forwardPaxosCommit() via commitPaxos() + * forwardPaxosCommit(reconciled, consistencyLevel, replicaPlan); + * + * With CAS-level forwarding disabled, a CAS from node 4 (non-replica for KEY=5) + * executes locally through V1 doPaxos -> commitPaxos. Inside commitPaxos, + * shouldBeTracked=true and requiresPaxosCommitForwarding returns + * true (node 4 not in replicas) -> forwardPaxosCommit fires, forwarding the commit to a replica + * via PaxosCommitForwardHandler. + */ + @Test + public void testV1CommitForwardingFromNonReplica() throws Throwable + { + String ks = newKeyspace("tracked"); + + try (MessageSpy spy = on(cluster, Verb.PAXOS_COMMIT_FORWARD_REQ) + .expect(1) + .start()) + { + Object[][] result = cluster.coordinator(4) + .execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, + ConsistencyLevel.QUORUM); + + assertCasApplied(result); + spy.await(); + assertEquals("PAXOS_COMMIT_FORWARD_REQ should have been sent exactly once", + 1, spy.total()); + } + + assertReplicasHaveValue(cluster, ks, KEY, 42, 1, 2, 3); + } + + /* + * V1 commit forwarding with migration to untracked during forward + * + * Covers: PaxosCommitForwardHandler.doVerb() untracked fallback path + * + * CAS from node 4 (non-replica) forwards the commit to a replica via + * PAXOS_COMMIT_FORWARD_REQ. The forward is delayed at the replica. During the delay, + * the keyspace migrates to untracked. When the handler processes the forward, it finds + * the keyspace is untracked and commits via the untracked path. No mutation ID needs to + * be stripped because the proposal arrives with MutationId.none() — the ID is only + * assigned inside commitPaxosTracked, which forwardPaxosCommit bypasses. + */ + @Test + public void testV1CommitForwardingFallbackToUntracked() throws Throwable + { + String ks = newKeyspace("tracked"); + + // Delay PAXOS_COMMIT_FORWARD_REQ at the receiving replica until ALTER completes. + // After migration to untracked, the forward handler commits via the untracked path. + // The proposal never had a mutation ID (it is only assigned inside commitPaxosTracked), + // so we verify no PAXOS_COMMIT_REQ carries one. + MessageSpy hold = on(cluster, Verb.PAXOS_COMMIT_FORWARD_REQ) + .holdAll() + .start(); + try (MessageSpy commitSpy = on(cluster, Verb.PAXOS_COMMIT_REQ) + .to(1, 2, 3) + .checkMutationId() + .start()) + { + CompletableFuture casResult = casAsync(cluster, 4, + "INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 99) IF NOT EXISTS"); + + hold.awaitFirstArrival(); + alterReplicationType(cluster, ks, "untracked"); + hold.release(); + + Object[][] result = casResult.get(60, TimeUnit.SECONDS); + assertCasApplied(result); + + assertReplicasHaveValue(cluster, ks, KEY, 99, 1, 2, 3); + + assertEquals("No PAXOS_COMMIT_REQ should carry a mutation ID after fallback to untracked", + 0, commitSpy.withMutationId()); + } + finally + { + // Ensure release + filter off even if assertions fail, so filter threads don't leak. + hold.close(); + } + } + + /* + * V1 commit forwarding with COORDINATOR_BEHIND retry after migration + * + * Covers: StorageProxy.java forwardPaxosCommit callback + commitPaxos retry loop + * + * The forward handler (node 1) still thinks tracked, calls commitPaxosTracked, sends + * PAXOS_COMMIT_REQ to nodes 2,3 which have already migrated to untracked. They reject + * with COORDINATOR_BEHIND. The forward handler catches the CoordinatorBehindException + * and sends COORDINATOR_BEHIND back to the original coordinator (node 4). The coordinator's + * retry loop should catch this and retry with fresh metadata (untracked path). + * + * Without the fix: the callback wraps COORDINATOR_BEHIND as WriteTimeoutException, which + * propagates out of the retry loop and the CAS fails with CasWriteTimeout. + */ + @Test + public void testV1CommitForwardingRetryAfterCoordinatorBehind() throws Throwable + { + String ks = newKeyspace("tracked"); + + // Pre-insert so the CAS condition is met on first try + cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 1)", + ConsistencyLevel.ALL); + + // Strategy: Hold PAXOS_COMMIT_REQ at nodes 2 and 3 (the non-forwarding replicas). + // While held, ALTER the keyspace to untracked. When released, nodes 2,3 see the + // commit request from the stale forward handler (node 1) and reject with + // COORDINATOR_BEHIND. This bubbles back to node 4's retry loop. + AssertingLatch commitArrived = new AssertingLatch("PAXOS_COMMIT_REQ at nodes 2,3"); + AssertingLatch alterDone = new AssertingLatch("V1 commit forwarding retry - alter to untracked"); + + // Count ALL inbound PAXOS_COMMIT_REQ at nodes 2,3 regardless of source. After the retry, + // node 4 (now on untracked) calls commitPaxosUntracked directly and sends to replicas + // {1,2,3}, yielding 2 more messages at {2,3}. Total = initial 2 + retry 2 = 4. + // If the retry didn't happen (regression), total would be 2 and the CAS would fail. + AtomicInteger commitsAtReplicas = new AtomicInteger(); + + // Hold PAXOS_COMMIT_REQ at nodes 2 and 3 until ALTER completes. + // Also check retry messages (from node 4) for mutation ID absence. One filter handles + // both roles because the from-node distinguishes initial (node 1) from retry (node 4) + // and two overlapping MessageSpy instances on the same verb+destination would not + // compose cleanly — the cluster filter system applies ALL matching filters. + AtomicInteger retryCommitsWithId = new AtomicInteger(); + cluster.filters() + .inbound(true) + .verbs(Verb.PAXOS_COMMIT_REQ.id) + .to(2, 3) + .messagesMatching((from, to, msg) -> { + commitsAtReplicas.incrementAndGet(); + // Only hold commits from node 1 (the forward handler acting as coordinator) + if (from == 1) + { + commitArrived.countDown(); + alterDone.await(); + } + else if (from == 4) + { + boolean hasId = cluster.get(to).callsOnInstance(() -> messageHasMutationId(msg)).call(); + if (hasId) + retryCommitsWithId.incrementAndGet(); + } + return false; + }).drop(); + + // Start the CAS from node 4 (non-replica) async + CompletableFuture casResult = casAsync(cluster, 4, + "UPDATE " + ks + ".tbl SET v = 100 WHERE k = " + KEY + " IF v = 1"); + + // Wait for the PAXOS_COMMIT_REQ to arrive (proves forward path was taken), then ALTER + // to untracked while the commit is held. All nodes see this immediately. Release the + // held commits in finally so the held filter thread never strands if a step throws. + try + { + commitArrived.await(); + + alterReplicationType(cluster, ks, "untracked"); + } + finally + { + // Release the held commits — nodes 2,3 now see migration mismatch -> COORDINATOR_BEHIND + alterDone.countDown(); + } + + // With the fix: node 4's retry loop catches CoordinatorBehindException, retries + // with fresh metadata (untracked), succeeds. + // Without the fix: WriteTimeoutException escapes the loop -> test fails. + try + { + Object[][] result = casResult.get(30, TimeUnit.SECONDS); + assertCasApplied(result); + } + catch (Exception e) + { + throw new AssertionError("CAS should succeed via COORDINATOR_BEHIND retry but got: " + e.getMessage(), e); + } + + // Verify the write took effect on each replica individually. The retry path uses + // commitPaxosUntracked at QUORUM, so all 3 replicas (1,2,3) should have the data + // (node 1 gets it via the tracked local write in the initial forward, nodes 2,3 via retry). + assertReplicasHaveValue(cluster, ks, KEY, 100, 1, 2, 3); + + // Authenticate that the retry actually happened by counting inbound PAXOS_COMMIT_REQ at + // the two non-forwarding replicas. Initial attempt (from node 1, held then released) = 2, + // retry (from node 4 directly on untracked path) = 2. Total = 4. A result of 2 would mean + // the CAS succeeded without retry (bug). + assertEquals("Expected initial (2) + retry (2) = 4 PAXOS_COMMIT_REQ at nodes 2,3", + 4, commitsAtReplicas.get()); + + // Retry messages from node 4 (untracked path) must not carry mutation IDs + assertEquals("Retry PAXOS_COMMIT_REQ from node 4 should NOT carry mutation IDs (untracked path)", + 0, retryCommitsWithId.get()); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMutationTrackingForwardingV2Test.java b/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMutationTrackingForwardingV2Test.java new file mode 100644 index 000000000000..5d0e639213e1 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMutationTrackingForwardingV2Test.java @@ -0,0 +1,573 @@ +/* + * 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.tracking; + +import java.lang.reflect.Field; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.impl.Instance; +import org.apache.cassandra.distributed.shared.ClusterUtils; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.MessageSpy; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.Verb; +import org.apache.cassandra.service.paxos.Commit; + +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.alterReplicationType; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertCasApplied; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertCasNotApplied; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertReplicaHasNoRow; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertReplicasAreExactly; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertReplicasHaveValue; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.buildPaxosCluster; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.casAsync; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.createKeyspace; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.messageHasMutationId; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.on; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.pauseHintsAndReconciler; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.respondWithTimeout; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class PaxosMutationTrackingForwardingV2Test extends TestBaseImpl +{ + private static Cluster cluster; + private static final int KEY = 5; + + @BeforeClass + public static void setup() throws Throwable + { + CassandraRelevantProperties.DISABLE_CONSENSUS_REQUEST_FORWARDING.setBoolean(true); + cluster = init(buildPaxosCluster(4, "v2").start()); + pauseHintsAndReconciler(cluster); + } + + @AfterClass + public static void teardown() + { + if (cluster != null) + cluster.close(); + CassandraRelevantProperties.DISABLE_CONSENSUS_REQUEST_FORWARDING.reset(); + } + + @After + public void resetFilters() + { + cluster.filters().reset(); + ClusterUtils.awaitTCMCatchUp(cluster); + } + + private String newKeyspace(String replicationType) + { + String ks = createKeyspace(cluster, "pmt_fwd", replicationType); + ClusterUtils.awaitTCMCatchUp(cluster); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + return ks; + } + + @Test + public void testPrepareRefreshForwardHandler() throws Throwable + { + String ks = newKeyspace("untracked"); + + // Partial commit on node 1 only (commits to replicas 2 and 3 blocked) so they remain stale + // when the keyspace flips to tracked. The subsequent CAS from node 4 will force a + // prepare-refresh that exercises PrepareRefreshForwardHandler. + cluster.filters().verbs(Verb.PAXOS_COMMIT_REQ.id).from(1).to(2, 3, 4).drop(); + + try + { + cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + fail("CAS should have thrown because commits to nodes 2,3,4 were blocked"); + } + catch (Exception e) + { + // First CAS is expected to throw: the commit phase was intentionally blocked to 2,3,4. + } + + cluster.filters().reset(); + + alterReplicationType(cluster, ks, "tracked"); + ClusterUtils.awaitTCMCatchUp(cluster); + + // Count PAXOS_PREPARE_REFRESH_FORWARD_REQ arrivals at any replica. + MessageSpy forwardSpy = on(cluster, Verb.PAXOS_PREPARE_REFRESH_FORWARD_REQ) + .inbound() + .start(); + + // Custom spy: extract the mutation ID string from each PAXOS2_PREPARE_REFRESH_REQ payload so + // we can assert that the forward handler produced a non-empty mutation ID. MessageSpy can + // only count by ID presence — it cannot capture the string representation — so this filter + // stays manual. + AtomicInteger refreshWithMutationId = new AtomicInteger(); + Set observedMutationIds = ConcurrentHashMap.newKeySet(); + cluster.filters() + .inbound(true) + .verbs(Verb.PAXOS2_PREPARE_REFRESH_REQ.id) + .messagesMatching((from, to, msg) -> { + String mutId = cluster.get(to).callsOnInstance(() -> { + try + { + Message deserialized = Instance.deserializeMessage(msg); + Object payload = deserialized.payload; + Field f = payload.getClass().getDeclaredField("missingCommit"); + f.setAccessible(true); + Commit commit = (Commit) f.get(payload); + if (!commit.mutation.id().isNone()) + return commit.mutation.id().toString(); + return null; + } + catch (Exception e) + { + throw new RuntimeException(e); + } + }).call(); + if (mutId != null) + { + refreshWithMutationId.incrementAndGet(); + observedMutationIds.add(mutId); + } + return false; + }).drop(); + + // Drop prepare responses from node 3 so quorum can only be formed by node 1 + node 2. + // Node 1 has the latest committed ballot; when its response arrives (in any order relative + // to node 2), the coordinator will see withLatest()=1 < quorum=2 with + // haveReadResponseWithLatest=true, triggering refreshStaleParticipants rather than + // FOUND_INCOMPLETE_COMMITTED. + cluster.filters() + .inbound(true) + .verbs(Verb.PAXOS2_PREPARE_RSP.id) + .from(3) + .to(4) + .drop(); + + // The CAS may timeout during the commit phase due to migration state, but the handler's + // effect is observable via the intercepted PAXOS2_PREPARE_REFRESH_REQ messages. + boolean casThrew = false; + Object[][] casResult = null; + try + { + try + { + casResult = cluster.coordinator(4).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 99) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + } + catch (Exception e) + { + casThrew = true; + } + + // Node 2 is the sole stale participant (it responded with an older commit). Node 3 never + // responded to prepare (its PAXOS2_PREPARE_RSP was dropped), so it is NOT in needLatest + // and is not targeted for refresh. The forward sends refresh only to node 2. + int forwards = forwardSpy.total(); + assertTrue("At least 1 forward should have been sent, got " + forwards, + forwards >= 1); + + // The stale participant (node 2) should receive a refresh with a mutation ID. This may + // come via a network PAXOS2_PREPARE_REFRESH_REQ (counted here) or via local execution when + // the forward handler is itself the target (shouldExecuteOnSelf). + int refreshes = refreshWithMutationId.get(); + assertTrue("Expected 1 refresh with mutation ID, got " + refreshes, + refreshes >= 1); + + assertFalse("Observed mutation IDs should not be empty", observedMutationIds.isEmpty()); + + // Two deterministic possibilities for the CAS outcome: + // - CAS succeeded: IF NOT EXISTS saw v=42 → not applied + // - CAS threw: migration state disrupted commit phase + // In either case v=99 must not be committed: the IF NOT EXISTS precondition fails because + // the first CAS's v=42 is present on node 1's system.paxos. + if (!casThrew) + assertCasNotApplied(casResult); + } + finally + { + cluster.filters().reset(); + forwardSpy.close(); + } + + // Prepare-refresh commits v=42 to at least a quorum of the 3 replicas as part of the + // prepare phase. Whether the subsequent CAS commit succeeds or times out, the refresh + // durability guarantee still holds. + int nodesWithData = 0; + for (int i : new int[]{ 1, 2, 3 }) + { + Object[][] nodeResult = cluster.get(i).executeInternal("SELECT v FROM " + ks + ".tbl WHERE k = " + KEY); + if (nodeResult.length == 1 && Integer.valueOf(42).equals(nodeResult[0][0])) + nodesWithData++; + } + assertTrue("Quorum (2 or 3) of replicas should have v=42 after prepare-refresh, but " + nodesWithData + " had it", + nodesWithData >= 2 && nodesWithData <= 3); + } + + /** + * FAILED_SENTINEL path when quorum is NOT achievable: the sole refresh target (node 2) fails + * (message intercepted with immediate failure), causing a FAILED_SENTINEL response. The CAS + * must fail (not hang) because quorum cannot be met (only node 1 has the latest commit; + * node 3 never responded to prepare so is not targeted for refresh). + * + * Verifies sentinel response on the wire, CAS fails with appropriate error, no data on stale + * nodes, no deadlock. + */ + @Test + public void testPrepareRefreshForwardHandlerTargetFailure() throws Throwable + { + String ks = newKeyspace("untracked"); + + cluster.filters().verbs(Verb.PAXOS_COMMIT_REQ.id).from(1).to(2, 3, 4).drop(); + + try + { + cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + fail("CAS should have thrown because commits to nodes 2,3,4 were blocked"); + } + catch (Exception e) + { + // Expected — commit phase intentionally blocked. + } + + cluster.filters().reset(); + + alterReplicationType(cluster, ks, "tracked"); + ClusterUtils.awaitTCMCatchUp(cluster); + + // Intercept PAXOS2_PREPARE_REFRESH_REQ at nodes 2 and 3, immediately respond with failure + // (simulating timeout), and drop the original. In practice only node 2 will be targeted + // (node 3's prepare response is dropped below so it won't be in needLatest). This triggers + // FAILED_SENTINEL instantly without waiting for the real write_request_timeout. + // respondWithTimeout actively produces a response, so this filter stays manual. + cluster.filters() + .inbound(true) + .verbs(Verb.PAXOS2_PREPARE_REFRESH_REQ.id) + .to(2, 3) + .messagesMatching((from, to, msg) -> { + cluster.get(to).runOnInstance(() -> respondWithTimeout(msg)); + return true; + }).drop(); + + MessageSpy forwardSpy = on(cluster, Verb.PAXOS_PREPARE_REFRESH_FORWARD_REQ) + .inbound() + .start(); + + // Drop prepare responses from node 3 so quorum can only be formed by node 1 + node 2. + // Node 1 has the latest committed ballot, triggering refreshStaleParticipants. + cluster.filters() + .inbound(true) + .verbs(Verb.PAXOS2_PREPARE_RSP.id) + .from(3) + .to(4) + .drop(); + + // Node 2's refresh receives an immediate failure (FAILED_SENTINEL), quorum unreachable, CAS must throw. + boolean casThrew = false; + try + { + try + { + cluster.coordinator(4).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 99) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + } + catch (Exception e) + { + casThrew = true; + } + + assertTrue("CAS should have thrown due to quorum unreachable from FAILED_SENTINEL responses", casThrew); + + int forwards = forwardSpy.total(); + assertTrue("At least 1 forward should have been sent, got " + forwards, forwards >= 1); + + // The sentinel response on the wire proves FAILED_SENTINEL was generated by + // PrepareRefreshForwardHandler and delivered to ForwardCallback. The CAS throwing proves + // the path: PrepareRefreshForwardHandler received an immediate failure response (injected + // via respondWithTimeout), sent Ballot.none() as supersededBy, ForwardCallback routed to + // onRefreshFailure, PaxosPrepare.onFailure incremented failures until quorum became + // unreachable. + } + finally + { + cluster.filters().reset(); + forwardSpy.close(); + } + + // Node 1 always has the data (original committer); neither node 2 nor node 3 did. + assertReplicasHaveValue(cluster, ks, KEY, 42, 1); + assertReplicaHasNoRow(cluster, ks, KEY, 2); + assertReplicaHasNoRow(cluster, ks, KEY, 3); + } + + /** + * When node 3's prepare response is dropped, only node 2 becomes a refresh target (it + * responded with an older commit). The single refresh to node 2 succeeds, achieving quorum + * (node 1 + node 2 = 2). Verifies that quorum is reached with a single successful refresh + * when one electorate member is unresponsive during prepare. + */ + @Test + public void testPrepareRefreshForwardHandlerPartialFailureAchievesQuorum() throws Throwable + { + String ks = newKeyspace("untracked"); + + cluster.filters().verbs(Verb.PAXOS_COMMIT_REQ.id).from(1).to(2, 3, 4).drop(); + + try + { + cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + fail("CAS should have thrown because commits to nodes 2,3,4 were blocked"); + } + catch (Exception e) + { + // Expected — commit phase intentionally blocked. + } + + cluster.filters().reset(); + + alterReplicationType(cluster, ks, "tracked"); + ClusterUtils.awaitTCMCatchUp(cluster); + + // Defensive filter: intercept PAXOS2_PREPARE_REFRESH_REQ at node 3 in case it somehow + // ends up in needLatest. In practice, node 3's prepare response is dropped below so it + // will not be targeted for refresh. Node 2's refresh proceeds unblocked, giving quorum + // (node 1 + node 2 = 2 >= quorum). + cluster.filters() + .inbound(true) + .verbs(Verb.PAXOS2_PREPARE_REFRESH_REQ.id) + .to(3) + .messagesMatching((from, to, msg) -> { + cluster.get(to).runOnInstance(() -> respondWithTimeout(msg)); + return true; + }).drop(); + + MessageSpy forwardSpy = on(cluster, Verb.PAXOS_PREPARE_REFRESH_FORWARD_REQ) + .inbound() + .start(); + + // Drop prepare responses from node 3 so quorum can only be formed by node 1 + node 2. + // Node 1 has the latest committed ballot, triggering refreshStaleParticipants. + cluster.filters() + .inbound(true) + .verbs(Verb.PAXOS2_PREPARE_RSP.id) + .from(3) + .to(4) + .drop(); + + // Only node 2 is targeted for refresh (node 3 never responded to prepare). The single + // refresh succeeds: quorum = 2 with node 1 (has data) + node 2 (refresh success). + try + { + Object[][] casResult = cluster.coordinator(4).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 99) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + // CAS succeeded -- IF NOT EXISTS saw v=42 already present. + assertCasNotApplied(casResult); + + int forwards = forwardSpy.total(); + assertTrue("At least 1 forward should have been sent, got " + forwards, forwards >= 1); + } + finally + { + cluster.filters().reset(); + forwardSpy.close(); + } + + // Node 2 received the refresh; node 3 has no data (it was never targeted for refresh + // because it did not respond to prepare). + assertReplicasHaveValue(cluster, ks, KEY, 42, 2); + assertReplicaHasNoRow(cluster, ks, KEY, 3); + } + + // V2 Commit Forwarding Tests (Paxos2CommitForwardHandler) + + @Test + public void testV2CommitForwardingFromNonReplica() throws Throwable + { + String ks = newKeyspace("tracked"); + + MessageSpy forwardSpy = on(cluster, Verb.PAXOS2_COMMIT_FORWARD_REQ) + .inbound() + .expect(1) + .start(); + + Object[][] result = cluster.coordinator(4).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + assertCasApplied(result); + forwardSpy.await(); + + assertReplicasHaveValue(cluster, ks, KEY, 42, 1, 2, 3); + forwardSpy.close(); + } + + @Test + public void testV2CommitForwardingFallbackToUntracked() throws Throwable + { + String ks = newKeyspace("tracked"); + + // Hold PAXOS2_COMMIT_FORWARD_REQ so we can ALTER the keyspace to untracked mid-flight; + // the forward handler should then commit on the untracked path (no mutation IDs on + // PAXOS_COMMIT_REQ). + MessageSpy hold = on(cluster, Verb.PAXOS2_COMMIT_FORWARD_REQ) + .inbound() + .holdAll() + .start(); + + // The forward handler on the target replica commits locally and sends PAXOS_COMMIT_REQ to + // the other 2 replicas. Quorum = 2, so the 2nd remote commit may still be in-flight when + // the CAS returns — expect(2) waits for both. + MessageSpy commitSpy = on(cluster, Verb.PAXOS_COMMIT_REQ) + .inbound() + .to(1, 2, 3) + .checkMutationId() + .expect(2) + .start(); + + CompletableFuture casFuture = casAsync(cluster, 4, + "INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 99) IF NOT EXISTS"); + + try + { + hold.awaitFirstArrival(); + alterReplicationType(cluster, ks, "untracked"); + } + finally + { + hold.release(); + } + + Object[][] result = casFuture.get(60, TimeUnit.SECONDS); + assertCasApplied(result); + + commitSpy.await(); + cluster.filters().reset(); + + assertReplicasHaveValue(cluster, ks, KEY, 99, 1, 2, 3); + + // After fallback to untracked no commit should carry a mutation ID. + assertEquals("No PAXOS_COMMIT_REQ should carry a mutation ID after fallback to untracked", + 0, commitSpy.withMutationId()); + + hold.close(); + commitSpy.close(); + } + + @Test + public void testV2CommitForwardingRetryAfterCoordinatorBehind() throws Throwable + { + String ks = newKeyspace("tracked"); + + // Pre-insert so the CAS condition (IF v = 1) is met on first try. + cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 1)", + ConsistencyLevel.ALL); + + AtomicInteger forwardTarget = new AtomicInteger(-1); + AssertingLatch commitArrived = new AssertingLatch("PAXOS_COMMIT_REQ from forward target"); + AssertingLatch alterDone = new AssertingLatch("ALTER to untracked to complete"); + AtomicInteger commitsAtReplicas = new AtomicInteger(); + AtomicInteger retryCommitsWithId = new AtomicInteger(); + + // Learn which replica the snitch chose for the forward — any of {1,2,3} is possible. + cluster.filters() + .inbound(true) + .verbs(Verb.PAXOS2_COMMIT_FORWARD_REQ.id) + .messagesMatching((from, to, msg) -> { + forwardTarget.set(to); + return false; + }).drop(); + + // Hold PAXOS_COMMIT_REQ from the forward target until ALTER completes. Count ALL inbound + // PAXOS_COMMIT_REQ at replicas to verify the retry fires, and check retry messages (from + // node 4) for mutation ID absence. Conditional behaviour inside one filter — stays manual. + cluster.filters() + .inbound(true) + .verbs(Verb.PAXOS_COMMIT_REQ.id) + .to(1, 2, 3) + .messagesMatching((from, to, msg) -> { + commitsAtReplicas.incrementAndGet(); + if (from == forwardTarget.get()) + { + commitArrived.countDown(); + alterDone.await(); + } + else if (from == 4) + { + boolean hasId = cluster.get(to).callsOnInstance(() -> messageHasMutationId(msg)).call(); + if (hasId) + retryCommitsWithId.incrementAndGet(); + } + return false; + }).drop(); + + CompletableFuture casFuture = casAsync(cluster, 4, + "UPDATE " + ks + ".tbl SET v = 100 WHERE k = " + KEY + " IF v = 1"); + + try + { + commitArrived.await(); + alterReplicationType(cluster, ks, "untracked"); + } + finally + { + alterDone.release(); + commitArrived.release(); + } + + // With the fix: node 4's retry loop catches CoordinatorBehindException and retries with + // fresh metadata (untracked), succeeding. Without the fix, WriteTimeoutException escapes + // the loop and the test fails. + try + { + Object[][] result = casFuture.get(30, TimeUnit.SECONDS); + assertCasApplied(result); + } + catch (Exception e) + { + throw new AssertionError("CAS should succeed via COORDINATOR_BEHIND retry but got: " + e.getMessage(), e); + } + + assertReplicasHaveValue(cluster, ks, KEY, 100, 1, 2, 3); + + // Retry verification: forward target sends to 2 other replicas (initial attempt), node 4 + // retries directly on the untracked path sending to all 3 replicas. Total = 5. + assertTrue("Forward target should have been identified", forwardTarget.get() > 0); + assertEquals("Expected initial (2) + retry (3) = 5 PAXOS_COMMIT_REQ at replicas", + 5, commitsAtReplicas.get()); + + // Retry messages from node 4 (untracked path) must not carry mutation IDs. + assertEquals("Retry PAXOS_COMMIT_REQ from node 4 should NOT carry mutation IDs (untracked path)", + 0, retryCommitsWithId.get()); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMutationTrackingMigrationV1Test.java b/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMutationTrackingMigrationV1Test.java new file mode 100644 index 000000000000..3f32f76b63ec --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMutationTrackingMigrationV1Test.java @@ -0,0 +1,399 @@ +/* + * 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.tracking; + +import java.util.concurrent.CompletableFuture; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.shared.ClusterUtils; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.EpochPin; +import org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.MessageSpy; +import org.apache.cassandra.net.Verb; +import org.apache.cassandra.replication.MutationTrackingService; +import org.apache.cassandra.schema.ReplicationType; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationState; +import org.apache.cassandra.tcm.ClusterMetadata; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.alterReplicationType; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.alterReplicationTypeFrom; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertAllNodesSee; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertCasApplied; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertCasException; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertCasNotApplied; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertReplicasHaveValue; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.awaitReplicationType; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.buildPaxosCluster; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.casAsync; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.createKeyspace; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.epochPin; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.on; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.pauseHintsAndReconciler; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Tests that Paxos V1 CAS operations work correctly during mutation tracking migration. + * + * Each test documents the exact message verb being verified and asserts that the + * expected code path was taken (not just that the CAS "succeeds"). + * + * Uses a shared 3-node cluster with paxos_variant=v1. + */ +public class PaxosMutationTrackingMigrationV1Test extends TestBaseImpl +{ + private static Cluster cluster; + + @BeforeClass + public static void setup() throws Throwable + { + cluster = init(buildPaxosCluster(3, "v1").start()); + pauseHintsAndReconciler(cluster); + } + + @AfterClass + public static void teardown() + { + if (cluster != null) + cluster.close(); + } + + @After + public void resetFilters() + { + cluster.filters().reset(); + ClusterUtils.awaitTCMCatchUp(cluster); + } + + /* + * Message: PAXOS_COMMIT_REQ + * Path: V1 -> StorageProxy.commitPaxosTracked + * Verifies: MigrationRouter.shouldUseTrackedForWrites() returns true during migration + */ + @Test + public void testCasDuringMigrationToTracked() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v1", "untracked"); + + cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (1, 0)", + ConsistencyLevel.QUORUM); + + alterReplicationType(cluster, ks, "tracked"); + + // Precondition: migration is active and schema reads tracked on every node. + for (int i = 1; i <= cluster.size(); i++) + { + final String keyspace = ks; + int nodeId = i; + cluster.get(i).runOnInstance(() -> { + ClusterMetadata cm = ClusterMetadata.current(); + MutationTrackingMigrationState state = cm.mutationTrackingMigrationState; + assertTrue("Node " + nodeId + ": keyspace should be migrating", + state.isMigrating(keyspace)); + assertTrue("Node " + nodeId + ": schema should show tracked", + cm.schema.getKeyspaceMetadata(keyspace).params.replicationType.isTracked()); + }); + } + + // Count PAXOS_COMMIT_REQ messages that carry a mutation ID. Counting raw messages is not + // enough -- 2 untracked commits on RF=3 would also produce count==2. The tracked path + // differs from the untracked path by carrying a non-none mutation ID on the Commit payload. + try (MessageSpy spy = on(cluster, Verb.PAXOS_COMMIT_REQ) + .checkMutationId() + .expect(2) + .start()) + { + Object[][] result = cluster.coordinator(1).execute("UPDATE " + ks + ".tbl SET v = 1 WHERE k = 1 IF v = 0", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + spy.await(); + assertCasApplied(result); + assertEquals("Tracked CAS should send exactly 2 PAXOS_COMMIT_REQ carrying a mutation ID (to 2 remote replicas)", + 2, spy.withMutationId()); + } + + for (int i = 1; i <= cluster.size(); i++) + { + int nodeId = i; + cluster.get(i).runOnInstance(() -> { + assertTrue("Node " + nodeId + ": MutationTrackingService should be enabled", + MutationTrackingService.isEnabled()); + }); + } + + assertReplicasHaveValue(cluster, ks, 1, 1, 1, 2, 3); + } + + /* + * Scenario: first CAS commits locally but remote commits are dropped. Migrate to untracked. + * Second CAS discovers the stale ballot and recommits with the ID stripped. + * + * The coordinator strips the ID BEFORE sending, so the handler never sees a mismatch. + * Zero mutation IDs on outbound commits proves stripping worked (no COORDINATOR_BEHIND rejection). + * + * V1 message: PAXOS_COMMIT_REQ via StorageProxy.sendCommit() + */ + @Test + public void testStaleIdStrippedOnRecommit() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v1", "tracked"); + + // Drop remote commits to leave an uncommitted ballot with a mutation ID on node 1. + // The tracked commit path writes locally synchronously, so node 1 has the commit. + cluster.filters().verbs(Verb.PAXOS_COMMIT_REQ.id).drop(); + + boolean threw = false; + try + { + cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (1, 1) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + } + catch (Exception e) + { + threw = true; + assertCasException(e); + } + assertTrue("First CAS should have thrown (commits were blocked)", threw); + + cluster.filters().reset(); + + // Migrate to untracked (instant). The stale ballot in system.paxos has a mutation ID. + alterReplicationType(cluster, ks, "untracked"); + assertAllNodesSee(cluster, ks, ReplicationType.untracked); + + // Spy on PAXOS_COMMIT_REQ messages to remote replicas. The V1 repair path's + // beginAndRepairPaxos loop may run 1+ iterations depending on timing (when a replica + // acknowledges the previous repair before the next prepare). Counting total messages + // is non-deterministic, but the INVARIANT is: every message after migration-to-untracked + // must have mutation_id stripped (.id().isNone() == true). If stripping failed, the + // handler would reject with CoordinatorBehindException -> CAS times out. + try (MessageSpy spy = on(cluster, Verb.PAXOS_COMMIT_REQ) + .to(2, 3) + .checkMutationId() + .start()) + { + // Second CAS: INSERT IF NOT EXISTS. The first CAS's local commit left v=1 on node 1. + // After beginAndRepairPaxos repairs, all replicas have v=1. The condition IF NOT EXISTS + // therefore evaluates FALSE -> CAS NOT applied, result[0][0] == false. + // If stripping failed, CAS would time out (no exception/result here). + Object[][] result = cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (1, 1) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + assertCasNotApplied(result); + + assertReplicasHaveValue(cluster, ks, 1, 1, 1, 2, 3); + + // Core invariant: every repair commit must have its mutation_id stripped after migration + // to untracked. Otherwise handlers reject with CoordinatorBehindException and the CAS + // would time out (it didn't -- see CAS success above). + assertEquals("No PAXOS_COMMIT_REQ should carry a mutation ID after migration to untracked", + 0, spy.withMutationId()); + } + } + + /* + * Scenario: tracked -> untracked migration completes, then a fresh CAS runs. + * No stale ballots, no epoch mismatches. The commit goes through the untracked path + * and all replicas accept on the first try. + * + * Message: PAXOS_COMMIT_REQ + * V1 path: StorageProxy.commitPaxosUntracked() + * + * Exact inbound count == 2 proves no COORDINATOR_BEHIND retry. + */ + @Test + public void testCommitAfterMigrationToUntracked() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v1", "tracked"); + + alterReplicationType(cluster, ks, "untracked"); + assertAllNodesSee(cluster, ks, ReplicationType.untracked); + + try (MessageSpy spy = on(cluster, Verb.PAXOS_COMMIT_REQ) + .to(2, 3) + .checkMutationId() + .expect(2) + .start()) + { + // Fresh CAS on untracked keyspace -- clean commit, no stale data, no race + Object[][] result = cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (1, 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + spy.await(); + + assertCasApplied(result); + + // Exactly 2 PAXOS_COMMIT_REQ (one per remote replica, no retry) + assertEquals("PAXOS_COMMIT_REQ should be sent to exactly 2 remote replicas (no retry)", + 2, spy.total()); + + assertEquals("PAXOS_COMMIT_REQ should NOT carry mutation IDs on untracked keyspace", + 0, spy.withMutationId()); + } + + assertReplicasHaveValue(cluster, ks, 1, 42, 1, 2, 3); + } + + /* + * Scenario: CAS on tracked keyspace. Inbound filter delays PAXOS_COMMIT_REQ delivery + * at nodes 2, 3 while we ALTER to untracked. When released, the handlers see the stale + * epoch and reject with COORDINATOR_BEHIND. The coordinator retries with fresh routing. + * + * Message: PAXOS_COMMIT_REQ + * V1 retry: StorageProxy.commitPaxos() catches CoordinatorBehindException (thrown by + * AbstractWriteResponseHandler.get()), re-evaluates routing, resends. + * + * Inbound count == 4 proves the retry occurred (initial batch 2 + retry batch 2). + * Of those 4, only the initial 2 should carry mutation IDs (tracked era); the retry + * batch (untracked era) should have IDs stripped. + */ + @Test + public void testCommitCoordinatorBehindRetry() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v1", "tracked"); + + // holdFirst(2) blocks the initial batch at the destination until release() is called; + // subsequent messages (the retry batch) pass through immediately. The initial batch + // arrives with old epoch (tracked), but by the time the handler processes it, the + // node is at new epoch (untracked) -> COORDINATOR_BEHIND -> coordinator retries. + try (MessageSpy hold = on(cluster, Verb.PAXOS_COMMIT_REQ) + .from(1) + .to(2, 3) + .holdFirst(2) + .checkMutationId() + .expect(4) + .start()) + { + CompletableFuture casResult = casAsync(cluster, 1, + "INSERT INTO " + ks + ".tbl (k, v) VALUES (1, 42) IF NOT EXISTS"); + + hold.awaitFirstArrival(); + + try + { + // ALTER to untracked while commits are delayed at the destination + alterReplicationType(cluster, ks, "untracked"); + assertAllNodesSee(cluster, ks, ReplicationType.untracked); + } + finally + { + hold.release(); + } + + Object[][] result = casResult.get(60, SECONDS); + hold.await(); + + assertCasApplied(result); + + // Initial batch: 2 inbound (to nodes 2, 3) -> both COORDINATOR_BEHIND -> retry. + // Retry batch: 2 more inbound (to nodes 2, 3 again). + // Total = 4 proves the COORDINATOR_BEHIND retry occurred. + assertEquals("Expected initial (2) + retry (2) = 4 PAXOS_COMMIT_REQ", + 4, hold.total()); + + // Only the initial batch (tracked era) carries mutation IDs. The retry batch + // (untracked era) must have IDs stripped. Total-with-id == 2 proves this: + // if retry messages also had IDs, the count would be 4. + assertEquals("Only the initial stale batch should carry mutation IDs (retry batch should not)", + 2, hold.withMutationId()); + } + + assertReplicasHaveValue(cluster, ks, 1, 42, 1, 2, 3); + assertAllNodesSee(cluster, ks, ReplicationType.untracked); + } + + /* + * Covers: StorageProxy.commitPaxos() -- WriteTimeoutException thrown when the V1 + * commitPaxosTracked response handler times out because maybeFetchLogs blocks before + * delivering the COORDINATOR_BEHIND failure callback. + * + * Node 2 as coordinator is blocked from receiving TCM updates, so it stays at the + * old epoch (tracked). After ALTER to untracked, nodes 1,3 are at the new epoch. + * Every commit from node 2 triggers COORDINATOR_BEHIND on nodes 1,3. Node 2 can't + * catch up, so the first commit attempt's inner timeout fires, failing the CAS without retrying. + */ + @Test + public void testCommitRetryLoopTimeout() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v1", "tracked"); + + // EpochPin strands node 2 at its current (tracked) epoch by holding TCM traffic in/out. + // "Hold" rather than "drop": messages block on a latch with a safety-net timeout so they + // eventually flow after the test, preventing an indefinite fetchLogFromPeerOrCMS stall + // on node 2 that would bleed into later tests. + try (EpochPin pin = epochPin(cluster, 2)) + { + // ALTER to untracked via node 1 at CL.ONE so we don't wait for schema agreement + // on TCM-blocked node 2. + alterReplicationTypeFrom(cluster, 1, ks, "untracked", ConsistencyLevel.ONE); + + // Wait for nodes 1,3 to see untracked. Node 2 should still see tracked. + awaitReplicationType(cluster, ks, ReplicationType.untracked, 1, 3); + final String keyspace = ks; + assertTrue("Node 2 should still see tracked (TCM blocked)", + cluster.get(2).callOnInstance(() -> + ClusterMetadata.current().schema.getKeyspaceMetadata(keyspace) + .params.replicationType.isTracked())); + + // Spy on PAXOS_COMMIT_REQ from node 2 to verify commit attempts were made. + try (MessageSpy spy = on(cluster, Verb.PAXOS_COMMIT_REQ) + .from(2) + .checkMutationId() + .start()) + { + // CAS from node 2. Node 2 thinks tracked -> commitPaxosTracked sends PAXOS_COMMIT_REQ + // to nodes 1,3. They detect epoch mismatch -> COORDINATOR_BEHIND. CBE propagates back; + // maybeFetchLogs on node 2 tries to fetch but the fetches are held by the EpochPin. + // The CAS ultimately fails via the inner response-handler timeout at + // write_request_timeout, before the held fetch is released. + boolean threw = false; + try + { + cluster.coordinator(2).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (1, 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + } + catch (Exception e) + { + threw = true; + assertTrue("Expected CAS timeout but got: " + e.getMessage(), + e.getMessage().contains("CAS operation timed out")); + } + + // Exactly one iteration fires: inner timeout expires before the held TCM fetch + // releases and allows cb.onFailure to let commitPaxos catch the CBE and retry. + assertEquals("Exactly 1 iteration runs (held fetch outlasts write_request_timeout); 2 commits = 1 iteration x 2 replicas", + 2, spy.total()); + + // All commits from stranded node 2 carry mutation IDs (tracked path), + // proving COORDINATOR_BEHIND is the rejection reason (untracked handlers reject tracked commits) + assertEquals("All commit attempts from stranded coordinator should carry mutation IDs (tracked path)", + spy.total(), spy.withMutationId()); + + assertTrue("CAS should have failed", threw); + } + } + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMutationTrackingMigrationV2Test.java b/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMutationTrackingMigrationV2Test.java new file mode 100644 index 000000000000..64ea6b8dc1a1 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/tracking/PaxosMutationTrackingMigrationV2Test.java @@ -0,0 +1,1217 @@ +/* + * 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.tracking; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.db.SinglePartitionReadCommand; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.shared.ClusterUtils; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.EpochPin; +import org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.MessageSpy; +import org.apache.cassandra.hints.HintsService; +import org.apache.cassandra.metrics.StorageMetrics; +import org.apache.cassandra.net.Verb; +import org.apache.cassandra.replication.MutationTrackingService; +import org.apache.cassandra.schema.ReplicationType; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.replication.migration.MigrationRouter; +import org.apache.cassandra.service.replication.migration.MutationTrackingMigrationState; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.utils.FBUtilities; + +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.alterReplicationType; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.alterReplicationTypeFrom; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertAllNodesSee; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertCasApplied; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertCasException; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertCasNotApplied; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertNodeSees; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertReplicaHasNoRow; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertReplicasAreExactly; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.assertReplicasHaveValue; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.awaitReplicationType; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.buildPaxosCluster; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.casAsync; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.casAsyncExpectingFailure; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.createKeyspace; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.epochPin; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.on; +import static org.apache.cassandra.distributed.test.tracking.PaxosMigrationTestUtils.pauseHintsAndReconciler; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Tests that Paxos V2 CAS operations work correctly during mutation tracking migration. + * + * Each test documents the exact message verb being verified and asserts that the + * expected code path was taken (not just that the CAS "succeeds"). + * + * Uses a shared 4-node cluster with paxos_variant=v2 and RF=3. With RF=3 on 4 nodes, + * some keys have node 1 as a replica (2 remote replicas) and some do not (3 remote + * replicas). All tests handle both cases. + */ +public class PaxosMutationTrackingMigrationV2Test extends TestBaseImpl +{ + private static Cluster cluster; + + /* + * With Murmur3Partitioner, 4 nodes, and SimpleStrategy RF=3: + * Key 5 → replicas on nodes 1, 2, 3 (node 4 excluded) + * This key is used by most tests because node 1 is a replica (avoids CAS forwarding) + * and node 4 is not a replica (useful for testCommitAndPrepareViaIncompleteAccepted). + */ + private static final int KEY = 5; + + @BeforeClass + public static void setup() throws Throwable + { + cluster = init(buildPaxosCluster(4, "v2").start()); + pauseHintsAndReconciler(cluster); + } + + @AfterClass + public static void teardown() + { + if (cluster != null) + cluster.close(); + } + + @After + public void resetFilters() + { + cluster.filters().reset(); + cluster.forEach(instance -> instance.runOnInstance(() -> HintsService.setRejectHintsBeforeNanos(0))); + ClusterUtils.awaitTCMCatchUp(cluster); + } + + /* + * CAS during active to-tracked migration + * + * Message: PAXOS_COMMIT_REQ + * Path: V2 -> PaxosCommit.start() (tracked path) + * Verifies: MigrationRouter.shouldUseTrackedForWrites() returns true during migration + */ + @Test + public void testCasDuringMigrationToTracked() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "untracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 0)", + ConsistencyLevel.QUORUM); + + alterReplicationType(cluster, ks, "tracked"); + + for (int i = 1; i <= cluster.size(); i++) + { + final String keyspace = ks; + int nodeId = i; + cluster.get(i).runOnInstance(() -> { + ClusterMetadata cm = ClusterMetadata.current(); + MutationTrackingMigrationState state = cm.mutationTrackingMigrationState; + assertTrue("Node " + nodeId + ": keyspace should be migrating", + state.isMigrating(keyspace)); + assertTrue("Node " + nodeId + ": schema should show tracked", + cm.schema.getKeyspaceMetadata(keyspace).params.replicationType.isTracked()); + }); + } + + // Count PAXOS_COMMIT_REQ messages that carry a mutation ID. Counting raw messages is not + // enough — 2 untracked commits on RF=3 would also produce count==2. The tracked path + // differs from the untracked path by carrying a non-none mutation ID on the Commit payload. + try (MessageSpy spy = on(cluster, Verb.PAXOS_COMMIT_REQ) + .checkMutationId() + .expect(2) + .start()) + { + Object[][] result = cluster.coordinator(1).execute("UPDATE " + ks + ".tbl SET v = 1 WHERE k = " + KEY + " IF v = 0", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + spy.await(); + assertCasApplied(result); + assertEquals("Tracked CAS should send exactly 2 PAXOS_COMMIT_REQ carrying a mutation ID (to 2 remote replicas)", + 2, spy.withMutationId()); + } + + for (int i = 1; i <= cluster.size(); i++) + { + int nodeId = i; + cluster.get(i).runOnInstance(() -> { + assertTrue("Node " + nodeId + ": MutationTrackingService should be enabled", + MutationTrackingService.isEnabled()); + }); + } + + assertReplicasHaveValue(cluster, ks, KEY, 1, 1, 2, 3); + } + + /* + * In V2 Paxos, the read is embedded in the PAXOS2_PREPARE_REQ payload. Tracked reads use + * TrackedRead.DataRequest/SummaryRequest; untracked uses SinglePartitionReadCommand. + * The handler validates via checkPaxosPrepareReadMigration(). Since both types use the + * same verb (PAXOS2_PREPARE_REQ), we verify routing by checking MigrationRouter.shouldUseTracked() + * on the coordinator, and confirm handler acceptance by checking no COORDINATOR_BEHIND occurred + * (proven by exact PAXOS_COMMIT_REQ counts -- retries would inflate the count). + * + * 2a: During to-tracked migration, reads should be UNTRACKED (safe default). + * 2b: After migration to untracked, reads should be UNTRACKED. + * 2c: On a fully-tracked keyspace (no migration), reads should be TRACKED. + */ + + /** + * 2a: During to-tracked migration, the Paxos prepare read uses untracked routing. + * + * MigrationRouter.shouldUseTracked() returns false during migration because tracked + * reads require ALL writes to be tracked for monotonicity. + */ + @Test + public void testPrepareReadDuringMigrationIsUntracked() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "untracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42)", + ConsistencyLevel.QUORUM); + + alterReplicationType(cluster, ks, "tracked"); + + // Verify migration is active and the prepare read routing is UNTRACKED + for (int i = 1; i <= cluster.size(); i++) + { + final String keyspace = ks; + int nodeId = i; + cluster.get(i).runOnInstance(() -> { + assertTrue("Node " + nodeId + ": migration should be active", + ClusterMetadata.current().mutationTrackingMigrationState.isMigrating(keyspace)); + + TableMetadata tbl = ClusterMetadata.current().schema.getKeyspaceMetadata(keyspace).getTableOrViewNullable("tbl"); + SinglePartitionReadCommand cmd = SinglePartitionReadCommand.fullPartitionRead(tbl, FBUtilities.nowInSeconds(), tbl.partitioner.decorateKey(tbl.partitionKeyType.fromString(String.valueOf(KEY)))); + assertFalse("Node " + nodeId + ": prepare read should be UNTRACKED during migration", + MigrationRouter.shouldUseTracked(cmd)); + }); + } + + // CAS works with untracked read (correct routing during migration) + Object[][] result = cluster.coordinator(1).execute("SELECT * FROM " + ks + ".tbl WHERE k = " + KEY, + ConsistencyLevel.SERIAL); + + assertNotNull("CAS read should return result", result); + assertEquals("Should have one row", 1, result.length); + assertEquals("Value should be 42", 42, result[0][1]); + } + + /** + * 2b: After migration to untracked, the Paxos prepare read uses untracked routing. + */ + @Test + public void testPrepareReadAfterMigrationToUntrackedIsUntracked() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "tracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + alterReplicationType(cluster, ks, "untracked"); + + // Verify all nodes see untracked and the prepare read routing is UNTRACKED + for (int i = 1; i <= cluster.size(); i++) + { + final String keyspace = ks; + int nodeId = i; + cluster.get(i).runOnInstance(() -> { + assertFalse("Node " + nodeId + ": should be untracked", + ClusterMetadata.current().schema.getKeyspaceMetadata(keyspace).params.replicationType.isTracked()); + + TableMetadata tbl = ClusterMetadata.current().schema.getKeyspaceMetadata(keyspace).getTableOrViewNullable("tbl"); + SinglePartitionReadCommand cmd = SinglePartitionReadCommand.fullPartitionRead(tbl, FBUtilities.nowInSeconds(), tbl.partitioner.decorateKey(tbl.partitionKeyType.fromString(String.valueOf(KEY)))); + assertFalse("Node " + nodeId + ": prepare read should be UNTRACKED after migration", + MigrationRouter.shouldUseTracked(cmd)); + }); + } + + // Spy on PAXOS_COMMIT_REQ to verify no COORDINATOR_BEHIND retry. + try (MessageSpy spy = on(cluster, Verb.PAXOS_COMMIT_REQ) + .to(2, 3, 4) + .expect(2) + .start()) + { + Object[][] result = cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + spy.await(); + assertCasApplied(result); + assertEquals("PAXOS_COMMIT_REQ should match remote replica count (no retry)", + 2, spy.total()); + } + + assertReplicasHaveValue(cluster, ks, KEY, 42, 1, 2, 3); + } + + /** + * 2c: On a fully-tracked keyspace (created as tracked, no migration), the Paxos + * prepare read uses tracked routing. + */ + @Test + public void testPrepareReadOnTrackedKeyspaceIsTracked() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "tracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + // Verify the prepare read routing is TRACKED on all nodes + for (int i = 1; i <= cluster.size(); i++) + { + final String keyspace = ks; + int nodeId = i; + cluster.get(i).runOnInstance(() -> { + assertTrue("Node " + nodeId + ": should be tracked", + ClusterMetadata.current().schema.getKeyspaceMetadata(keyspace).params.replicationType.isTracked()); + + TableMetadata tbl = ClusterMetadata.current().schema.getKeyspaceMetadata(keyspace).getTableOrViewNullable("tbl"); + SinglePartitionReadCommand cmd = SinglePartitionReadCommand.fullPartitionRead(tbl, FBUtilities.nowInSeconds(), tbl.partitioner.decorateKey(tbl.partitionKeyType.fromString(String.valueOf(KEY)))); + assertTrue("Node " + nodeId + ": prepare read should be TRACKED on tracked keyspace", + MigrationRouter.shouldUseTracked(cmd)); + }); + } + + // Spy on PAXOS_COMMIT_REQ to verify no COORDINATOR_BEHIND retry from read mismatch. + try (MessageSpy spy = on(cluster, Verb.PAXOS_COMMIT_REQ) + .to(2, 3, 4) + .expect(2) + .start()) + { + Object[][] result = cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + spy.await(); + assertCasApplied(result); + assertEquals("PAXOS_COMMIT_REQ should match remote replica count (no retry)", + 2, spy.total()); + } + + assertReplicasHaveValue(cluster, ks, KEY, 42, 1, 2, 3); + } + + /* + * Scenario: first CAS commits locally but remote commits are dropped. Migrate to untracked. + * Second CAS discovers the stale ballot and recommits with the ID stripped. + * + * The coordinator strips the ID BEFORE sending, so the handler never sees a mismatch. + * Exact inbound message count proves no COORDINATOR_BEHIND retry occurred. + * + * V2 message: PAXOS2_PREPARE_REFRESH_REQ via PaxosPrepareRefresh.refresh() + */ + @Test + public void testStaleIdStrippedOnRecommit() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "tracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + // Drop remote commits to leave an uncommitted ballot with a mutation ID on node 1. + // The tracked commit path writes locally synchronously, so node 1 has the commit. + cluster.filters().verbs(Verb.PAXOS_COMMIT_REQ.id).drop(); + + try + { + cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 1) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + fail("CAS should have thrown because remote commits were dropped"); + } + catch (Exception e) + { + assertCasException(e); + } + + cluster.filters().reset(); + + // Migrate to untracked (instant). The stale ballot in system.paxos has a mutation ID. + alterReplicationType(cluster, ks, "untracked"); + assertNodeSees(cluster, 1, ks, ReplicationType.untracked); + + // Spy on the recommit verb AND on PAXOS_COMMIT_REQ to detect any retry overhead. + // V2 recommit: PaxosPrepareRefresh -> PAXOS2_PREPARE_REFRESH_REQ to remote replicas. + try (MessageSpy recommitSpy = on(cluster, Verb.PAXOS2_PREPARE_REFRESH_REQ) + .to(2, 3, 4) + .checkMutationId() + .expect(2) + .start(); + // Also count PAXOS_COMMIT_REQ separately to verify no unexpected commits. + // (The second CAS's condition is not met so no new commit should be sent.) + MessageSpy commitSpy = on(cluster, Verb.PAXOS_COMMIT_REQ) + .to(2, 3, 4) + .start()) + { + // Second CAS discovers the stale ballot and recommits with the ID stripped. + // If the ID is NOT stripped, the handler rejects (tracked mutation in untracked + // keyspace) and the CAS fails. + Object[][] result = cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 1) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + recommitSpy.await(); + + assertCasNotApplied(result); + + // Assert: the recommit verb was sent to the remote replicas (2). + assertEquals("PAXOS2_PREPARE_REFRESH_REQ should match remote replica count", + 2, recommitSpy.total()); + + // After migration to untracked, refresh messages must NOT carry mutation IDs + assertEquals("PAXOS2_PREPARE_REFRESH_REQ should NOT carry mutation IDs after migration to untracked", + 0, recommitSpy.withMutationId()); + + // No PAXOS_COMMIT_REQ should be sent (condition not met -> no new commit) + assertEquals("No PAXOS_COMMIT_REQ expected for V2 (condition not met)", 0, commitSpy.total()); + } + + assertReplicasHaveValue(cluster, ks, KEY, 1, 1, 2, 3); + } + + /* + * Scenario: tracked -> untracked migration completes, then a fresh CAS runs. + * No stale ballots, no epoch mismatches. The commit goes through the untracked path + * and all replicas accept on the first try. + * + * Message: PAXOS_COMMIT_REQ + * V2 path: PaxosCommit.start() (untracked, no mutation ID) + * + * Inbound count == 2 proves no COORDINATOR_BEHIND retry (one batch to 2 remote replicas). + */ + @Test + public void testCommitAfterMigrationToUntracked() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "tracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + alterReplicationType(cluster, ks, "untracked"); + assertAllNodesSee(cluster, ks, ReplicationType.untracked); + + // Count inbound PAXOS_COMMIT_REQ at remote nodes. + // Also count PAXOS2_PREPARE_REFRESH_REQ to verify no stale ballot refresh. + try (MessageSpy commitSpy = on(cluster, Verb.PAXOS_COMMIT_REQ) + .to(2, 3, 4) + .expect(2) + .start(); + MessageSpy refreshSpy = on(cluster, Verb.PAXOS2_PREPARE_REFRESH_REQ) + .to(2, 3, 4) + .start()) + { + // Fresh CAS on untracked keyspace -- clean commit, no stale data, no race + Object[][] result = cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + commitSpy.await(); + + assertCasApplied(result); + + // Assert: PAXOS_COMMIT_REQ sent to remote replicas (2), no retry + assertEquals("PAXOS_COMMIT_REQ should match remote replica count (no retry)", + 2, commitSpy.total()); + + // Assert: no PAXOS2_PREPARE_REFRESH_REQ (no stale ballot to refresh) + assertEquals("No PAXOS2_PREPARE_REFRESH_REQ expected (no stale data)", + 0, refreshSpy.total()); + } + + assertReplicasHaveValue(cluster, ks, KEY, 42, 1, 2, 3); + } + + /* + * Scenario: CAS on tracked keyspace. Inbound filter delays PAXOS_COMMIT_REQ delivery + * at nodes 2, 3, 4 while we ALTER to untracked. When released, the handlers see the stale + * epoch and reject with COORDINATOR_BEHIND. The coordinator retries with fresh routing. + * + * Message: PAXOS_COMMIT_REQ + * V2 retry: Paxos.cas() checks failure.failures map for COORDINATOR_BEHIND, + * creates new PaxosCommit which re-evaluates MigrationRouter, resends. + * + * Inbound count == 4 proves the retry occurred (initial batch 2 + retry batch 2). + */ + @Test + public void testCommitCoordinatorBehindRetry() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "tracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + // Inbound filter at nodes 2, 3, 4: hold ALL PAXOS_COMMIT_REQ arrivals + // until after the schema change. The first batch arrives with old epoch (tracked), + // but by the time the handler processes it, the node is at new epoch (untracked). + // checkPaxosCommitMigration detects the mismatch -> COORDINATOR_BEHIND. + try (MessageSpy hold = on(cluster, Verb.PAXOS_COMMIT_REQ) + .from(1) + .to(2, 3, 4) + .holdAll() + .checkMutationId() + .expect(4) + .start()) + { + CompletableFuture casResult = casAsync(cluster, 1, + "INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS"); + + hold.awaitFirstArrival(); + + try + { + // ALTER to untracked while commits are delayed at the destination + alterReplicationType(cluster, ks, "untracked"); + assertAllNodesSee(cluster, ks, ReplicationType.untracked); + } + finally + { + hold.release(); + } + + Object[][] result = casResult.get(60, TimeUnit.SECONDS); + hold.await(); + + assertCasApplied(result); + + // Initial batch: 2 inbound (to remote replicas) -> COORDINATOR_BEHIND -> retry. + // Retry batch: 2 more inbound. + // Total = 4 proves the COORDINATOR_BEHIND retry occurred. + assertEquals("Expected initial (2) + retry (2) = 4 PAXOS_COMMIT_REQ", + 4, hold.total()); + + // Initial batch (2) carries mutation IDs (tracked era), retry batch (2) does not. + // Total with ID == 2 proves the retry used the untracked path (no mutation IDs). + assertEquals("Only the initial stale batch should carry mutation IDs (retry batch should not)", + 2, hold.withMutationId()); + } + + assertReplicasHaveValue(cluster, ks, KEY, 42, 1, 2, 3); + assertAllNodesSee(cluster, ks, ReplicationType.untracked); + } + + /* + * Scenario: first CAS commits on untracked keyspace (no mutation ID in system.paxos), + * then migrate to tracked. Second CAS discovers the stale ballot without a mutation ID. + * PaxosPrepareRefresh must generate a mutation ID for the now-tracked keyspace. + * + * Message: PAXOS2_PREPARE_REFRESH_REQ + * Path: PaxosPrepareRefresh.refresh() -> tracked=true, commit.mutation.id().isNone()=true + * -> generates mutation ID locally (calls generateMutationIdAndPersistLocally()) + */ + @Test + public void testPrepareRefreshGeneratesMutationId() throws Throwable + { + // Create as UNTRACKED -- commits will have no mutation ID + String ks = createKeyspace(cluster, "pmt_v2", "untracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + // Block remote commits to create an uncommitted ballot WITHOUT a mutation ID + cluster.filters().verbs(Verb.PAXOS_COMMIT_REQ.id).drop(); + + try + { + cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 1) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + fail("CAS should have thrown because remote commits were dropped"); + } + catch (Exception e) + { + assertCasException(e); + } + + cluster.filters().reset(); + + // Migrate to TRACKED. The stale ballot in system.paxos has NO mutation ID. + alterReplicationType(cluster, ks, "tracked"); + assertAllNodesSee(cluster, ks, ReplicationType.tracked); + + // Spy on PAXOS2_PREPARE_REFRESH_REQ to verify the refresh path fires. + try (MessageSpy refreshSpy = on(cluster, Verb.PAXOS2_PREPARE_REFRESH_REQ) + .to(2, 3, 4) + .checkMutationId() + .expect(2) + .start()) + { + // Second CAS discovers the stale ballot. PaxosPrepareRefresh.refresh() runs: + // tracked=true, commit.mutation.id().isNone()=true -> generates ID locally + Object[][] result = cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 1) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + refreshSpy.await(); + + // First CAS (untracked era) only committed locally on node 1. Refresh propagates that + // commit to all replicas (with a mutation ID generated during migration). After refresh, + // node 1 has v=1 and the condition IF NOT EXISTS evaluates FALSE → second CAS not applied. + assertCasNotApplied(result); + + // Refresh should have been sent to the remote replicas (2) + assertEquals("PAXOS2_PREPARE_REFRESH_REQ should match remote replica count", + 2, refreshSpy.total()); + + // All refresh messages should carry a mutation ID (generated for the now-tracked keyspace) + assertEquals("All PAXOS2_PREPARE_REFRESH_REQ should carry a mutation ID (tracked keyspace)", + refreshSpy.total(), refreshSpy.withMutationId()); + } + + assertReplicasHaveValue(cluster, ks, KEY, 1, 1, 2, 3); + } + + /* + * Scenario: CAS on tracked keyspace. Inbound filter delays PAXOS2_PREPARE_REQ delivery + * at nodes 2, 3, 4 while we ALTER to untracked. When released, the handler's + * checkPaxosPrepareReadMigration detects the mismatch -> COORDINATOR_BEHIND. + * Unlike the commit path, the prepare path has no internal COORDINATOR_BEHIND retry; + * the CAS fails. The coordinator catches up via the response epoch, and a client-level + * retry succeeds. + * + * Covers: MigrationRouter.checkPaxosPrepareReadMigration() coordinator-behind branch + */ + @Test + public void testPrepareReadCoordinatorBehind() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "tracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + // Delay PAXOS2_PREPARE_REQ at nodes 2, 3, 4 until after migration. + // The prepare arrives with old epoch (tracked read routing). + // After migration, handler is at new epoch (untracked). + // checkPaxosPrepareReadMigration detects mismatch -> COORDINATOR_BEHIND. + try (MessageSpy hold = on(cluster, Verb.PAXOS2_PREPARE_REQ) + .from(1) + .to(2, 3, 4) + .holdAll() + .start()) + { + CompletableFuture casResult = casAsyncExpectingFailure(cluster, 1, + "INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS"); + + hold.awaitFirstArrival(); + + try + { + // ALTER to untracked while prepares are delayed + alterReplicationType(cluster, ks, "untracked"); + assertAllNodesSee(cluster, ks, ReplicationType.untracked); + } + finally + { + hold.release(); + } + + // First CAS should fail (COORDINATOR_BEHIND from prepare, no internal retry) + // Expected: COORDINATOR_BEHIND from prepare causes WriteFailureException. + // The prepare path does not internally retry COORDINATOR_BEHIND (unlike commit). + Throwable error = casResult.get(60, TimeUnit.SECONDS); + assertNotNull("First CAS should fail with COORDINATOR_BEHIND", error); + assertCasException((Exception) error); + + // Exactly one prepare round was attempted (2 remote replicas for KEY=5 with RF=3 from node 1) + assertEquals("Exactly one prepare round should be attempted (2 messages to remote replicas)", + 2, hold.total()); + } + + cluster.filters().reset(); + + // The coordinator has caught up by now (ResponseVerbHandler.maybeFetchLogs triggered + // by the COORDINATOR_BEHIND response). A client-level retry succeeds. + Object[][] result = cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + assertCasApplied(result); + + assertReplicasHaveValue(cluster, ks, KEY, 42, 1, 2, 3); + } + + /* + * Covers: Paxos.cas() -- the throw inside the COORDINATOR_BEHIND retry while-loop + * when the failure is NOT COORDINATOR_BEHIND (coordinatorBehindCount == 0). + * + * The V2 commit while-loop was added to retry on COORDINATOR_BEHIND. For all other + * failures, it throws immediately. This test blocks commit messages to trigger a timeout. + */ + @Test + public void testCommitFailureThrowsFromRetryLoop() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "tracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + // Block commit messages to remotes and respond with TIMEOUT immediately so the + // callback fires without waiting for the real write_request_timeout. + // Local commit succeeds (1/3) but quorum (2) is not met -> Paxos.cas() throws. + cluster.filters() + .inbound(true) + .verbs(Verb.PAXOS_COMMIT_REQ.id) + .from(1).to(2, 3, 4) + .messagesMatching((from, to, msg) -> { + cluster.get(to).runOnInstance(() -> PaxosMigrationTestUtils.respondWithTimeout(msg)); + return true; + }).drop(); + + boolean threw = false; + try + { + cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + } + catch (Exception e) + { + threw = true; + assertCasException(e); + } + + assertTrue("CAS should have failed due to commit timeout", threw); + } + + /* + * First CAS from node 2 with commits blocked to nodes 1,3 -> only node 2 commits. + * Second CAS from node 1: coordinator = data node for TrackedRead. Node 2 (summary, + * readResponse=null) has the committed ballot. Node 3 is also a summary node without + * the committed ballot. + * + * Key subtlety: PaxosPrepare signals outcome as soon as quorum is reached. If self + + * node 3 reach quorum before node 2's response, latestCommitted is still 'none' and + * FOUND_INCOMPLETE_ACCEPTED fires instead. To ensure FOUND_INCOMPLETE_COMMITTED, we + * delay node 3's prepare so node 2 responds first, making latestCommitted known before + * quorum is reached. Then hasInProgressProposal() returns false (same ballot), and the + * check falls through to FOUND_INCOMPLETE_COMMITTED. + */ + @Test + public void testCommitAndPrepareViaIncompleteCommitted() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "tracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + // Both node 1 and node 2 are replicas for KEY (5): node 2 coordinates the + // first CAS (local commit), and node 1 coordinates the second CAS (TrackedRead data node). + + // First CAS from NODE 2. Block commits to nodes 1 and 3. + // Node 2 commits locally (executeOnSelf), nodes 1,3 only have ACCEPTED state. + cluster.filters().verbs(Verb.PAXOS_COMMIT_REQ.id).from(2).to(1, 3).drop(); + + try + { + cluster.coordinator(2).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + fail("CAS should have thrown because commits to nodes 1,3 were blocked"); + } + catch (Exception e) + { + assertCasException(e); + } + + cluster.filters().reset(); + + // Ensure node 2's prepare response arrives at node 1 before node 3's. + // This guarantees FOUND_INCOMPLETE_COMMITTED (not FOUND_INCOMPLETE_ACCEPTED). + AssertingLatch node2Responded = new AssertingLatch("testCommitAndPrepareViaIncompleteCommitted node2Responded"); + cluster.filters() + .inbound(true) + .verbs(Verb.PAXOS2_PREPARE_RSP.id) + .from(2) + .to(1) + .messagesMatching((from, to, msg) -> { + node2Responded.countDown(); + return false; + }).drop(); + + cluster.filters() + .inbound(true) + .verbs(Verb.PAXOS2_PREPARE_RSP.id) + .from(3) + .to(1) + .messagesMatching((from, to, msg) -> { + node2Responded.await(); + return false; + }).drop(); + + // Spy on verbs to prove the TRACKED commitAndPrepare path: + // - PAXOS_COMMIT_REQ: the tracked path sends separate commit (PaxosCommit.commit()) + // - PAXOS2_COMMIT_AND_PREPARE_REQ: the UNTRACKED combined path — should NOT appear + // - PAXOS2_PREPARE_REQ: separate prepare after commit (prepareWithBallot) + try (MessageSpy commitSpy = on(cluster, Verb.PAXOS_COMMIT_REQ) + .from(1) + .checkMutationId() + .expect(4) + .start(); + MessageSpy combinedSpy = on(cluster, Verb.PAXOS2_COMMIT_AND_PREPARE_REQ) + .from(1) + .start(); + MessageSpy prepareSpy = on(cluster, Verb.PAXOS2_PREPARE_REQ) + .from(1) + .expect(4) + .start()) + { + // Second CAS from NODE 1 (coordinator = data node for TrackedRead). + // Response order: self (node 1, no commit) + node 2 (summary, has commit) = quorum. + // latestCommitted known, hasInProgressProposal() = false (same ballot). + // withLatest = {node2} (1) < quorum (2), haveReadResponseWithLatest = false. + // -> FOUND_INCOMPLETE_COMMITTED -> commitAndPrepare() + Object[][] result = cluster.coordinator(1).execute("UPDATE " + ks + ".tbl SET v2 = 99 WHERE k = " + KEY + " IF EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + commitSpy.await(); + prepareSpy.await(); + + assertNotNull("CAS should return result", result); + assertEquals(1, result.length); + assertTrue("Second CAS (UPDATE IF EXISTS) should apply — row was repaired to v=42", + (boolean) result[0][0]); + + // Verify TRACKED commitAndPrepare path was taken: + // 1. Separate PAXOS_COMMIT_REQ sent: 2 from repair commit + 2 from CAS's own commit = 4 + assertEquals("Tracked path should send 4 PAXOS_COMMIT_REQ (2 repair + 2 CAS commit)", + 4, commitSpy.total()); + // 2. All commits carry a mutation ID (tracked keyspace) + assertEquals("All PAXOS_COMMIT_REQ should carry a mutation ID", + commitSpy.total(), commitSpy.withMutationId()); + // 3. Combined PAXOS2_COMMIT_AND_PREPARE_REQ NOT sent (that's the untracked path) + assertEquals("Combined commit+prepare should NOT be used for tracked keyspace", + 0, combinedSpy.total()); + // 4. PAXOS2_PREPARE_REQ messages: initial CAS prepare (2) + commitAndPrepare's prepareWithBallot (2) = 4 + assertEquals("Tracked path: initial prepare (2) + post-repair prepareWithBallot (2) = 4", + 4, prepareSpy.total()); + } + + // Verify BOTH the old commit (v=42) and new commit (v2=99) applied on all REPLICAS. + // KEY=5 maps to replicas [1,2,3] — node 4 is not a replica and holds no local data. + for (int i : new int[]{ 1, 2, 3 }) + { + Object[][] nodeResult = cluster.get(i).executeInternal("SELECT v, v2 FROM " + ks + ".tbl WHERE k = " + KEY); + assertEquals("Node " + i + " should have the committed row", 1, nodeResult.length); + assertEquals("Node " + i + " should have v=42 from first commit", 42, nodeResult[0][0]); + assertEquals("Node " + i + " should have v2=99 from second commit", 99, nodeResult[0][1]); + } + } + + /* + * Triggers the V2 commitAndPrepare() path by creating an ACCEPTED-but-not-COMMITTED + * state on all replicas. Node 4 (not a replica for the key) is the CAS coordinator. + * In V2 PaxosCommit.start(), executeOnSelf() skips the local commit (not in range). + * Blocking PAXOS_COMMIT_REQ prevents remote commits. All 3 replicas end up in ACCEPTED + * only. A second CAS discovers FOUND_INCOMPLETE_ACCEPTED -> re-propose -> commitAndPrepare(). + */ + @Test + public void testCommitAndPrepareViaIncompleteAccepted() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "untracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + cluster.filters().verbs(Verb.PAXOS_COMMIT_REQ.id).drop(); + + try + { + cluster.coordinator(4).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 1) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + fail("CAS should have thrown because all commits were dropped"); + } + catch (Exception e) + { + assertCasException(e); + } + + cluster.filters().reset(); + + try (MessageSpy spy = on(cluster, Verb.PAXOS2_COMMIT_AND_PREPARE_REQ) + .start()) + { + Object[][] result = cluster.coordinator(4).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 1) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + assertCasNotApplied(result); + + // Coordinator is node 4 (NOT a replica for KEY=5; replicas are [1,2,3]). + // Non-replica coordinator sends commitAndPrepare to ALL 3 replicas (no local execute). + assertEquals("commitAndPrepare should send PAXOS2_COMMIT_AND_PREPARE_REQ to all 3 replicas (coordinator is non-replica)", + 3, spy.total()); + } + + assertReplicasHaveValue(cluster, ks, KEY, 1, 1, 2, 3); + } + + /* + * Tests the handler-side PaxosPrepare.RequestHandler.doVerb() behavior: it calls + * checkPaxosPrepareReadMigration() BEFORE execute() runs, so a coordinator that hasn't + * yet enacted the ALTER is rejected at the prepare phase — commitAndPrepare is never + * reached. The CAS fails via timeout at cas_contention_timeout (the EpochPin blocks + * ResponseVerbHandler.maybeFetchLogs, preventing the failure callback from completing). + * + * NOTE: Despite the name's historical origin, this test does NOT exercise + * PaxosCommitAndPrepare.RequestHandler — the prepare-level rejection fires first. + * A separate test (testCommitAndPrepareHandlerRejectsAfterMigration) exercises the + * commitAndPrepare handler by engineering a scenario where prepare succeeds first. + */ + @Test + public void testPrepareRejectsAfterMigration() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "untracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + // Create incomplete ACCEPTED state: CAS from node 4, block commits globally. + // Node 4 is NOT a replica (KEY=5, RF=3 → replicas 1,2,3), so all 3 replicas + // only reach ACCEPTED state (commit fails to reach them). + cluster.filters().verbs(Verb.PAXOS_COMMIT_REQ.id).drop(); + + try + { + cluster.coordinator(4).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + fail("CAS should have thrown because all commits were dropped"); + } + catch (Exception e) + { + assertCasException(e); + } + + cluster.filters().reset(); + + try (EpochPin pin = epochPin(cluster, 2)) + { + // ALTER to tracked from node 1 (CMS). Use CL.ONE to avoid waiting for node 2's agreement. + alterReplicationTypeFrom(cluster, 1, ks, "tracked", ConsistencyLevel.ONE); + + // Wait for node 3 to see tracked. Node 2 should still see untracked. + awaitReplicationType(cluster, ks, ReplicationType.tracked, 3); + assertNodeSees(cluster, 2, ks, ReplicationType.untracked); + + // CAS from node 2. Node 2 sees untracked; handlers on nodes 1,3 see tracked. + // PaxosPrepare.RequestHandler.doVerb() calls checkPaxosPrepareReadMigration() BEFORE + // execute(), which detects the disagreement and throws CoordinatorBehindException. + // The FAILURE_RSP propagates back to node 2 — its ResponseVerbHandler triggers + // maybeFetchLogs which attempts a TCM fetch. The fetch is held by the EpochPin; + // the CAS ultimately fails via the inner response-handler timeout at cas_contention_timeout. + boolean threw = false; + try + { + cluster.coordinator(2).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + } + catch (Exception e) + { + threw = true; + assertCasException(e); + } + + assertTrue("CAS should have failed due to handler COORDINATOR_BEHIND", threw); + + // Verify data state: the first CAS's incomplete ACCEPTED ballot (v=42) was NEVER + // successfully committed on any replica. The handler rejected the new CAS's commit, + // and the prior round never reached a committed state either. + assertReplicaHasNoRow(cluster, ks, KEY, 1); + } + } + + /* + * Exercises the commitAndPrepare handler's checkPaxosCommitMigration disagree case + * specifically. Unlike testPrepareRejectsAfterMigration (where prepare fails first), + * this test engineers a scenario where: + * 1. Prepare succeeds — all nodes agree on current migration state at prepare time + * 2. commitAndPrepare is triggered (via FOUND_INCOMPLETE_ACCEPTED) + * 3. PAXOS2_COMMIT_AND_PREPARE_REQ is held in flight via message filter + * 4. While held: ALTER to tracked fires on nodes 1,3 (coordinator node 2 has TCM blocked + * so stays untracked) + * 5. Held messages are released → handlers see tracked, coordinator said untracked → + * checkPaxosCommitMigration throws CoordinatorBehindException + * 6. CAS fails with CAS exception + */ + @Test + public void testCommitAndPrepareHandlerRejectsAfterMigration() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "untracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + // Create incomplete ACCEPTED state. CAS from node 4 (non-replica), block commits. + cluster.filters().verbs(Verb.PAXOS_COMMIT_REQ.id).drop(); + + try + { + cluster.coordinator(4).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + fail("CAS should have thrown because all commits were dropped"); + } + catch (Exception e) + { + assertCasException(e); + } + + cluster.filters().reset(); + + // Hold inbound PAXOS2_COMMIT_AND_PREPARE_REQ at nodes 1,3 until released. Node 2 is the + // coordinator (NOT one of the handlers) — messages flow OUT of node 2 TO nodes 1,3. + try (MessageSpy hold = on(cluster, Verb.PAXOS2_COMMIT_AND_PREPARE_REQ) + .from(2) + .to(1, 3) + .holdAll() + .expect(2) + .start()) + { + // Start the second CAS from node 2 asynchronously. All nodes currently see untracked, + // so prepare succeeds. FOUND_INCOMPLETE_ACCEPTED triggers commitAndPrepare. + CompletableFuture casResult = casAsyncExpectingFailure(cluster, 2, + "INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS"); + + EpochPin pin = null; + try + { + hold.awaitFirstArrival(); + + // Pin node 2 at its current epoch so it stays untracked while we ALTER via node 1. + pin = epochPin(cluster, 2); + + // ALTER to tracked from node 1 (CMS). CL.ONE so we don't wait for node 2. + alterReplicationTypeFrom(cluster, 1, ks, "tracked", ConsistencyLevel.ONE); + + // Wait for node 3 to see tracked. Node 2 stays untracked. + awaitReplicationType(cluster, ks, ReplicationType.tracked, 3); + assertNodeSees(cluster, 2, ks, ReplicationType.untracked); + } + finally + { + hold.release(); + } + + try + { + // Held messages are now released. Handlers see tracked; coordinator's message carries + // untracked-era payload (no mutation ID, coordinatorSaysTracked=false). The handler's + // checkPaxosCommitMigration detects the disagreement → CoordinatorBehindException. + Throwable error = casResult.get(30, TimeUnit.SECONDS); + assertNotNull("Second CAS must fail because commitAndPrepare handler rejects", error); + assertCasException((Exception) error); + + // Verify the commitAndPrepare handler was actually invoked (otherwise we didn't test it). + assertEquals("PAXOS2_COMMIT_AND_PREPARE_REQ must arrive at both tracked handlers", + 2, hold.total()); + + // Verify data state: no replica should have a committed v=42 row. + assertReplicaHasNoRow(cluster, ks, KEY, 1); + } + finally + { + if (pin != null) + pin.close(); + } + } + } + + /* + * Covers: PaxosCommitAndPrepare.java + * commit = commit.withMutationId(MutationId.none()); + * + * First CAS while tracked stores committed ballot WITH mutation ID in system.paxos. + * Migrate to untracked. Second CAS triggers FOUND_INCOMPLETE_COMMITTED which calls + * commitAndPrepare(). The loaded commit has an ID but shouldBeTracked is now false, + * so the code strips the ID before committing via the untracked path. + */ + @Test + public void testCommitAndPrepareStripsIdAfterMigrationToUntracked() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "tracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + // First CAS from node 2. Block commits to nodes 1,3,4 so only node 2 commits. + cluster.filters().verbs(Verb.PAXOS_COMMIT_REQ.id).from(2).to(1, 3, 4).drop(); + + try + { + cluster.coordinator(2).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + fail("CAS should have thrown because commits to nodes 1,3,4 were blocked"); + } + catch (Exception e) + { + assertCasException(e); + } + + cluster.filters().reset(); + + // Migrate to untracked. The committed ballot in system.paxos on node 2 still has + // its mutation ID from the tracked era. + alterReplicationType(cluster, ks, "untracked"); + assertAllNodesSee(cluster, ks, ReplicationType.untracked); + + // Spy on commit verbs AND refresh verbs. Count messages that still carry a mutation ID + // after migration to untracked — any non-zero count indicates stripping failed at one + // of these sites: + // - PaxosCommitAndPrepare.java (commitAndPrepare path) + // - PaxosPrepareRefresh.java (refresh path) + // - StorageProxy.java sendCommit (V1 repair path) + // - StorageProxy.java commitPaxos (reconcile path) + // Which path fires depends on PaxosPrepare outcome (FOUND_INCOMPLETE_COMMITTED vs refresh + // via haveReadResponseWithLatest), which varies with prepare response timing. We verify + // the invariant that applies to ALL paths: after migration to untracked, no commit/refresh + // message sent to a remote replica should carry a mutation ID. + try (MessageSpy spy = on(cluster, + Verb.PAXOS_COMMIT_REQ, + Verb.PAXOS2_COMMIT_AND_PREPARE_REQ, + Verb.PAXOS2_PREPARE_REFRESH_REQ) + .to(2, 3, 4) + .checkMutationId() + .expect(2) + .start()) + { + // Second CAS from node 1 (data node for TrackedRead). Prepare discovers + // FOUND_INCOMPLETE_COMMITTED from node 2's committed ballot (which has a mutation ID). + // PaxosCommitAndPrepare.commitAndPrepare() loads the commit with ID. + // shouldBeTracked=false -> the code strips the mutation ID. + // If the stripping didn't fire, the untracked commit path would reject the mutation with an ID. + Object[][] result = cluster.coordinator(1).execute("UPDATE " + ks + ".tbl SET v = 99 WHERE k = " + KEY + " IF v = 42", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + spy.await(); + + assertCasApplied(result); + + // Core invariant: no PAXOS_COMMIT_REQ / PAXOS2_COMMIT_AND_PREPARE_REQ / PAXOS2_PREPARE_REFRESH_REQ + // should carry a mutation ID after migration to untracked. If stripping failed anywhere, the + // handler would reject with CoordinatorBehindException → CAS fails. + assertEquals("No commit/refresh message should carry a mutation ID after migration to untracked", + 0, spy.withMutationId()); + } + + assertReplicasHaveValue(cluster, ks, KEY, 99, 1, 2, 3); + } + + /* + * Verifies: PaxosCommit.onFailure() does NOT call submitHint() for tracked mutations. + * Tracked mutations use MutationTrackingService for retries, not the hint system. + * Writing hints with mutation IDs causes IllegalStateException on replay after migration. + */ + @Test + public void testTrackedPaxosCommitDoesNotWriteHints() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "tracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + // Reject hints for mutations created before now — prevents prior tests' + // delayed PaxosCommit.onFailure callbacks from contaminating the hint count. + // Current test's mutations are created after this threshold and pass through. + // Capture inside node 1 (the coordinator) so both sides of the comparison use the same clock. + long now = cluster.get(1).callOnInstance(() -> System.nanoTime()); + cluster.forEach(instance -> instance.runOnInstance(() -> HintsService.setRejectHintsBeforeNanos(now))); + + long hintsBefore = cluster.get(1).callOnInstance(() -> + StorageMetrics.totalHints.getCount()); + + // Drop PAXOS_COMMIT_REQ from node 1 to node 3 and respond with TIMEOUT immediately + // so PaxosCommit.onFailure() fires without waiting for the real write_request_timeout. + // shouldHint() returns true for a live node; with the fix, isTracked()=true prevents the hint write. + cluster.filters() + .inbound(true) + .verbs(Verb.PAXOS_COMMIT_REQ.id) + .from(1).to(3) + .messagesMatching((from, to, msg) -> { + cluster.get(to).runOnInstance(() -> PaxosMigrationTestUtils.respondWithTimeout(msg)); + return true; + }).drop(); + + // CAS from node 1 at QUORUM. Nodes 1,2 succeed (quorum met), node 3 times out. + // PaxosCommit.onFailure fires for node 3. With the fix, no hint is written. + Object[][] result = cluster.coordinator(1).execute("INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS", + ConsistencyLevel.SERIAL, ConsistencyLevel.QUORUM); + + assertCasApplied(result); + + // Synthetic TIMEOUT fires the callback almost immediately; 1s is a generous safety margin. + // With isTracked()=true, submitHint should NOT be called. + Thread.sleep(1000); + + long hintsAfter = cluster.get(1).callOnInstance(() -> + StorageMetrics.totalHints.getCount()); + assertEquals("No hints should be written for tracked Paxos commits", hintsBefore, hintsAfter); + + // Verify data state: quorum committed on nodes 1,2 (node 3 was blocked). + cluster.filters().reset(); + assertReplicasHaveValue(cluster, ks, KEY, 42, 1, 2); + } + + /* + * Exercises the commit-retry loop's deadline-exit path in Paxos.cas(). V2 checks migration + * state at PREPARE (via checkPaxosPrepareReadMigration), so to ever reach the commit retry + * loop we must let prepare succeed at the initial agreement, then create a per-replica + * migration disagreement that only surfaces at commit time. We do that by TCM-stranding + * node 2 as the coordinator, holding PAXOS_COMMIT_REQ at nodes 1,3 until after an ALTER + * that nodes 1,3 enact but node 2 cannot, then releasing. The handlers then reject with + * COORDINATOR_BEHIND, and node 2 — unable to catch up via TCM — exhausts commitDeadline + * before a retry iteration can fire. + * + * Known ambiguity: the CAS could also fail via ConditionAsConsumer.awaitUntil(commitDeadline) + * expiring before cb.onFailure signals (if COORDINATOR_BEHIND responses arrive at the + * boundary). The two paths produce different exceptions (CasWriteTimeoutException vs + * WriteFailureException); the test accepts both via assertCasException. The invariant under + * test is the user-visible one: V2 CAS fails with a CAS exception when the coordinator is + * persistently behind after prepare succeeds. Exact count==2 asserts only the initial + * commit batch landed (no retry iteration). + */ + @Test + public void testCommitRetryLoopTimeout() throws Throwable + { + String ks = createKeyspace(cluster, "pmt_v2", "tracked"); + assertReplicasAreExactly(cluster, ks, KEY, new int[]{ 1, 2, 3 }); + + try (EpochPin pin = epochPin(cluster, 2)) + { + // Hold PAXOS_COMMIT_REQ from node 2 at nodes 1,3 until after the ALTER. Prepare + // is not filtered so it proceeds to completion. + try (MessageSpy hold = on(cluster, Verb.PAXOS_COMMIT_REQ) + .from(2) + .to(1, 3) + .holdAll() + .checkMutationId() + .start()) + { + CompletableFuture casResult = casAsyncExpectingFailure(cluster, 2, + "INSERT INTO " + ks + ".tbl (k, v) VALUES (" + KEY + ", 42) IF NOT EXISTS"); + + try + { + hold.awaitFirstArrival(); + + // ALTER to untracked via node 1 at CL.ONE so we don't wait for schema agreement + // on TCM-blocked node 2. + alterReplicationTypeFrom(cluster, 1, ks, "untracked", ConsistencyLevel.ONE); + + // Wait for nodes 1 and 3 to observe untracked. Node 2 must still see tracked. + awaitReplicationType(cluster, ks, ReplicationType.untracked, 1, 3); + assertNodeSees(cluster, 2, ks, ReplicationType.tracked); + } + finally + { + hold.release(); + } + + Throwable error = casResult.get(60, TimeUnit.SECONDS); + assertNotNull("CAS from stranded coordinator must fail", error); + assertTrue("Expected an Exception but got " + error.getClass().getName() + ": " + error.getMessage(), + error instanceof Exception); + assertCasException((Exception) error); + + // Prepare succeeded (no filter on it), so the commit phase ran. The initial commit + // batch to replicas 1,3 is 2 messages; the inner response-handler timeout fires + // before the held TCM fetch releases and lets cb.onFailure signal the retry loop. + assertEquals("Expected exactly 2 inbound PAXOS_COMMIT_REQ (initial batch only; retry iteration never fires because deadline expires before held TCM fetch releases)", + 2, hold.total()); + + // All commits from stranded node 2 carry mutation IDs (tracked path), + // proving COORDINATOR_BEHIND is the rejection reason + assertEquals("All commit attempts from stranded coordinator should carry mutation IDs (tracked path)", + 2, hold.withMutationId()); + } + } + } +} diff --git a/test/unit/org/apache/cassandra/service/replication/migration/MigrationRouterTest.java b/test/unit/org/apache/cassandra/service/replication/migration/MigrationRouterTest.java index 767d1d7c6f70..52a0c91ebefa 100644 --- a/test/unit/org/apache/cassandra/service/replication/migration/MigrationRouterTest.java +++ b/test/unit/org/apache/cassandra/service/replication/migration/MigrationRouterTest.java @@ -37,10 +37,13 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.DataRange; import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.EmbeddableSinglePartitionReadCommand; 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.ReadKind; +import org.apache.cassandra.db.SinglePartitionReadCommand; import org.apache.cassandra.db.Slices; import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter; import org.apache.cassandra.db.filter.ColumnFilter; @@ -56,6 +59,11 @@ 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.locator.InetAddressAndPort; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.NoPayload; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.replication.MutationId; import org.apache.cassandra.schema.DistributedSchema; import org.apache.cassandra.schema.KeyspaceMetadata; @@ -68,6 +76,7 @@ import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.FBUtilities; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -590,10 +599,9 @@ public void testWriteRoutingToTracked_AlwaysTracked() assertTrue(MigrationRouter.shouldUseTrackedForWrites(metadata, TEST_KEYSPACE, testTable.id, tokenOutsidePending)); } - /** - * Test mutation routing with multiple tables - some tracked, some untracked. - * This verifies that routeMutations correctly filters mutations to separate tracked/untracked tables. + * Test mutation routing with multiple tables during migration to tracked. + * All tables in the mutation are routed as tracked (both migrating and already-completed). */ @Test public void testMultiTableMutationRouting_ToTracked() @@ -608,7 +616,7 @@ public void testMultiTableMutationRouting_ToTracked() ClusterMetadata.Transformer transformer = metadata.transformer(); - // table1 migrating to untracked, table2 complete + // table1 still migrating (pending), table2 migration complete MutationTrackingMigrationState migrationState = metadata.mutationTrackingMigrationState.withKeyspaceMigrating(ksm.name, Collections.singleton(table1.id), transformer.epoch()); metadata = transformer.with(migrationState).build().metadata; @@ -628,4 +636,229 @@ public void testMultiTableMutationRouting_ToTracked() assertEquals(0, routed.untrackedMutations.size()); } + + // Null keyspace metadata guard tests + + @Test + public void testShouldUseTrackedForWritesWithNullKeyspace() + { + // ClusterMetadata without the test keyspace — simulates concurrent keyspace drop + ClusterMetadata metadata = ClusterMetadata.current(); + + // Ensure the non-existent keyspace is not in the metadata + assertFalse("Keyspace should not exist in metadata", + metadata.schema.getKeyspaces().containsKeyspace("nonexistent_ks")); + + // Should return false, not NPE + assertFalse(MigrationRouter.shouldUseTrackedForWrites(metadata, "nonexistent_ks", + TableId.generate(), createToken(100))); + } + + @Test + public void testShouldUseTrackedForWritesWithNullKeyspaceDuringMigration() + { + // Create metadata WITH migration info but WITHOUT the keyspace itself + ClusterMetadata metadata = ClusterMetadata.current(); + + // Add migration info for a keyspace that doesn't exist in the schema + KeyspaceMigrationInfo migrationInfo = new KeyspaceMigrationInfo("dropped_ks", + Collections.emptyMap(), + Epoch.create(1)); + MutationTrackingMigrationState migrationState = new MutationTrackingMigrationState(Epoch.create(1), + ImmutableMap.of("dropped_ks", migrationInfo)); + metadata = withMigrationInfo(metadata, migrationState); + + // Should return false, not NPE + assertFalse(MigrationRouter.shouldUseTrackedForWrites(metadata, "dropped_ks", + TableId.generate(), createToken(100))); + } + + @Test + public void testSplitRangeReadForMigrationWithNullKeyspace() + { + ClusterMetadata metadata = ClusterMetadata.current(); + + // Create a range read command for a non-existent keyspace using a table metadata + // that references a keyspace not in the cluster metadata + TableMetadata nonexistentTable = TableMetadata.builder("nonexistent_ks", "test_table") + .addPartitionKeyColumn("pk", UTF8Type.instance) + .addRegularColumn("value", UTF8Type.instance) + .partitioner(partitioner) + .build(); + + PartitionRangeReadCommand cmd = createRangeCommand(nonexistentTable, createToken(0), createToken(1000)); + + List result = + MigrationRouter.splitRangeRead(metadata, cmd); + + // Should return single untracked entry, not NPE + assertEquals(1, result.size()); + assertFalse("Should be untracked for non-existent keyspace", result.get(0).useTracked); + } + + @Test + public void testShouldUseTrackedWithNullKeyspace() + { + TableMetadata nonexistentTable = TableMetadata.builder("nonexistent_ks", "test_table") + .addPartitionKeyColumn("pk", UTF8Type.instance) + .addRegularColumn("value", UTF8Type.instance) + .partitioner(partitioner) + .build(); + + DecoratedKey key = partitioner.decorateKey(UTF8Type.instance.decompose("test_key")); + SinglePartitionReadCommand cmd = SinglePartitionReadCommand.fullPartitionRead(nonexistentTable, 0, key); + + assertFalse(MigrationRouter.shouldUseTracked(cmd)); + } + + /** + * Minimal test-only stub for EmbeddableSinglePartitionReadCommand that lets us control + * the kind() (and therefore isTracked()) without constructing a full TrackedRead.DataRequest + * or TrackedRead.SummaryRequest. + */ + private static EmbeddableSinglePartitionReadCommand readStub(TableMetadata table, DecoratedKey key, ReadKind kind) + { + return new EmbeddableSinglePartitionReadCommand() + { + @Override + public ReadKind kind() + { + return kind; + } + + @Override + public TableMetadata metadata() + { + return table; + } + + @Override + public DecoratedKey partitionKey() + { + return key; + } + }; + } + + @Test + public void testCheckPaxosCommitMigration_Agreement() + { + ClusterMetadata metadata = createMetadata(true, Collections.emptyList()); + TableMetadata table = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + Token token = createToken(0L); + Message msg = Message.builder(Verb._TEST_1, NoPayload.noPayload).withEpoch(metadata.epoch).build(); + InetAddressAndPort respondTo = FBUtilities.getBroadcastAddressAndPort(); + + // Tracked keyspace, no migration → handler says tracked. Coordinator says tracked → agreement. + ClusterMetadata result = MigrationRouter.checkPaxosCommitMigration(metadata, msg, respondTo, + TEST_KEYSPACE, table.id, token, true); + Assert.assertSame("Metadata should be returned unchanged on agreement", metadata, result); + } + + @Test + public void testCheckPaxosCommitMigration_CoordinatorBehind() + { + ClusterMetadata metadata = createMetadata(true, Collections.emptyList()); + TableMetadata table = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + Token token = createToken(0L); + // Message epoch is strictly before metadata.epoch. + Message msg = Message.builder(Verb._TEST_1, NoPayload.noPayload).withEpoch(Epoch.EMPTY).build(); + InetAddressAndPort respondTo = FBUtilities.getBroadcastAddressAndPort(); + + try + { + // Coordinator says untracked (stale), handler says tracked → disagreement with older epoch → CBE. + MigrationRouter.checkPaxosCommitMigration(metadata, msg, respondTo, TEST_KEYSPACE, table.id, token, false); + Assert.fail("Expected CoordinatorBehindException"); + } + catch (CoordinatorBehindException e) + { + Assert.assertTrue("Exception message should mention coordinator is behind: " + e.getMessage(), + e.getMessage().contains("coordinator") && e.getMessage().contains("behind")); + } + } + + @Test + public void testCheckPaxosCommitMigration_SameEpochDisagreement() + { + ClusterMetadata metadata = createMetadata(true, Collections.emptyList()); + TableMetadata table = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + Token token = createToken(0L); + Message msg = Message.builder(Verb._TEST_1, NoPayload.noPayload).withEpoch(metadata.epoch).build(); + InetAddressAndPort respondTo = FBUtilities.getBroadcastAddressAndPort(); + + try + { + // Same epoch, but coordinator says untracked while handler says tracked → inconsistent routing → ISE. + MigrationRouter.checkPaxosCommitMigration(metadata, msg, respondTo, TEST_KEYSPACE, table.id, token, false); + Assert.fail("Expected IllegalStateException"); + } + catch (IllegalStateException e) + { + Assert.assertTrue("Exception message should mention inconsistent routing: " + e.getMessage(), + e.getMessage().contains("Inconsistent")); + } + } + + @Test + public void testCheckPaxosPrepareReadMigration_Agreement() + { + ClusterMetadata metadata = createMetadata(true, Collections.emptyList()); + TableMetadata table = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + DecoratedKey key = partitioner.decorateKey(UTF8Type.instance.decompose("test_key")); + Message msg = Message.builder(Verb._TEST_1, NoPayload.noPayload).withEpoch(metadata.epoch).build(); + InetAddressAndPort respondTo = FBUtilities.getBroadcastAddressAndPort(); + + // Tracked keyspace, no migration → handler says tracked. Coordinator says tracked (TRACKED_DATA) → agreement. + EmbeddableSinglePartitionReadCommand trackedRead = readStub(table, key, ReadKind.TRACKED_DATA); + ClusterMetadata result = MigrationRouter.checkPaxosPrepareReadMigration(metadata, msg, respondTo, trackedRead); + Assert.assertSame("Metadata should be returned unchanged on agreement", metadata, result); + } + + @Test + public void testCheckPaxosPrepareReadMigration_CoordinatorBehind() + { + ClusterMetadata metadata = createMetadata(true, Collections.emptyList()); + TableMetadata table = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + DecoratedKey key = partitioner.decorateKey(UTF8Type.instance.decompose("test_key")); + // Message epoch is strictly before metadata.epoch. + Message msg = Message.builder(Verb._TEST_1, NoPayload.noPayload).withEpoch(Epoch.EMPTY).build(); + InetAddressAndPort respondTo = FBUtilities.getBroadcastAddressAndPort(); + + try + { + // Coordinator says untracked (stale), handler says tracked → disagreement with older epoch → CBE. + EmbeddableSinglePartitionReadCommand untrackedRead = readStub(table, key, ReadKind.UNTRACKED); + MigrationRouter.checkPaxosPrepareReadMigration(metadata, msg, respondTo, untrackedRead); + Assert.fail("Expected CoordinatorBehindException"); + } + catch (CoordinatorBehindException e) + { + Assert.assertTrue("Exception message should mention coordinator is behind: " + e.getMessage(), + e.getMessage().contains("coordinator") && e.getMessage().contains("behind")); + } + } + + @Test + public void testCheckPaxosPrepareReadMigration_SameEpochDisagreement() + { + ClusterMetadata metadata = createMetadata(true, Collections.emptyList()); + TableMetadata table = metadata.schema.getKeyspaceMetadata(TEST_KEYSPACE).getTableOrViewNullable(TEST_TABLE); + DecoratedKey key = partitioner.decorateKey(UTF8Type.instance.decompose("test_key")); + Message msg = Message.builder(Verb._TEST_1, NoPayload.noPayload).withEpoch(metadata.epoch).build(); + InetAddressAndPort respondTo = FBUtilities.getBroadcastAddressAndPort(); + + try + { + // Same epoch, but coordinator says untracked (UNTRACKED) while handler says tracked → ISE. + EmbeddableSinglePartitionReadCommand untrackedRead = readStub(table, key, ReadKind.UNTRACKED); + MigrationRouter.checkPaxosPrepareReadMigration(metadata, msg, respondTo, untrackedRead); + Assert.fail("Expected IllegalStateException"); + } + catch (IllegalStateException e) + { + Assert.assertTrue("Exception message should mention inconsistent routing: " + e.getMessage(), + e.getMessage().contains("Inconsistent")); + } + } }