From d74f11b0d802e93666937ae1b1aa1a890bf33733 Mon Sep 17 00:00:00 2001 From: Abe Ratnofsky Date: Fri, 12 Dec 2025 12:32:44 -0500 Subject: [PATCH 1/2] WIP: Checkpointing progress for full repair support for tracked keyspaces This comes with lots of comments and a handful of integration tests, but the tests aren't passing and it coul use more unit test coverage. There's a refactor (introducing SyncTasks) that's only partially complete. The rest of this commit message is a description of how full repair is intended to work for tracked keyspaces. Tracked keyspaces cannot accept new data without first registering it in the log. Any unreconciled data that isn't present in the log will break read monotonicity, since mutation tracking uses a single data read and can only read reconcile mutation IDs that are present in the log. For more information about how bulk transfers work on tracked keyspaces, see TrackedImportTransfer. Full repair sync tasks also deliver data to replicas, and require integration with the log just like imports do. For more details on a read anomaly that could happen without integration with the bulk transfer machinery, see TrackedKeyspaceRepairSupportTest#testFullRepairPartiallyCompleteAnomaly. The general design of this integration is to give repair SyncTasks the same two-phase commit as import transfers, where we stream SSTables to a pending/ directory, then once sufficient streams complete successfully, we "activate" those streams and move them out of the pending directory and into the live set. The first step is to ensure that each SyncTask is aligned to a single Mutation Tracking shard, by splitting SyncTasks along the shard boundaries. Each SyncTask will then stream data within a single shard, and permit us to assign a single transfer ID to each SyncTask. Each participant in a repair may receive different SyncTasks (or none at all, if they're already in-sync). This means that TransferActivation needs to be made more flexible, and support a single TransferActivation with multiple plan IDs, or no plan IDs at all. This increase in flexibility has not yet been implemented. patch by Abe Ratnofsky; reviewed by Caleb Rackliffe and ? for CASSANDRA-21066 --- .../cassandra/db/ColumnFamilyStore.java | 1 + .../apache/cassandra/db/SSTableImporter.java | 13 +- .../db/streaming/CassandraStreamManager.java | 3 +- .../db/streaming/CassandraStreamReceiver.java | 5 +- .../repair/AsymmetricRemoteSyncTask.java | 28 +- .../repair/CompletableRemoteSyncTask.java | 3 +- .../cassandra/repair/LocalSyncTask.java | 30 +- .../cassandra/repair/RepairCoordinator.java | 25 + .../apache/cassandra/repair/RepairJob.java | 40 +- .../cassandra/repair/RepairSession.java | 2 +- .../cassandra/repair/StreamingRepairTask.java | 4 +- .../repair/SymmetricRemoteSyncTask.java | 26 +- .../org/apache/cassandra/repair/SyncStat.java | 28 +- .../org/apache/cassandra/repair/SyncTask.java | 44 +- .../apache/cassandra/repair/SyncTasks.java | 107 +++ .../repair/messages/SyncResponse.java | 22 +- .../AbstractCoordinatedBulkTransfer.java | 398 ++++++++++ .../replication/ActiveLogReconciler.java | 8 +- .../replication/CoordinatedTransfer.java | 720 ------------------ .../cassandra/replication/CoordinatorLog.java | 2 +- .../cassandra/replication/LocalTransfers.java | 203 ++++- .../replication/MutationTrackingService.java | 170 ++++- .../replication/PendingLocalTransfer.java | 10 +- .../apache/cassandra/replication/Shard.java | 13 +- .../replication/TrackedImportTransfer.java | 389 ++++++++++ ...sfers.java => TrackedImportTransfers.java} | 18 +- .../TrackedRepairSyncTransfer.java | 101 +++ .../replication/TransferActivation.java | 35 +- .../cassandra/replication/TransferFailed.java | 2 +- .../replication/UnreconciledMutations.java | 5 +- .../cassandra/service/StorageService.java | 10 + .../cassandra/streaming/StreamPlan.java | 17 +- .../cassandra/streaming/StreamSession.java | 9 + .../tcm/ownership/ReplicaGroups.java | 9 +- .../cassandra/utils/concurrent/Ref.java | 4 +- test/conf/logback-dtest.xml | 2 +- test/conf/logback-test.xml | 6 +- .../TrackedKeyspaceRepairSupportTest.java | 519 +++++++++++++ .../test/tracking/BulkTransfersTest.java | 27 +- .../cassandra/repair/LocalSyncTaskTest.java | 4 +- .../cassandra/repair/RepairJobTest.java | 4 +- .../repair/SymmetricRemoteSyncTaskTest.java | 2 +- .../RepairMessageSerializationsTest.java | 2 +- .../AlignedToShardBoundariesTest.java | 218 ++++++ .../replication/LocalTransfersTest.java | 106 +-- .../cassandra/service/SerializationsTest.java | 4 +- 46 files changed, 2495 insertions(+), 903 deletions(-) create mode 100644 src/java/org/apache/cassandra/repair/SyncTasks.java create mode 100644 src/java/org/apache/cassandra/replication/AbstractCoordinatedBulkTransfer.java delete mode 100644 src/java/org/apache/cassandra/replication/CoordinatedTransfer.java create mode 100644 src/java/org/apache/cassandra/replication/TrackedImportTransfer.java rename src/java/org/apache/cassandra/replication/{CoordinatedTransfers.java => TrackedImportTransfers.java} (76%) create mode 100644 src/java/org/apache/cassandra/replication/TrackedRepairSyncTransfer.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/TrackedKeyspaceRepairSupportTest.java create mode 100644 test/unit/org/apache/cassandra/replication/AlignedToShardBoundariesTest.java diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index f4631e689a54..f289e881b388 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -1690,6 +1690,7 @@ public Refs getAndReferenceOverlappingLiveSSTables(Iterable importNewSSTables(Options options) cfs.indexManager.buildSSTableAttachedIndexesBlocking(newSSTables); if (isTracked) - TrackedBulkTransfer.execute(cfs.keyspace.getName(), newSSTables); + MutationTrackingService.instance.executeTransfers(cfs.keyspace.getName(), newSSTables, ConsistencyLevel.ALL); else cfs.getTracker().addSSTables(newSSTables); @@ -253,17 +253,6 @@ synchronized List importNewSSTables(Options options) return failedDirectories; } - /** - * TODO: Support user-defined consistency level for import, for import with replicas down - */ - private static class TrackedBulkTransfer - { - private static void execute(String keyspace, Set sstables) - { - MutationTrackingService.instance.executeTransfers(keyspace, sstables, ConsistencyLevel.ALL); - } - } - /** * Check the state of this node and throws an {@link InterruptedException} if it is currently draining * diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java index 2fc802b03614..5aece1dc7e21 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java @@ -130,9 +130,10 @@ public Collection createOutgoingStreams(StreamSession session, R Set sstables = Sets.newHashSet(); SSTableIntervalTree intervalTree = buildSSTableIntervalTree(ImmutableList.copyOf(view.select(SSTableSet.CANONICAL))); Predicate predicate; + // reconciledKeyspaceOffsets are only included when mutation logs are streamed, since we include logs + // for all unreconciled mutations, and SSTables for all reconciled mutations if (reconciledKeyspaceOffsets != null) { - // TODO: relax these restrictions as repair support is add Preconditions.checkArgument(previewKind == PreviewKind.NONE); Preconditions.checkArgument(pendingRepair == ActiveRepairService.NO_PENDING_REPAIR); predicate = getSSTablePredicateForKeyspaceRanges(reconciledKeyspaceOffsets); diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java index 89f869ee0618..6dcd40d5c2b3 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java @@ -54,7 +54,6 @@ import org.apache.cassandra.replication.MutationTrackingService; import org.apache.cassandra.replication.PendingLocalTransfer; import org.apache.cassandra.streaming.IncomingStream; -import org.apache.cassandra.streaming.StreamOperation; import org.apache.cassandra.streaming.StreamReceiver; import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.tcm.ClusterMetadata; @@ -135,7 +134,7 @@ public synchronized void received(IncomingStream stream) sstables.addAll(finished); receivedEntireSSTable = file.isEntireSSTable(); - if (session.streamOperation() == StreamOperation.TRACKED_TRANSFER) + if (session.requiresTrackedActivation()) { Preconditions.checkState(cfs.metadata().replicationType().isTracked()); PendingLocalTransfer transfer = new PendingLocalTransfer(cfs.metadata().id, session.planId(), sstables); @@ -267,7 +266,7 @@ public void finished() logger.debug("[Stream #{}] Received {} sstables from {} ({})", session.planId(), readers.size(), session.peer, readers); // Don't mark as live until activated by the stream coordinator - if (session.streamOperation() == StreamOperation.TRACKED_TRANSFER) + if (session.requiresTrackedActivation()) return; cfs.addSSTables(readers); diff --git a/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java b/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java index 475c38967871..93056a8e2a6b 100644 --- a/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java +++ b/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java @@ -18,17 +18,23 @@ package org.apache.cassandra.repair; +import java.util.ArrayList; +import java.util.Collection; import java.util.List; +import com.google.common.base.Preconditions; + import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.RepairException; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.repair.messages.SyncRequest; +import org.apache.cassandra.replication.ShortMutationId; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.streaming.SessionSummary; import org.apache.cassandra.tracing.Tracing; +import org.apache.cassandra.utils.TimeUUID; /** * AsymmetricRemoteSyncTask sends {@link SyncRequest} to target node to repair(stream) @@ -38,9 +44,23 @@ */ public class AsymmetricRemoteSyncTask extends SyncTask implements CompletableRemoteSyncTask { - public AsymmetricRemoteSyncTask(SharedContext ctx, RepairJobDesc desc, InetAddressAndPort to, InetAddressAndPort from, List> differences, PreviewKind previewKind) + public AsymmetricRemoteSyncTask(SharedContext ctx, RepairJobDesc desc, InetAddressAndPort to, InetAddressAndPort from, List> differences, PreviewKind previewKind, ShortMutationId transferId) + { + super(ctx, desc, to, from, differences, previewKind, transferId); + } + + @Override + public SyncTask withRanges(Collection> newRanges) + { + List> rangeList = newRanges instanceof List ? (List>) newRanges : new ArrayList<>(newRanges); + return new AsymmetricRemoteSyncTask(ctx, desc, nodePair.coordinator, nodePair.peer, rangeList, previewKind, transferId); + } + + @Override + public SyncTask withTransferId(ShortMutationId transferId) { - super(ctx, desc, to, from, differences, previewKind); + Preconditions.checkState(this.transferId == null); + return new AsymmetricRemoteSyncTask(ctx, desc, nodePair.coordinator, nodePair.peer, rangesToSync, previewKind, transferId); } public void startSync() @@ -52,11 +72,11 @@ public void startSync() sendRequest(request, request.src); } - public void syncComplete(boolean success, List summaries) + public void syncComplete(boolean success, List summaries, TimeUUID planId) { if (success) { - trySuccess(stat.withSummaries(summaries)); + trySuccess(stat.withSummaries(summaries, planId)); } else { diff --git a/src/java/org/apache/cassandra/repair/CompletableRemoteSyncTask.java b/src/java/org/apache/cassandra/repair/CompletableRemoteSyncTask.java index c4fe6c8b20c5..a059559b6a46 100644 --- a/src/java/org/apache/cassandra/repair/CompletableRemoteSyncTask.java +++ b/src/java/org/apache/cassandra/repair/CompletableRemoteSyncTask.java @@ -21,8 +21,9 @@ import java.util.List; import org.apache.cassandra.streaming.SessionSummary; +import org.apache.cassandra.utils.TimeUUID; public interface CompletableRemoteSyncTask { - void syncComplete(boolean success, List summaries); + void syncComplete(boolean success, List summaries, TimeUUID planId); } diff --git a/src/java/org/apache/cassandra/repair/LocalSyncTask.java b/src/java/org/apache/cassandra/repair/LocalSyncTask.java index 379ba4b2a1b6..0a243a326f81 100644 --- a/src/java/org/apache/cassandra/repair/LocalSyncTask.java +++ b/src/java/org/apache/cassandra/repair/LocalSyncTask.java @@ -17,6 +17,8 @@ */ package org.apache.cassandra.repair; +import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; @@ -30,6 +32,7 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.RangesAtEndpoint; +import org.apache.cassandra.replication.ShortMutationId; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.streaming.ProgressInfo; import org.apache.cassandra.streaming.StreamEvent; @@ -66,9 +69,9 @@ public class LocalSyncTask extends SyncTask implements StreamEventHandler public LocalSyncTask(SharedContext ctx, RepairJobDesc desc, InetAddressAndPort local, InetAddressAndPort remote, List> diff, TimeUUID pendingRepair, - boolean requestRanges, boolean transferRanges, PreviewKind previewKind) + boolean requestRanges, boolean transferRanges, PreviewKind previewKind, ShortMutationId transferId) { - super(ctx, desc, local, remote, diff, previewKind); + super(ctx, desc, local, remote, diff, previewKind, transferId); Preconditions.checkArgument(requestRanges || transferRanges, "Nothing to do in a sync job"); Preconditions.checkArgument(local.equals(ctx.broadcastAddressAndPort())); @@ -77,6 +80,27 @@ public LocalSyncTask(SharedContext ctx, RepairJobDesc desc, InetAddressAndPort l this.transferRanges = transferRanges; } + public LocalSyncTask(SharedContext ctx, RepairJobDesc desc, InetAddressAndPort local, InetAddressAndPort remote, + List> diff, TimeUUID pendingRepair, + boolean requestRanges, boolean transferRanges, PreviewKind previewKind) + { + this(ctx, desc, local, remote, diff, pendingRepair, requestRanges, transferRanges, previewKind, null); + } + + @Override + public SyncTask withRanges(Collection> newRanges) + { + List> rangeList = newRanges instanceof List ? (List>) newRanges : new ArrayList<>(newRanges); + return new LocalSyncTask(ctx, desc, nodePair.coordinator, nodePair.peer, rangeList, pendingRepair, requestRanges, transferRanges, previewKind); + } + + @Override + public SyncTask withTransferId(ShortMutationId transferId) + { + Preconditions.checkState(this.transferId == null); + return new LocalSyncTask(ctx, desc, nodePair.coordinator, nodePair.peer, rangesToSync, pendingRepair, requestRanges, transferRanges, previewKind, transferId); + } + @VisibleForTesting StreamPlan createStreamPlan() { @@ -167,7 +191,7 @@ public void onSuccess(StreamState result) status, desc.sessionId, nodePair.coordinator, nodePair.peer, desc.columnFamily); logger.info("{} {}", previewKind.logPrefix(desc.sessionId), message); Tracing.traceRepair(message); - trySuccess(result.hasAbortedSession() ? stat : stat.withSummaries(result.createSummaries())); + trySuccess(result.hasAbortedSession() ? stat : stat.withSummaries(result.createSummaries(), result.planId)); finished(); } } diff --git a/src/java/org/apache/cassandra/repair/RepairCoordinator.java b/src/java/org/apache/cassandra/repair/RepairCoordinator.java index b511d081c984..cce30c0e293c 100644 --- a/src/java/org/apache/cassandra/repair/RepairCoordinator.java +++ b/src/java/org/apache/cassandra/repair/RepairCoordinator.java @@ -67,6 +67,9 @@ import org.apache.cassandra.repair.messages.RepairOption; import org.apache.cassandra.repair.state.CoordinatorState; import org.apache.cassandra.repair.state.ParticipateState; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.ReplicationType; +import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.SystemDistributedKeyspace; import org.apache.cassandra.schema.TableMetadata; @@ -298,6 +301,28 @@ private static void validate(RepairOption options, List colum if (options.paxosOnly() && options.accordOnly()) throw new IllegalArgumentException("Cannot specify a repair as both paxos only and accord only"); + // Check for tracked keyspaces - repairs are not supported on tracked keyspaces + Set trackedKeyspaces = new HashSet<>(); + for (ColumnFamilyStore cfs : columnFamilies) + { + String keyspaceName = cfs.keyspace.getName(); + KeyspaceMetadata keyspaceMetadata = Schema.instance.getKeyspaceMetadata(keyspaceName); + if (keyspaceMetadata != null && keyspaceMetadata.params.replicationType == ReplicationType.tracked) + { + trackedKeyspaces.add(keyspaceName); + } + } + + /* + if (!trackedKeyspaces.isEmpty()) + { + throw new IllegalArgumentException(String.format("Repair is not supported on tracked keyspaces: %s. " + + "Tracked keyspaces use mutation tracking for durability guarantees " + + "and are incompatible with traditional repair mechanisms.", + String.join(", ", trackedKeyspaces))); + } + */ + for (ColumnFamilyStore cfs : columnFamilies) { TableMetadata metadata = cfs.metadata(); diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java index 28640a60a4d2..5a49417db153 100644 --- a/src/java/org/apache/cassandra/repair/RepairJob.java +++ b/src/java/org/apache/cassandra/repair/RepairJob.java @@ -50,6 +50,7 @@ import org.apache.cassandra.repair.asymmetric.PreferedNodeFilter; import org.apache.cassandra.repair.asymmetric.ReduceHelper; import org.apache.cassandra.repair.state.JobState; +import org.apache.cassandra.replication.LocalTransfers; import org.apache.cassandra.schema.SystemDistributedKeyspace; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.accord.IAccordService; @@ -120,7 +121,6 @@ public RepairJob(RepairSession session, String columnFamily) if ((!session.repairData && !session.repairPaxos) && !metadata.requiresAccordSupport()) throw new IllegalArgumentException(String.format("Cannot run accord only repair on %s.%s, which isn't configured for accord operations", cfs.keyspace.getName(), cfs.name)); - } public long getNowInSeconds() @@ -136,6 +136,11 @@ public long getNowInSeconds() } } + public Collection getSyncTasks() + { + return syncTasks; + } + @Override public void run() { @@ -254,6 +259,11 @@ protected void runRepair() // that there are in memory at once. When all validations complete, submit sync tasks out of the scheduler. syncResults = session.validationScheduler.schedule(() -> createSyncTasks(accordRepair, allSnapshotTasks, allEndpoints), taskExecutor) .flatMap(this::executeTasks, taskExecutor); + + // For tracked keyspaces, we need to ensure sync'd data is present in the log + boolean isTracked = cfs.metadata().replicationType().isTracked(); + if (isTracked) + syncResults = LocalTransfers.instance().onRepairSyncCompletion(this, syncResults, taskExecutor); } else { @@ -306,7 +316,7 @@ public void onFailure(Throwable t) }, taskExecutor); } - private Future> createSyncTasks(Future accordRepair, Future allSnapshotTasks, List allEndpoints) + private Future createSyncTasks(Future accordRepair, Future allSnapshotTasks, List allEndpoints) { Future> treeResponses; if (allSnapshotTasks != null) @@ -330,9 +340,14 @@ private Future> createSyncTasks(Future accord return a; }); - return treeResponses.map(session.optimiseStreams && !session.pullRepair - ? this::createOptimisedSyncingSyncTasks - : this::createStandardSyncTasks, taskExecutor); + return treeResponses.map(trees -> { + List syncTasks; + if (session.optimiseStreams && !session.pullRepair) + syncTasks = createOptimisedSyncingSyncTasks(trees); + else + syncTasks = createStandardSyncTasks(trees); + return SyncTasks.alignedToShardBoundaries(desc, syncTasks); + }, taskExecutor); } public synchronized void abort(@Nullable Throwable reason) @@ -413,18 +428,18 @@ static List createStandardSyncTasks(SharedContext ctx, continue; task = new LocalSyncTask(ctx, desc, self.endpoint, remote.endpoint, differences, isIncremental ? desc.parentSessionId : null, - requestRanges, transferRanges, previewKind); + requestRanges, transferRanges, previewKind, null); } else if (isTransient.test(r1.endpoint) || isTransient.test(r2.endpoint)) { // Stream only from transient replica TreeResponse streamFrom = isTransient.test(r1.endpoint) ? r1 : r2; TreeResponse streamTo = isTransient.test(r1.endpoint) ? r2 : r1; - task = new AsymmetricRemoteSyncTask(ctx, desc, streamTo.endpoint, streamFrom.endpoint, differences, previewKind); + task = new AsymmetricRemoteSyncTask(ctx, desc, streamTo.endpoint, streamFrom.endpoint, differences, previewKind, null); } else { - task = new SymmetricRemoteSyncTask(ctx, desc, r1.endpoint, r2.endpoint, differences, previewKind); + task = new SymmetricRemoteSyncTask(ctx, desc, r1.endpoint, r2.endpoint, differences, previewKind, null); } syncTasks.add(task); } @@ -437,7 +452,7 @@ else if (isTransient.test(r1.endpoint) || isTransient.test(r2.endpoint)) } @VisibleForTesting - Future> executeTasks(List tasks) + Future> executeTasks(SyncTasks tasks) { try { @@ -447,6 +462,9 @@ Future> executeTasks(List tasks) if (!tasks.isEmpty()) state.phase.streamSubmitted(); + if (cfs.metadata().replicationType().isTracked()) + LocalTransfers.instance().onRepairSyncExecution(this, desc, tasks); + for (SyncTask task : tasks) { if (!task.isLocal()) @@ -535,11 +553,11 @@ static List createOptimisedSyncingSyncTasks(SharedContext ctx, if (address.equals(local)) { task = new LocalSyncTask(ctx, desc, address, fetchFrom, toFetch, isIncremental ? desc.parentSessionId : null, - true, false, previewKind); + true, false, previewKind, null); } else { - task = new AsymmetricRemoteSyncTask(ctx, desc, address, fetchFrom, toFetch, previewKind); + task = new AsymmetricRemoteSyncTask(ctx, desc, address, fetchFrom, toFetch, previewKind, null); } syncTasks.add(task); diff --git a/src/java/org/apache/cassandra/repair/RepairSession.java b/src/java/org/apache/cassandra/repair/RepairSession.java index 106950bb4afd..74145e44a0bc 100644 --- a/src/java/org/apache/cassandra/repair/RepairSession.java +++ b/src/java/org/apache/cassandra/repair/RepairSession.java @@ -277,7 +277,7 @@ public void syncComplete(RepairJobDesc desc, Message message) if (logger.isDebugEnabled()) logger.debug("{} Repair completed between {} and {} on {}", previewKind.logPrefix(getId()), nodes.coordinator, nodes.peer, desc.columnFamily); - task.syncComplete(message.payload.success, message.payload.summaries); + task.syncComplete(message.payload.success, message.payload.summaries, message.payload.planId); } @VisibleForTesting diff --git a/src/java/org/apache/cassandra/repair/StreamingRepairTask.java b/src/java/org/apache/cassandra/repair/StreamingRepairTask.java index 0f84d66893ef..ba41ce3a8706 100644 --- a/src/java/org/apache/cassandra/repair/StreamingRepairTask.java +++ b/src/java/org/apache/cassandra/repair/StreamingRepairTask.java @@ -120,7 +120,7 @@ public void onSuccess(StreamState state) { logger.info("[repair #{}] streaming task succeed, returning response to {}", desc.sessionId, initiator); this.state.phase.success(); - RepairMessage.sendMessageWithRetries(ctx, new SyncResponse(desc, src, dst, true, state.createSummaries()), SYNC_RSP, initiator); + RepairMessage.sendMessageWithRetries(ctx, new SyncResponse(desc, src, dst, true, state.createSummaries(), state.planId), SYNC_RSP, initiator); } /** @@ -130,6 +130,6 @@ public void onSuccess(StreamState state) public void onFailure(Throwable t) { this.state.phase.fail(t); - RepairMessage.sendMessageWithRetries(ctx, new SyncResponse(desc, src, dst, false, Collections.emptyList()), SYNC_RSP, initiator); + RepairMessage.sendMessageWithRetries(ctx, new SyncResponse(desc, src, dst, false, Collections.emptyList(), null), SYNC_RSP, initiator); } } diff --git a/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java b/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java index 95b8959407a3..d6f8372f1931 100644 --- a/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java +++ b/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java @@ -17,6 +17,8 @@ */ package org.apache.cassandra.repair; +import java.util.ArrayList; +import java.util.Collection; import java.util.List; import com.google.common.base.Preconditions; @@ -28,9 +30,11 @@ import org.apache.cassandra.exceptions.RepairException; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.repair.messages.SyncRequest; +import org.apache.cassandra.replication.ShortMutationId; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.streaming.SessionSummary; import org.apache.cassandra.tracing.Tracing; +import org.apache.cassandra.utils.TimeUUID; /** * SymmetricRemoteSyncTask sends {@link SyncRequest} to remote(non-coordinator) node @@ -42,9 +46,23 @@ public class SymmetricRemoteSyncTask extends SyncTask implements CompletableRemo { private static final Logger logger = LoggerFactory.getLogger(SymmetricRemoteSyncTask.class); - public SymmetricRemoteSyncTask(SharedContext ctx, RepairJobDesc desc, InetAddressAndPort r1, InetAddressAndPort r2, List> differences, PreviewKind previewKind) + public SymmetricRemoteSyncTask(SharedContext ctx, RepairJobDesc desc, InetAddressAndPort r1, InetAddressAndPort r2, List> differences, PreviewKind previewKind, ShortMutationId transferId) { - super(ctx, desc, r1, r2, differences, previewKind); + super(ctx, desc, r1, r2, differences, previewKind, transferId); + } + + @Override + public SyncTask withRanges(Collection> newRanges) + { + List> rangeList = newRanges instanceof List ? (List>) newRanges : new ArrayList<>(newRanges); + return new SymmetricRemoteSyncTask(ctx, desc, nodePair.coordinator, nodePair.peer, rangeList, previewKind, transferId); + } + + @Override + public SyncTask withTransferId(ShortMutationId transferId) + { + Preconditions.checkState(this.transferId == null); + return new AsymmetricRemoteSyncTask(ctx, desc, nodePair.coordinator, nodePair.peer, rangesToSync, previewKind, transferId); } @Override @@ -59,11 +77,11 @@ protected void startSync() sendRequest(request, request.src); } - public void syncComplete(boolean success, List summaries) + public void syncComplete(boolean success, List summaries, TimeUUID planId) { if (success) { - trySuccess(stat.withSummaries(summaries)); + trySuccess(stat.withSummaries(summaries, planId)); } else { diff --git a/src/java/org/apache/cassandra/repair/SyncStat.java b/src/java/org/apache/cassandra/repair/SyncStat.java index 2241915ab4ab..b20eb456efe9 100644 --- a/src/java/org/apache/cassandra/repair/SyncStat.java +++ b/src/java/org/apache/cassandra/repair/SyncStat.java @@ -23,6 +23,7 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.streaming.SessionSummary; +import org.apache.cassandra.utils.TimeUUID; /** * Statistics about synchronizing two replica @@ -32,21 +33,42 @@ public class SyncStat public final SyncNodePair nodes; public final Collection> differences; public final List summaries; + public final TimeUUID planId; public SyncStat(SyncNodePair nodes, Collection> differences) { this(nodes, differences, null); } - public SyncStat(SyncNodePair nodes, Collection> differences, List summaries) + private SyncStat(SyncNodePair nodes, Collection> differences, List summaries) { this.nodes = nodes; this.summaries = summaries; this.differences = differences; + this.planId = null; } - public SyncStat withSummaries(List summaries) + private SyncStat(SyncNodePair nodes, Collection> differences, List summaries, TimeUUID planId) { - return new SyncStat(nodes, differences, summaries); + this.nodes = nodes; + this.summaries = summaries; + this.differences = differences; + this.planId = planId; + } + + public SyncStat withSummaries(List summaries, TimeUUID planId) + { + return new SyncStat(nodes, differences, summaries, planId); + } + + @Override + public String toString() + { + return "SyncStat{" + + "nodes=" + nodes + + ", differences=" + differences + + ", summaries=" + summaries + + ", planId=" + planId + + '}'; } } diff --git a/src/java/org/apache/cassandra/repair/SyncTask.java b/src/java/org/apache/cassandra/repair/SyncTask.java index a3b1a574937d..f11752c60598 100644 --- a/src/java/org/apache/cassandra/repair/SyncTask.java +++ b/src/java/org/apache/cassandra/repair/SyncTask.java @@ -18,12 +18,14 @@ package org.apache.cassandra.repair; +import java.util.Collection; import java.util.List; import java.util.concurrent.TimeUnit; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import org.apache.cassandra.replication.ShortMutationId; import org.apache.cassandra.utils.concurrent.AsyncFuture; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,8 +36,10 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.repair.messages.RepairMessage; import org.apache.cassandra.repair.messages.SyncRequest; +import org.apache.cassandra.replication.MutationId; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.tracing.Tracing; +import org.apache.cassandra.utils.TimeUUID; import static org.apache.cassandra.net.Verb.SYNC_REQ; import static org.apache.cassandra.repair.messages.RepairMessage.notDone; @@ -50,11 +54,13 @@ public abstract class SyncTask extends AsyncFuture implements Runnable public final List> rangesToSync; protected final PreviewKind previewKind; protected final SyncNodePair nodePair; + protected final ShortMutationId transferId; + protected volatile TimeUUID planId; protected volatile long startTime = Long.MIN_VALUE; protected final SyncStat stat; - protected SyncTask(SharedContext ctx, RepairJobDesc desc, InetAddressAndPort primaryEndpoint, InetAddressAndPort peer, List> rangesToSync, PreviewKind previewKind) + protected SyncTask(SharedContext ctx, RepairJobDesc desc, InetAddressAndPort primaryEndpoint, InetAddressAndPort peer, List> rangesToSync, PreviewKind previewKind, ShortMutationId transferId) { Preconditions.checkArgument(!peer.equals(primaryEndpoint), "Sending and receiving node are the same: %s", peer); this.ctx = ctx; @@ -62,16 +68,52 @@ protected SyncTask(SharedContext ctx, RepairJobDesc desc, InetAddressAndPort pri this.rangesToSync = rangesToSync; this.nodePair = new SyncNodePair(primaryEndpoint, peer); this.previewKind = previewKind; + this.transferId = transferId; this.stat = new SyncStat(nodePair, rangesToSync); + + addCallback((syncStat, failure) -> { + if (syncStat != null && syncStat.planId != null) + this.planId = syncStat.planId; + else if (failure instanceof org.apache.cassandra.streaming.StreamException) + this.planId = ((org.apache.cassandra.streaming.StreamException) failure).finalState.planId; + }); + } + + protected SyncTask(SharedContext ctx, RepairJobDesc desc, InetAddressAndPort primaryEndpoint, InetAddressAndPort peer, List> rangesToSync, PreviewKind previewKind) + { + this(ctx, desc, primaryEndpoint, peer, rangesToSync, previewKind, null); } protected abstract void startSync(); + /** + * Creates a new SyncTask with the same parameters but different ranges. + * Used for splitting sync tasks on shard boundaries. + */ + public abstract SyncTask withRanges(Collection> newRanges); + + public abstract SyncTask withTransferId(ShortMutationId transferId); + public SyncNodePair nodePair() { return nodePair; } + public ShortMutationId getTransferId() + { + return transferId; + } + + /** + * Returns the planId associated with this sync task's streaming operation. + * The planId is captured when the task completes (successfully or with a StreamException). + * @return the planId if streaming has completed, null otherwise + */ + public TimeUUID getPlanId() + { + return planId; + } + /** * Compares trees, and triggers repairs for any ranges that mismatch. */ diff --git a/src/java/org/apache/cassandra/repair/SyncTasks.java b/src/java/org/apache/cassandra/repair/SyncTasks.java new file mode 100644 index 000000000000..04dda09c10c5 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/SyncTasks.java @@ -0,0 +1,107 @@ +/* + * 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.repair; + +import java.util.AbstractCollection; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.function.BiConsumer; + +import com.google.common.collect.Iterators; + +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.replication.MutationId; +import org.apache.cassandra.replication.MutationTrackingService; +import org.apache.cassandra.replication.Participants; +import org.apache.cassandra.replication.ShortMutationId; + +public class SyncTasks extends AbstractCollection +{ + private final Map tasks = new HashMap<>(); + + public static class Entry + { + public Participants participants; + public Collection tasks; + + private Entry(Participants participants, Collection tasks) + { + this.participants = participants; + this.tasks = tasks; + } + } + + static SyncTasks untracked(Collection tasks) + { + SyncTasks syncTasks = new SyncTasks(); + syncTasks.tasks.put(MutationId.none(), new Entry(null, tasks)); + return syncTasks; + } + + /** + * Mutation Tracking manages tracking metadata within shards that are each responsible for a piece of the owned + * token space. Executing a full repair across an entire node's ownership will span multiple shards, so repair sync + * tasks need to be split to each align within a single tracking shard. + */ + static SyncTasks alignedToShardBoundaries(RepairJobDesc desc, List tasks) + { + Keyspace keyspace = Keyspace.open(desc.keyspace); + if (keyspace == null || !keyspace.getMetadata().params.replicationType.isTracked()) + { + return untracked(tasks); + } + + SyncTasks into = new SyncTasks(); + MutationTrackingService.instance.alignToShardBoundaries(desc.keyspace, tasks, into); + return into; + } + + public void addAll(ShortMutationId id, Participants participants, Collection syncTasks) + { + tasks.computeIfAbsent(id, key -> new Entry(participants, new HashSet<>())).tasks.addAll(syncTasks); + } + + public void forEach(BiConsumer consumer) + { + tasks.forEach(consumer); + } + + @Override + public Iterator iterator() + { + Iterator entries = tasks.values().iterator(); + Iterator> tasks = Iterators.transform(entries, + entry -> entry == null ? Collections.emptyIterator() : entry.tasks.iterator()); + return Iterators.concat(tasks); + } + + @Override + public int size() + { + int sum = 0; + for (Entry entry : tasks.values()) + sum += entry.tasks.size(); + return sum; + } +} diff --git a/src/java/org/apache/cassandra/repair/messages/SyncResponse.java b/src/java/org/apache/cassandra/repair/messages/SyncResponse.java index e7b5446badb5..fe5909a0d31f 100644 --- a/src/java/org/apache/cassandra/repair/messages/SyncResponse.java +++ b/src/java/org/apache/cassandra/repair/messages/SyncResponse.java @@ -31,6 +31,7 @@ import org.apache.cassandra.repair.RepairJobDesc; import org.apache.cassandra.repair.SyncNodePair; import org.apache.cassandra.streaming.SessionSummary; +import org.apache.cassandra.utils.TimeUUID; /** * @@ -45,20 +46,24 @@ public class SyncResponse extends RepairMessage public final List summaries; - public SyncResponse(RepairJobDesc desc, SyncNodePair nodes, boolean success, List summaries) + public final TimeUUID planId; + + public SyncResponse(RepairJobDesc desc, SyncNodePair nodes, boolean success, List summaries, TimeUUID planId) { super(desc); this.nodes = nodes; this.success = success; this.summaries = summaries; + this.planId = planId; } - public SyncResponse(RepairJobDesc desc, InetAddressAndPort endpoint1, InetAddressAndPort endpoint2, boolean success, List summaries) + public SyncResponse(RepairJobDesc desc, InetAddressAndPort endpoint1, InetAddressAndPort endpoint2, boolean success, List summaries, TimeUUID planId) { super(desc); this.summaries = summaries; this.nodes = new SyncNodePair(endpoint1, endpoint2); this.success = success; + this.planId = planId; } @Override @@ -70,13 +75,14 @@ public boolean equals(Object o) return desc.equals(other.desc) && success == other.success && nodes.equals(other.nodes) && - summaries.equals(other.summaries); + summaries.equals(other.summaries) && + Objects.equals(planId, other.planId); } @Override public int hashCode() { - return Objects.hash(desc, success, nodes, summaries); + return Objects.hash(desc, success, nodes, summaries, planId); } public static final IPartitionerDependentSerializer serializer = new IPartitionerDependentSerializer() @@ -92,6 +98,8 @@ public void serialize(SyncResponse message, DataOutputPlus out, int version) thr { SessionSummary.serializer.serialize(summary, out, version); } + + TimeUUID.Serializer.instance.serialize(message.planId, out); } @Override @@ -108,7 +116,9 @@ public SyncResponse deserialize(DataInputPlus in, IPartitioner partitioner, int summaries.add(SessionSummary.serializer.deserialize(in, partitioner, version)); } - return new SyncResponse(desc, nodes, success, summaries); + TimeUUID planId = TimeUUID.Serializer.instance.deserialize(in); + + return new SyncResponse(desc, nodes, success, summaries, planId); } public long serializedSize(SyncResponse message, int version) @@ -123,6 +133,8 @@ public long serializedSize(SyncResponse message, int version) size += SessionSummary.serializer.serializedSize(summary, version); } + size += TimeUUID.Serializer.instance.serializedSize(message.planId); + return size; } }; diff --git a/src/java/org/apache/cassandra/replication/AbstractCoordinatedBulkTransfer.java b/src/java/org/apache/cassandra/replication/AbstractCoordinatedBulkTransfer.java new file mode 100644 index 000000000000..9978e4b6d20e --- /dev/null +++ b/src/java/org/apache/cassandra/replication/AbstractCoordinatedBulkTransfer.java @@ -0,0 +1,398 @@ +/* + * 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.replication; + +import java.util.Collection; +import java.util.EnumSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import javax.annotation.CheckReturnValue; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.exceptions.RequestFailure; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.NoPayload; +import org.apache.cassandra.net.RequestCallbackWithFailure; +import org.apache.cassandra.net.Verb; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.Throwables; +import org.apache.cassandra.utils.TimeUUID; +import org.apache.cassandra.utils.concurrent.AsyncFuture; + +import static org.apache.cassandra.replication.AbstractCoordinatedBulkTransfer.SingleTransferResult.State.STREAM_COMPLETE; +import static org.apache.cassandra.replication.TransferActivation.Phase; + +public abstract class AbstractCoordinatedBulkTransfer +{ + private static final Logger logger = LoggerFactory.getLogger(AbstractCoordinatedBulkTransfer.class); + + String logPrefix() + { + return String.format("[%s #%s]", getClass().getSimpleName(), id); + } + + private final ShortMutationId id; + final ConcurrentMap streamResults; + + public AbstractCoordinatedBulkTransfer(ShortMutationId id) + { + this.id = id; + this.streamResults = new ConcurrentHashMap<>(); + } + + ShortMutationId id() + { + return id; + } + + public boolean isCommitted() + { + for (SingleTransferResult result : streamResults.values()) + { + if (result.state != SingleTransferResult.State.COMMITTED) + return false; + } + return true; + } + + final void activate(Collection peers) + { + Preconditions.checkState(!peers.isEmpty()); + logger.debug("{} Activating {} on {}", logPrefix(), this, peers); + LocalTransfers.instance().activating(this); + + // First phase ensures data is present on disk, then second phase does the actual import. This ensures that if + // something goes wrong (like a topology change during import), we don't have divergence. + class Prepare extends AsyncFuture implements RequestCallbackWithFailure + { + final Set responses = ConcurrentHashMap.newKeySet(); + + public Prepare() + { + responses.addAll(peers); + } + + @Override + public void onResponse(Message msg) + { + logger.debug("{} Got prepare response from: {}", logPrefix(), msg.from()); + responses.remove(msg.from()); + if (responses.isEmpty()) + trySuccess(null); + } + + @Override + public void onFailure(InetAddressAndPort from, RequestFailure failure) + { + logger.debug("{} Got prepare failure {} from {}", logPrefix(), failure, from); + AbstractCoordinatedBulkTransfer.this.streamResults.computeIfPresent(from, (peer, result) -> result.prepareFailed()); + tryFailure(new RuntimeException("Tracked import failed during PREPARE on " + from + " due to " + failure.reason)); + } + } + + Prepare prepare = new Prepare(); + for (InetAddressAndPort peer : peers) + { + TransferActivation activation = TransferActivation.create(this, peer, Phase.PREPARE); + Message msg = Message.out(Verb.TRACKED_TRANSFER_ACTIVATE_REQ, activation); + logger.debug("{} Sending prepare {} to peer {}", logPrefix(), activation, peer); + MessagingService.instance().sendWithCallback(msg, peer, prepare); + AbstractCoordinatedBulkTransfer.this.streamResults.computeIfPresent(peer, (peer0, result) -> result.preparing()); + } + try + { + prepare.get(); + } + catch (InterruptedException | ExecutionException e) + { + Throwable cause = e instanceof ExecutionException ? e.getCause() : e; + throw Throwables.unchecked(cause); + } + logger.debug("{} Activation prepare complete for {}", logPrefix(), peers); + + // Acknowledgement of activation is equivalent to a remote write acknowledgement. The imported SSTables + // are now part of the live set, visible to reads. + class Commit extends AsyncFuture implements RequestCallbackWithFailure + { + final Set responses = ConcurrentHashMap.newKeySet(); + + private Commit(Collection peers) + { + responses.addAll(peers); + } + + @Override + public void onResponse(Message msg) + { + logger.debug("{} Activation successfully applied on {}", logPrefix(), msg.from()); + AbstractCoordinatedBulkTransfer.this.streamResults.computeIfPresent(msg.from(), (peer, result) -> result.committed()); + + MutationTrackingService.instance.receivedActivationResponse(AbstractCoordinatedBulkTransfer.this, msg.from()); + responses.remove(msg.from()); + if (responses.isEmpty()) + { + // All activations complete, schedule cleanup to purge pending SSTables + LocalTransfers.instance().scheduleCleanup(); + trySuccess(null); + } + } + + @Override + public void onFailure(InetAddressAndPort from, RequestFailure failure) + { + logger.error("{} Failed activation commit on {} due to {}", logPrefix(), from, failure); + MutationTrackingService.instance.retryFailedTransfer(AbstractCoordinatedBulkTransfer.this, from, failure.failure); + // TODO(expected): should only fail if we don't meet requested CL + tryFailure(new RuntimeException("Tracked import failed during COMMIT on " + from + " due to " + failure.reason)); + } + } + + Commit commit = new Commit(peers); + for (InetAddressAndPort peer : peers) + { + TransferActivation activation = TransferActivation.create(this, peer, Phase.COMMIT); + Message msg = Message.out(Verb.TRACKED_TRANSFER_ACTIVATE_REQ, activation); + + logger.debug("{} Sending commit {} to peer {}", logPrefix(), activation, peer); + MessagingService.instance().sendWithCallback(msg, peer, commit); + AbstractCoordinatedBulkTransfer.this.streamResults.computeIfPresent(peer, (peer0, result) -> result.committing()); + } + + try + { + commit.get(); + } + catch (InterruptedException | ExecutionException e) + { + Throwable cause = e instanceof ExecutionException ? e.getCause() : e; + throw Throwables.unchecked(cause); + } + logger.debug("{} Activation commit complete for {}", logPrefix(), peers); + } + + /** + * Notify all replicas that this transfer failed, triggering cleanup of pending SSTables. + * This is used by both TrackedImportTransfer and TrackedRepairSyncTransfer. + */ + protected void notifyFailure() throws ExecutionException, InterruptedException + { + class NotifyFailure extends AsyncFuture implements RequestCallbackWithFailure + { + final Set responses = ConcurrentHashMap.newKeySet(streamResults.size()); + + @Override + public void onResponse(Message msg) + { + responses.remove(msg.from()); + if (responses.isEmpty()) + trySuccess(null); + } + + @Override + public void onFailure(InetAddressAndPort from, RequestFailure failure) + { + // Log but don't fail - best effort cleanup + logger.warn("{} Failed to notify {} of transfer failure: {}", logPrefix(), from, failure); + responses.remove(from); + if (responses.isEmpty()) + trySuccess(null); + } + } + + NotifyFailure notifyFailure = new NotifyFailure(); + for (Map.Entry entry : streamResults.entrySet()) + { + InetAddressAndPort to = entry.getKey(); + // Coordinator cleans up CoordinatedTransfer and PendingLocalTransfer separately, does not need to notify + if (FBUtilities.getBroadcastAddressAndPort().equals(to)) + continue; + + SingleTransferResult result = entry.getValue(); + if (result.planId() == null) + { + // No planId means streaming never completed, so there's nothing to clean up on the replica + logger.debug("{} Skipping notification of transfer failure to {} - no planId", logPrefix(), to); + continue; + } + + logger.debug("{} Notifying {} of transfer failure for plan {}", logPrefix(), to, result.planId()); + notifyFailure.responses.add(to); + Message msg = Message.out(Verb.TRACKED_TRANSFER_FAILED_REQ, new TransferFailed(result.planId())); + MessagingService.instance().sendWithCallback(msg, to, notifyFailure); + } + + // Only wait if we actually sent notifications + if (!notifyFailure.responses.isEmpty()) + notifyFailure.get(); + } + + /** + * Tracks the lifecycle of a transfer from the coordinator to a single replica, using a two-phase commit protocol: + * + *
    + *
  • {@link State#INIT}: Transfer created, not yet streaming.
  • + *
  • {@link State#STREAM_COMPLETE}: Streaming successful, SSTables received on replica in pending directory.
  • + *
  • {@link State#STREAM_NOOP}: No data streamed (e.g., SSTable contains no rows in target range).
  • + *
  • {@link State#STREAM_FAILED}: Streaming failed, may not have a streaming plan ID yet.
  • + *
  • {@link State#PREPARING}: Preparing for activation (first phase).
  • + *
  • {@link State#PREPARE_FAILED}: Prepare failed, aborting transfer.
  • + *
  • {@link State#COMMITTING}: Committing transferred SSTables from pending to live set (second phase).
  • + *
  • {@link State#COMMITTED}: Transfer commit acknowledged on coordinator. SSTables now live and visible to reads.
  • + *
+ * + *

Valid State Transitions:

+ *
+     *                                       ┌────────────────┐
+     *                                       ↓                │
+     *   INIT ──┬──→ STREAM_COMPLETE ──→ PREPARING ──┬──→ COMMITTING ──→ COMMITTED
+     *          │                                    │
+     *          ├──→ STREAM_NOOP                     └──→ PREPARE_FAILED
+     *          │
+     *          └──→ STREAM_FAILED
+     * 
+ * + * Failure states may be non-terminal if sufficient replicas reach successful states, depending on the transfer's + * consistency level. + */ + static class SingleTransferResult + { + enum State + { + INIT, + STREAM_NOOP, + STREAM_FAILED, + STREAM_COMPLETE, + PREPARING, + PREPARE_FAILED, + COMMITTING, + COMMITTED; + + EnumSet transitionFrom; + + static + { + INIT.transitionFrom = EnumSet.noneOf(State.class); + STREAM_NOOP.transitionFrom = EnumSet.of(INIT); + STREAM_FAILED.transitionFrom = EnumSet.of(INIT); + STREAM_COMPLETE.transitionFrom = EnumSet.of(INIT); + PREPARING.transitionFrom = EnumSet.of(STREAM_COMPLETE, COMMITTING); + PREPARE_FAILED.transitionFrom = EnumSet.of(PREPARING); + COMMITTING.transitionFrom = EnumSet.of(PREPARING); + COMMITTED.transitionFrom = EnumSet.of(COMMITTING); + } + } + + final State state; + private final TimeUUID planId; + + @VisibleForTesting + SingleTransferResult(State state, TimeUUID planId) + { + this.state = state; + this.planId = planId; + } + + private boolean canTransition(State to) + { + return to.transitionFrom.contains(state); + } + + public static SingleTransferResult Init() + { + return new SingleTransferResult(State.INIT, null); + } + + @VisibleForTesting + static SingleTransferResult StreamComplete(TimeUUID planId) + { + return new SingleTransferResult(STREAM_COMPLETE, planId); + } + + @VisibleForTesting + static SingleTransferResult Noop() + { + return new SingleTransferResult(State.STREAM_NOOP, null); + } + + @CheckReturnValue + private SingleTransferResult transition(State to, TimeUUID planId) + { + if (!canTransition(to)) + { + logger.error("Ignoring invalid transition from {} to {}", state, to); + return this; + } + // Don't overwrite if the stream succeeded but PREPARE failed, so we can clean up later + return new SingleTransferResult(to, planId == null ? this.planId : planId); + } + + @CheckReturnValue + public SingleTransferResult streamFailed(TimeUUID planId) + { + return transition(State.STREAM_FAILED, planId); + } + + @CheckReturnValue + public SingleTransferResult preparing() + { + return transition(State.PREPARING, this.planId); + } + + @CheckReturnValue + public SingleTransferResult prepareFailed() + { + return transition(State.PREPARE_FAILED, this.planId); + } + + @CheckReturnValue + public SingleTransferResult committing() + { + return transition(State.COMMITTING, this.planId); + } + + @CheckReturnValue + public SingleTransferResult committed() + { + return transition(State.COMMITTED, this.planId); + } + + public TimeUUID planId() + { + return planId; + } + + @Override + public String toString() + { + return "SingleTransferResult{" + + "state=" + state + + ", planId=" + planId + + '}'; + } + } +} diff --git a/src/java/org/apache/cassandra/replication/ActiveLogReconciler.java b/src/java/org/apache/cassandra/replication/ActiveLogReconciler.java index e7d5504e9070..29ae16e82828 100644 --- a/src/java/org/apache/cassandra/replication/ActiveLogReconciler.java +++ b/src/java/org/apache/cassandra/replication/ActiveLogReconciler.java @@ -132,7 +132,7 @@ private static abstract class Task implements RequestCallback { private static Task from(ShortMutationId id, InetAddressAndPort toHost) { - CoordinatedTransfer transfer = LocalTransfers.instance().getActivatedTransfer(id); + AbstractCoordinatedBulkTransfer transfer = LocalTransfers.instance().getActivatedTransfer(id); if (transfer != null) return new TransferTask(transfer, toHost); else @@ -209,10 +209,10 @@ void send() private static final class TransferTask extends Task { - private final CoordinatedTransfer transfer; + private final AbstractCoordinatedBulkTransfer transfer; private final InetAddressAndPort toHost; - TransferTask(CoordinatedTransfer transfer, InetAddressAndPort toHost) + TransferTask(AbstractCoordinatedBulkTransfer transfer, InetAddressAndPort toHost) { this.transfer = transfer; this.toHost = toHost; @@ -249,7 +249,7 @@ void send() LocalTransfers.instance().executor.submit(() -> { try { - transfer.activateOn(Collections.singleton(toHost)); + transfer.activate(Collections.singleton(toHost)); onResponse(null); } catch (Throwable t) diff --git a/src/java/org/apache/cassandra/replication/CoordinatedTransfer.java b/src/java/org/apache/cassandra/replication/CoordinatedTransfer.java deleted file mode 100644 index d82b27c9b270..000000000000 --- a/src/java/org/apache/cassandra/replication/CoordinatedTransfer.java +++ /dev/null @@ -1,720 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.replication; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.EnumSet; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.function.Supplier; - -import javax.annotation.CheckReturnValue; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.ConsistencyLevel; -import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.streaming.CassandraOutgoingFile; -import org.apache.cassandra.dht.Range; -import org.apache.cassandra.dht.Token; -import org.apache.cassandra.exceptions.RequestFailure; -import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.locator.AbstractReplicationStrategy; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.Message; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.NoPayload; -import org.apache.cassandra.net.RequestCallbackWithFailure; -import org.apache.cassandra.net.Verb; -import org.apache.cassandra.streaming.OutgoingStream; -import org.apache.cassandra.streaming.StreamException; -import org.apache.cassandra.streaming.StreamOperation; -import org.apache.cassandra.streaming.StreamPlan; -import org.apache.cassandra.streaming.StreamResultFuture; -import org.apache.cassandra.streaming.StreamState; -import org.apache.cassandra.tcm.ClusterMetadata; -import org.apache.cassandra.tcm.membership.NodeId; -import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.Throwables; -import org.apache.cassandra.utils.TimeUUID; -import org.apache.cassandra.utils.concurrent.AsyncFuture; -import org.apache.cassandra.utils.concurrent.Future; - -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.COMMITTED; -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.COMMITTING; -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.PREPARE_FAILED; -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.PREPARING; -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.INIT; -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.STREAM_COMPLETE; -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.STREAM_FAILED; -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.STREAM_NOOP; -import static org.apache.cassandra.replication.TransferActivation.Phase; - -/** - * Orchestrates the lifecycle of a tracked bulk data transfer for a single replica set, where the current instance is - * coordinating the transfer. - *

- * The transfer proceeds through these phases: - *

    - *
  1. - * Streaming - * The coordinator streams SSTables to all replicas in parallel. Replicas store received data in a "pending" - * location where it's persisted to disk but not yet visible to reads. Once sufficient replicas have received - * their streams to meet the requested {@link ConsistencyLevel}, the SSTables are activated using a two-phase - * commit protocol, making them part of the live set and visible to reads. - *
  2. - *
  3. - * Activation {@link Phase#PREPARE} - * The coordinator sends PREPARE messages to verify replicas have the data persisted on disk and are ready for - * activation. - *
  4. - *
  5. - * Activation {@link Phase#COMMIT} - * After successful PREPARE, the coordinator sends COMMIT messages to replicas. Replicas atomically move data from - * pending to live sets, making it visible to reads with the proper transfer ID in metadata. If commit succeeds - * on some replicas but not others, the transfer will be activated later on via existing the existing - * reconciliation processes (read reconciliation and background reconciliation). - *
  6. - *
- * - * For simplicity, the coordinator streams to itself rather than using direct file copy. This ensures we can use the - * same lifecycle management for crash-safety and atomic add. - *

- * If a tracked data read is executed on a replica that's missing an activation, the read reconciliation process will - * apply the missing activation during reconciliation and a subsequent read will succeed. To minimize the gap between - * activations across replicas, avoid expensive operations like file copies or index builds during - * {@link TransferActivation#apply()}. - */ -public class CoordinatedTransfer -{ - private static final Logger logger = LoggerFactory.getLogger(CoordinatedTransfer.class); - - String logPrefix() - { - return String.format("[CoordinatedTransfer #%s]", id); - } - - private final ShortMutationId id; - private final String keyspace; - private final Range range; - private final ConsistencyLevel cl; - final Collection sstables; - final ConcurrentMap streamResults; - - @VisibleForTesting - CoordinatedTransfer(Range range, MutationId id) - { - this.keyspace = null; - this.range = range; - this.sstables = Collections.emptyList(); - this.cl = null; - this.id = id; - this.streamResults = new ConcurrentHashMap<>(); - } - - CoordinatedTransfer(String keyspace, Range range, Participants participants, Collection sstables, ConsistencyLevel cl, Supplier nextId) - { - this.keyspace = keyspace; - this.range = range; - this.sstables = sstables; - this.cl = cl; - this.id = nextId.get(); - - ClusterMetadata cm = ClusterMetadata.current(); - this.streamResults = new ConcurrentHashMap<>(participants.size()); - for (int i = 0; i < participants.size(); i++) - { - InetAddressAndPort addr = cm.directory.getNodeAddresses(new NodeId(participants.get(i))).broadcastAddress; - this.streamResults.put(addr, SingleTransferResult.Init()); - } - } - - ShortMutationId id() - { - return id; - } - - void execute() - { - logger.debug("{} Executing tracked bulk transfer {}", logPrefix(), this); - LocalTransfers.instance().save(this); - stream(); - } - - private void stream() - { - // TODO: Don't stream multiple copies over the WAN, send one copy and indicate forwarding - List> streaming = new ArrayList<>(streamResults.size()); - for (InetAddressAndPort to : streamResults.keySet()) - { - Future stream = LocalTransfers.instance().executor.submit(() -> { - stream(to); - return null; - }); - streaming.add(stream); - } - - // Wait for all streams to complete, so we can clean up after failures. If we exit at the first failure, a - // future stream can complete. - LinkedList failures = null; - for (Future stream : streaming) - { - try - { - stream.get(); - } - catch (InterruptedException | ExecutionException e) - { - if (failures == null) - failures = new LinkedList<>(); - failures.add(e); - logger.error("{} Failed transfer due to", logPrefix(), e); - } - } - - if (failures != null && !failures.isEmpty()) - { - Throwable failure = failures.element(); - Throwable cause = failure instanceof ExecutionException ? failure.getCause() : failure; - maybeCleanupFailedStreams(cause); - - String msg = String.format("Failed streaming on %s instance(s): %s", failures.size(), failures); - throw new RuntimeException(msg, Throwables.unchecked(cause)); - } - - logger.info("{} All streaming completed successfully", logPrefix()); - } - - private boolean sufficient() - { - AbstractReplicationStrategy ars = Keyspace.open(keyspace).getReplicationStrategy(); - int blockFor = cl.blockFor(ars); - int responses = 0; - for (Map.Entry entry : streamResults.entrySet()) - { - if (entry.getValue().state == STREAM_COMPLETE) - responses++; - } - return responses >= blockFor; - } - - void stream(InetAddressAndPort to) - { - SingleTransferResult result; - try - { - result = streamTask(to); - } - catch (StreamException | ExecutionException | InterruptedException | TimeoutException e) - { - Throwable cause = e instanceof ExecutionException ? e.getCause() : e; - markStreamFailure(to, cause); - throw Throwables.unchecked(cause); - } - - try - { - streamComplete(to, result); - } - catch (ExecutionException | InterruptedException | TimeoutException e) - { - Throwable cause = e instanceof ExecutionException ? e.getCause() : e; - throw Throwables.unchecked(cause); - } - } - - private void notifyFailure() throws ExecutionException, InterruptedException - { - class NotifyFailure extends AsyncFuture implements RequestCallbackWithFailure - { - final Set responses = ConcurrentHashMap.newKeySet(streamResults.size()); - - @Override - public void onResponse(Message msg) - { - responses.remove(msg.from()); - if (responses.isEmpty()) - trySuccess(null); - } - - @Override - public void onFailure(InetAddressAndPort from, RequestFailure failure) - { - tryFailure(failure.failure); - } - } - - NotifyFailure notifyFailure = new NotifyFailure(); - for (Map.Entry entry : streamResults.entrySet()) - { - InetAddressAndPort to = entry.getKey(); - // Coordinator cleans up CoordinatedTransfer and PendingLocalTransfer separately, does not need to notify - if (FBUtilities.getBroadcastAddressAndPort().equals(to)) - continue; - - SingleTransferResult result = entry.getValue(); - if (result.planId == null) - { - logger.warn("{} Skipping notification of transfer failure to {} due to unknown planId", logPrefix(), to); - continue; - } - - logger.debug("{}, Notifying {} of transfer failure for plan {}", logPrefix(), to, result.planId); - notifyFailure.responses.add(to); - Message msg = Message.out(Verb.TRACKED_TRANSFER_FAILED_REQ, new TransferFailed(result.planId)); - MessagingService.instance().sendWithCallback(msg, to, notifyFailure); - } - notifyFailure.get(); - } - - private void markStreamFailure(InetAddressAndPort to, Throwable cause) - { - TimeUUID planId; - if (cause instanceof StreamException) - planId = ((StreamException) cause).finalState.planId; - else - planId = null; - streamResults.computeIfPresent(to, (peer, result) -> result.streamFailed(planId)); - } - - /** - * This shouldn't throw an exception, even if we fail to notify peers of the streaming failure. - */ - private void maybeCleanupFailedStreams(Throwable cause) - { - try - { - boolean purgeable = LocalTransfers.instance().purger.test(this); - if (!purgeable) - return; - - notifyFailure(); - LocalTransfers.instance().scheduleCleanup(); - } - catch (Throwable t) - { - if (cause != null) - t.addSuppressed(cause); - logger.error("{} Failed to notify peers of stream failure", logPrefix(), t); - } - } - - private void streamComplete(InetAddressAndPort to, SingleTransferResult result) throws ExecutionException, InterruptedException, TimeoutException - { - streamResults.put(to, result); - logger.info("{} Completed streaming to {}, {}", logPrefix(), to, this); - maybeActivate(); - } - - synchronized void maybeActivate() - { - // If any activations have already been sent out, send new activations to any received plans that have not yet - // been activated - boolean anyActivated = false; - Set awaitingActivation = new HashSet<>(); - for (Map.Entry entry : streamResults.entrySet()) - { - InetAddressAndPort peer = entry.getKey(); - SingleTransferResult result = entry.getValue(); - if (result.state == COMMITTING || result.state == COMMITTED) - { - anyActivated = true; - } - else if (result.state == STREAM_COMPLETE) - awaitingActivation.add(peer); - } - if (anyActivated && !awaitingActivation.isEmpty()) - { - logger.debug("{} Transfer already activated on some peers, sending activations to remaining: {}", logPrefix(), awaitingActivation); - activateOn(awaitingActivation); - return; - } - // If no activations have been sent out, check whether we have enough planIds back to meet the required CL - else if (sufficient()) - { - Set peers = new HashSet<>(); - for (Map.Entry entry : streamResults.entrySet()) - { - InetAddressAndPort peer = entry.getKey(); - SingleTransferResult result = entry.getValue(); - if (result.state == STREAM_COMPLETE) - peers.add(peer); - } - logger.debug("{} Transfer meets consistency level {}, sending activations to {}", logPrefix(), cl, peers); - activateOn(peers); - return; - } - - logger.debug("{} Nothing to activate", logPrefix()); - } - - void activateOn(Collection peers) - { - Preconditions.checkState(!peers.isEmpty()); - logger.debug("{} Activating {} on {}", logPrefix(), this, peers); - LocalTransfers.instance().activating(this); - - // First phase ensures data is present on disk, then second phase does the actual import. This ensures that if - // something goes wrong (like a topology change during import), we don't have divergence. - class Prepare extends AsyncFuture implements RequestCallbackWithFailure - { - final Set responses = ConcurrentHashMap.newKeySet(); - - public Prepare() - { - responses.addAll(peers); - } - - @Override - public void onResponse(Message msg) - { - logger.debug("{} Got response from: {}", logPrefix(), msg.from()); - responses.remove(msg.from()); - if (responses.isEmpty()) - trySuccess(null); - } - - @Override - public void onFailure(InetAddressAndPort from, RequestFailure failure) - { - logger.debug("{} Got failure {} from {}", logPrefix(), failure, from); - CoordinatedTransfer.this.streamResults.computeIfPresent(from, (peer, result) -> result.prepareFailed()); - tryFailure(new RuntimeException("Tracked import failed during PREPARE on " + from + " due to " + failure.reason)); - } - } - - Prepare prepare = new Prepare(); - for (InetAddressAndPort peer : peers) - { - TransferActivation activation = new TransferActivation(this, peer, Phase.PREPARE); - Message msg = Message.out(Verb.TRACKED_TRANSFER_ACTIVATE_REQ, activation); - logger.debug("{} Sending {} to peer {}", logPrefix(), activation, peer); - MessagingService.instance().sendWithCallback(msg, peer, prepare); - CoordinatedTransfer.this.streamResults.computeIfPresent(peer, (peer0, result) -> result.preparing()); - } - try - { - prepare.get(); - } - catch (InterruptedException | ExecutionException e) - { - Throwable cause = e instanceof ExecutionException ? e.getCause() : e; - throw Throwables.unchecked(cause); - } - logger.debug("{} Activation prepare complete for {}", logPrefix(), peers); - - // Acknowledgement of activation is equivalent to a remote write acknowledgement. The imported SSTables - // are now part of the live set, visible to reads. - class Commit extends AsyncFuture implements RequestCallbackWithFailure - { - final Set responses = ConcurrentHashMap.newKeySet(); - - private Commit(Collection peers) - { - responses.addAll(peers); - } - - @Override - public void onResponse(Message msg) - { - logger.debug("{} Activation successfully applied on {}", logPrefix(), msg.from()); - CoordinatedTransfer.this.streamResults.computeIfPresent(msg.from(), (peer, result) -> result.committed()); - - MutationTrackingService.instance.receivedActivationResponse(CoordinatedTransfer.this, msg.from()); - responses.remove(msg.from()); - if (responses.isEmpty()) - { - // All activations complete, schedule cleanup to purge pending SSTables - LocalTransfers.instance().scheduleCleanup(); - trySuccess(null); - } - } - - @Override - public void onFailure(InetAddressAndPort from, RequestFailure failure) - { - logger.error("{} Failed activation on {} due to {}", logPrefix(), from, failure); - MutationTrackingService.instance.retryFailedTransfer(CoordinatedTransfer.this, from, failure.failure); - // TODO(expected): should only fail if we don't meet requested CL - tryFailure(new RuntimeException("Tracked import failed during COMMIT on " + from + " due to " + failure.reason)); - } - } - - Commit commit = new Commit(peers); - for (InetAddressAndPort peer : peers) - { - TransferActivation activation = new TransferActivation(this, peer, Phase.COMMIT); - Message msg = Message.out(Verb.TRACKED_TRANSFER_ACTIVATE_REQ, activation); - - logger.debug("{} Sending {} to peer {}", logPrefix(), activation, peer); - MessagingService.instance().sendWithCallback(msg, peer, commit); - CoordinatedTransfer.this.streamResults.computeIfPresent(peer, (peer0, result) -> result.committing()); - } - - try - { - commit.get(); - } - catch (InterruptedException | ExecutionException e) - { - Throwable cause = e instanceof ExecutionException ? e.getCause() : e; - throw Throwables.unchecked(cause); - } - logger.debug("{} Activation commit complete for {}", logPrefix(), peers); - } - - public boolean isCommitted() - { - for (SingleTransferResult result : streamResults.values()) - { - if (result.state != COMMITTED) - return false; - } - return true; - } - - /** - * Tracks the lifecycle of a transfer from the coordinator to a single replica, using a two-phase commit protocol: - * - *

    - *
  • {@link State#INIT}: Transfer created, not yet streaming.
  • - *
  • {@link State#STREAM_COMPLETE}: Streaming successful, SSTables received on replica in pending directory.
  • - *
  • {@link State#STREAM_NOOP}: No data streamed (e.g., SSTable contains no rows in target range).
  • - *
  • {@link State#STREAM_FAILED}: Streaming failed, may not have a streaming plan ID yet.
  • - *
  • {@link State#PREPARING}: Preparing for activation (first phase).
  • - *
  • {@link State#PREPARE_FAILED}: Prepare failed, aborting transfer.
  • - *
  • {@link State#COMMITTING}: Committing transferred SSTables from pending to live set (second phase).
  • - *
  • {@link State#COMMITTED}: Transfer commit acknowledged on coordinator. SSTables now live and visible to reads.
  • - *
- * - *

Valid State Transitions:

- *
-     *                                       ┌────────────────┐
-     *                                       ↓                │
-     *   INIT ──┬──→ STREAM_COMPLETE ──→ PREPARING ──┬──→ COMMITTING ──→ COMMITTED
-     *          │                                    │
-     *          ├──→ STREAM_NOOP                     └──→ PREPARE_FAILED
-     *          │
-     *          └──→ STREAM_FAILED
-     * 
- * - * Failure states may be non-terminal if sufficient replicas reach successful states, depending on the transfer's - * consistency level. - */ - static class SingleTransferResult - { - enum State - { - INIT, - STREAM_NOOP, - STREAM_FAILED, - STREAM_COMPLETE, - PREPARING, - PREPARE_FAILED, - COMMITTING, - COMMITTED; - - EnumSet transitionFrom; - - static - { - INIT.transitionFrom = EnumSet.noneOf(State.class); - STREAM_NOOP.transitionFrom = EnumSet.of(INIT); - STREAM_FAILED.transitionFrom = EnumSet.of(INIT); - STREAM_COMPLETE.transitionFrom = EnumSet.of(INIT); - PREPARING.transitionFrom = EnumSet.of(STREAM_COMPLETE, COMMITTING); - PREPARE_FAILED.transitionFrom = EnumSet.of(PREPARING); - COMMITTING.transitionFrom = EnumSet.of(PREPARING); - COMMITTED.transitionFrom = EnumSet.of(COMMITTING); - } - } - - final State state; - private final TimeUUID planId; - - @VisibleForTesting - SingleTransferResult(State state, TimeUUID planId) - { - this.state = state; - this.planId = planId; - } - - private boolean canTransition(SingleTransferResult.State to) - { - return to.transitionFrom.contains(state); - } - - public static SingleTransferResult Init() - { - return new SingleTransferResult(INIT, null); - } - - @VisibleForTesting - static SingleTransferResult StreamComplete(TimeUUID planId) - { - return new SingleTransferResult(STREAM_COMPLETE, planId); - } - - @VisibleForTesting - static SingleTransferResult Noop() - { - return new SingleTransferResult(STREAM_NOOP, null); - } - - @CheckReturnValue - private SingleTransferResult transition(State to, TimeUUID planId) - { - if (!canTransition(to)) - { - logger.error("Ignoring invalid transition from {} to {}", state, to); - return this; - } - // Don't overwrite if the stream succeeded but PREPARE failed, so we can clean up later - return new SingleTransferResult(to, planId == null ? this.planId : planId); - } - - @CheckReturnValue - public SingleTransferResult streamFailed(TimeUUID planId) - { - return transition(STREAM_FAILED, planId); - } - - @CheckReturnValue - public SingleTransferResult preparing() - { - return transition(PREPARING, this.planId); - } - - @CheckReturnValue - public SingleTransferResult prepareFailed() - { - return transition(PREPARE_FAILED, this.planId); - } - - @CheckReturnValue - public SingleTransferResult committing() - { - return transition(COMMITTING, this.planId); - } - - @CheckReturnValue - public SingleTransferResult committed() - { - return transition(COMMITTED, this.planId); - } - - public TimeUUID planId() - { - return planId; - } - - @Override - public String toString() - { - return "SingleTransferResult{" + - "state=" + state + - ", planId=" + planId + - '}'; - } - } - - private SingleTransferResult streamTask(InetAddressAndPort to) throws StreamException, ExecutionException, InterruptedException, TimeoutException - { - StreamPlan plan = new StreamPlan(StreamOperation.TRACKED_TRANSFER); - - // No need to flush, only using non-live SSTables already on disk - plan.flushBeforeTransfer(false); - - for (SSTableReader sstable : sstables) - { - List> ranges = Collections.singletonList(range); - List positions = sstable.getPositionsForRanges(ranges); - long estimatedKeys = sstable.estimatedKeysForRanges(ranges); - OutgoingStream stream = new CassandraOutgoingFile(StreamOperation.TRACKED_TRANSFER, sstable.ref(), positions, ranges, estimatedKeys); - plan.transferStreams(to, Collections.singleton(stream)); - } - - long timeout = DatabaseDescriptor.getStreamTransferTaskTimeout().toMilliseconds(); - - logger.info("{} Starting streaming transfer {} to peer {}", logPrefix(), this, to); - StreamResultFuture execute = plan.execute(); - StreamState state; - try - { - state = execute.get(timeout, TimeUnit.MILLISECONDS); - logger.debug("{} Completed streaming transfer {} to peer {}", logPrefix(), this, to); - } - catch (InterruptedException | ExecutionException | TimeoutException e) - { - logger.error("Stream session failed with error", e); - throw e; - } - - if (state.hasFailedSession() || state.hasAbortedSession()) - throw new StreamException(state, "Stream failed due to failed or aborted sessions"); - - // If the SSTable doesn't contain any rows in the provided range, no streams delivered, nothing to activate - if (state.sessions().isEmpty()) - return SingleTransferResult.Noop(); - - return SingleTransferResult.StreamComplete(plan.planId()); - } - - @Override - public boolean equals(Object o) - { - if (o == null || getClass() != o.getClass()) return false; - CoordinatedTransfer transfer = (CoordinatedTransfer) o; - return Objects.equals(keyspace, transfer.keyspace) && Objects.equals(range, transfer.range) && Objects.equals(streamResults, transfer.streamResults) && Objects.equals(sstables, transfer.sstables) && cl == transfer.cl && Objects.equals(id, transfer.id); - } - - @Override - public int hashCode() - { - return Objects.hash(keyspace, range, streamResults, sstables, cl, id); - } - - @Override - public String toString() - { - return "CoordinatedTransfer{" + - "id=" + id + - ", keyspace='" + keyspace + '\'' + - ", range=" + range + - ", cl=" + cl + - ", sstables=" + sstables + - ", streamResults=" + streamResults + - '}'; - } -} diff --git a/src/java/org/apache/cassandra/replication/CoordinatorLog.java b/src/java/org/apache/cassandra/replication/CoordinatorLog.java index 56180c25945a..f259472fc411 100644 --- a/src/java/org/apache/cassandra/replication/CoordinatorLog.java +++ b/src/java/org/apache/cassandra/replication/CoordinatorLog.java @@ -392,7 +392,7 @@ void finishActivation(PendingLocalTransfer transfer, TransferActivation activati - On transfer coordinators after they've received a completed activation from a peer (onHostId != me) - On local replicas after coordinators have propagated their replicated offsets */ - void receivedActivationResponse(CoordinatedTransfer transfer, int onHostId) + void receivedActivationResponse(AbstractCoordinatedBulkTransfer transfer, int onHostId) { ShortMutationId transferId = transfer.id(); Preconditions.checkArgument(!transferId.isNone()); diff --git a/src/java/org/apache/cassandra/replication/LocalTransfers.java b/src/java/org/apache/cassandra/replication/LocalTransfers.java index 876bbeb47ebe..ee3428dca53a 100644 --- a/src/java/org/apache/cassandra/replication/LocalTransfers.java +++ b/src/java/org/apache/cassandra/replication/LocalTransfers.java @@ -18,60 +18,79 @@ package org.apache.cassandra.replication; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import javax.annotation.Nullable; import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.FutureCallback; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.concurrent.ExecutorPlus; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.File; +import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.IVerbHandler; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.NoPayload; +import org.apache.cassandra.repair.RepairJob; +import org.apache.cassandra.repair.RepairJobDesc; +import org.apache.cassandra.repair.SyncStat; +import org.apache.cassandra.repair.SyncTask; +import org.apache.cassandra.repair.SyncTasks; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.TimeUUID; +import org.apache.cassandra.utils.concurrent.AsyncPromise; +import org.apache.cassandra.utils.concurrent.Future; import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; /** * Singleton registry maintaining state for bulk data transfers on the local node. *

- * This includes {@link CoordinatedTransfer} instances that the current node is coordinating, and + * This includes {@link TrackedImportTransfer} instances that the current node is coordinating, and * {@link PendingLocalTransfer} instances that are coordinated by other nodes. Pending transfers are inactive until * activated by the coordinator. *

* TODO: Make changes to pending set durable with SystemKeyspace.savePendingLocalTransfer(transfer)? * TODO: Add vtable for visibility into local and coordinated transfers + * TODO: Rename this to TrackedTransferService? */ public class LocalTransfers { private static final Logger logger = LoggerFactory.getLogger(LocalTransfers.class); private final ReadWriteLock lock = new ReentrantReadWriteLock(); - private final Map coordinating = new ConcurrentHashMap<>(); + private final Map coordinating = new ConcurrentHashMap<>(); private final Map local = new ConcurrentHashMap<>(); final ExecutorPlus executor = executorFactory().pooled("LocalTrackedTransfers", Integer.MAX_VALUE); private static final LocalTransfers instance = new LocalTransfers(); - static LocalTransfers instance() + public static LocalTransfers instance() { return instance; } - void save(CoordinatedTransfer transfer) + void save(TrackedImportTransfer transfer) { lock.writeLock().lock(); try { - CoordinatedTransfer existing = coordinating.put(transfer.id(), transfer); + AbstractCoordinatedBulkTransfer existing = coordinating.put(transfer.id(), transfer); Preconditions.checkState(existing == null); } finally @@ -80,7 +99,7 @@ void save(CoordinatedTransfer transfer) } } - void activating(CoordinatedTransfer transfer) + void activating(AbstractCoordinatedBulkTransfer transfer) { Preconditions.checkNotNull(transfer.id()); lock.writeLock().lock(); @@ -111,6 +130,165 @@ void received(PendingLocalTransfer transfer) } } + /** + * We should track the repair as a CoordinatedLocalTransfer so when the sync is done we can either TransferActivation + * or TransferFailed. + * + * Track before any of the sync tasks execute because we need to send {@link TransferFailed} to all replicas if a + * failure happens. + */ + public void onRepairSyncExecution(RepairJob job, RepairJobDesc desc, SyncTasks tasks) + { + // One RepairJob may have multiple TrackedRepairSyncTransfers, if it spans across shards. + // Group tasks by their transfer ID and create one TrackedRepairSyncTransfer per unique ID. + // Create and register a TrackedRepairSyncTransfer for each unique transfer ID + lock.writeLock().lock(); + try + { + tasks.forEach((ShortMutationId id, SyncTasks.Entry entry) -> { + TrackedRepairSyncTransfer transfer = new TrackedRepairSyncTransfer(id, entry.participants, entry.tasks); + logger.debug("{} Saving {}", transfer.logPrefix(), transfer); + coordinating.put(id, transfer); + }); + } + finally + { + lock.writeLock().unlock(); + } + } + + /** + * Begin activation for the sync'd transfer(s) + */ + public Future> onRepairSyncCompletion(RepairJob job, Future> syncCompletion, Executor executor) + { + AsyncPromise> activationFuture = new AsyncPromise<>(); + + syncCompletion.addCallback(new FutureCallback>() + { + @Override + public void onSuccess(List syncs) + { + Map> syncsByTransferId; + Map transfersToActivate = new HashMap<>(); + + lock.writeLock().lock(); + try + { + logger.info("maybeActivate onSuccess {} {} {}", syncs, coordinating, local); + + // Collect all unique transfer IDs from the job's sync tasks + syncsByTransferId = new HashMap<>(); + + // Build a map of sync task ranges to their transfer IDs for lookup + Map>, ShortMutationId> rangeToTransferId = new HashMap<>(); + for (SyncTask task : job.getSyncTasks()) + { + ShortMutationId transferId = task.getTransferId(); + if (transferId != null) + rangeToTransferId.put(task.rangesToSync, transferId); + } + + // Group sync stats by transfer ID based on their ranges + for (SyncStat sync : syncs) + { + ShortMutationId transferId = rangeToTransferId.get(sync.differences); + if (transferId != null) + syncsByTransferId.computeIfAbsent(transferId, k -> new ArrayList<>()).add(sync); + } + + // Look up transfers while holding the lock + for (Map.Entry> entry : syncsByTransferId.entrySet()) + { + ShortMutationId transferId = entry.getKey(); + AbstractCoordinatedBulkTransfer transfer0 = coordinating.get(transferId); + Preconditions.checkState(transfer0 instanceof TrackedRepairSyncTransfer, + "Expected TrackedRepairSyncTransfer for %s but got %s", + transferId, transfer0); + transfersToActivate.put(transferId, (TrackedRepairSyncTransfer) transfer0); + } + } + finally + { + lock.writeLock().unlock(); + } + + // Activate transfers WITHOUT holding the lock (activate() acquires its own locks and can block) + try + { + for (Map.Entry entry : transfersToActivate.entrySet()) + { + ShortMutationId transferId = entry.getKey(); + TrackedRepairSyncTransfer transfer = entry.getValue(); + List syncsForTransfer = syncsByTransferId.get(transferId); + transfer.activate(syncsForTransfer); + } + + // Activation succeeded, complete the future with the sync stats + activationFuture.trySuccess(syncs); + } + catch (Throwable t) + { + // Activation failed, fail the future + // Note: cleanup will be triggered automatically when the async COMMIT responses complete + logger.error("Activation failed", t); + activationFuture.tryFailure(t); + } + } + + @Override + public void onFailure(Throwable t) + { + logger.info("maybeActivate onFailure - cleaning up pending transfers", t); + + lock.writeLock().lock(); + try + { + Set transferIds = new HashSet<>(); + for (SyncTask task : job.getSyncTasks()) + { + ShortMutationId transferId = task.getTransferId(); + Preconditions.checkNotNull(transferId); + transferIds.add(transferId); + + TimeUUID planId = task.getPlanId(); + if (planId == null) + continue; + + AbstractCoordinatedBulkTransfer transfer = coordinating.get(transferId); + InetAddressAndPort peer = task.nodePair().peer; + logger.debug("{} Task for peer {} has planId {}, updating streamResults", transfer.logPrefix(), peer, planId); + transfer.streamResults.put(peer, AbstractCoordinatedBulkTransfer.SingleTransferResult.Init().streamFailed(planId)); + } + + for (ShortMutationId transferId : transferIds) + { + AbstractCoordinatedBulkTransfer transfer = coordinating.get(transferId); + try + { + transfer.notifyFailure(); + } + catch (Throwable t0) + { + logger.error("{} Failed to notify peers of repair failure", transfer, t0); + } + } + + scheduleCleanup(); + + // Sync failed, fail the activation future as well + activationFuture.tryFailure(t); + } + finally + { + lock.writeLock().unlock(); + } + } + }, executor); + + return activationFuture; + } + Purger purger = new Purger(); static class Purger @@ -120,13 +298,13 @@ static class Purger * has completed everywhere. If a transfer is partially activated (on some replicas but not others), it's going * to be included in future reconciliations and needs to be preserved until reconciliation is complete. */ - boolean test(CoordinatedTransfer transfer) + boolean test(AbstractCoordinatedBulkTransfer transfer) { logger.debug("Checking whether we can purge {}", transfer); boolean failedBeforeActivation = false; boolean noneActivated = true; boolean allComplete = true; - for (CoordinatedTransfer.SingleTransferResult result : transfer.streamResults.values()) + for (TrackedImportTransfer.SingleTransferResult result : transfer.streamResults.values()) { switch (result.state) { @@ -163,7 +341,7 @@ private void cleanup() if (purger.test(transfer)) purge(transfer); - for (CoordinatedTransfer transfer : coordinating.values()) + for (AbstractCoordinatedBulkTransfer transfer : coordinating.values()) if (purger.test(transfer)) purge(transfer); } @@ -219,7 +397,7 @@ private void purge(PendingLocalTransfer transfer) } } - private void purge(CoordinatedTransfer transfer) + private void purge(AbstractCoordinatedBulkTransfer transfer) { logger.info("Cleaning up completed coordinated transfer: {}", transfer); @@ -231,7 +409,7 @@ private void purge(CoordinatedTransfer transfer) if (transfer.id() != null) coordinating.remove(transfer.id()); - CoordinatedTransfer.SingleTransferResult localPending = transfer.streamResults.get(FBUtilities.getBroadcastAddressAndPort()); + TrackedImportTransfer.SingleTransferResult localPending = transfer.streamResults.get(FBUtilities.getBroadcastAddressAndPort()); PendingLocalTransfer localTransfer; TimeUUID planId; if (localPending != null && (planId = localPending.planId()) != null && (localTransfer = local.get(planId)) != null) @@ -270,7 +448,8 @@ void scheduleCleanup() } } - @Nullable CoordinatedTransfer getActivatedTransfer(ShortMutationId transferId) + @Nullable + AbstractCoordinatedBulkTransfer getActivatedTransfer(ShortMutationId transferId) { lock.readLock().lock(); try diff --git a/src/java/org/apache/cassandra/replication/MutationTrackingService.java b/src/java/org/apache/cassandra/replication/MutationTrackingService.java index 18394bdf5390..bd8db6f71a33 100644 --- a/src/java/org/apache/cassandra/replication/MutationTrackingService.java +++ b/src/java/org/apache/cassandra/replication/MutationTrackingService.java @@ -18,10 +18,12 @@ package org.apache.cassandra.replication; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -46,6 +48,7 @@ import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.cql3.UntypedResultSet; import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.Mutation; import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.SystemKeyspace; @@ -56,11 +59,14 @@ import org.apache.cassandra.exceptions.RequestFailure; import org.apache.cassandra.gms.FailureDetector; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.locator.EndpointsForRange; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.metrics.MutationTrackingMetrics; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.Verb; +import org.apache.cassandra.repair.SyncTask; +import org.apache.cassandra.repair.SyncTasks; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.ReplicationParams; import org.apache.cassandra.schema.Schema; @@ -68,6 +74,7 @@ import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.reads.tracked.TrackedLocalReads; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.listeners.ChangeListener; @@ -234,6 +241,35 @@ public MutationId nextMutationId(String keyspace, Token token) } } + // Requires that ranges is aligned to a single shard + public MutationId nextMutationId(String keyspace, Collection> ranges) + { + shardLock.readLock().lock(); + try + { + KeyspaceShards shards = getOrCreateShards(keyspace); + Shard shard = null; + for (Range range : ranges) + { + Shard curShard = shards.lookUp(range); + if (curShard == null) + throw new UnknownShardException(range, shards.groups); + if (shard == null) + shard = curShard; + else if (shard != curShard) + throw new IllegalStateException(String.format("Cannot generate a mutation ID for ranges (%s) that span across more than one shard (%s, %s)", ranges, shard, curShard)); + } + Preconditions.checkNotNull(shard); + MutationId id = shard.nextId(); + logger.trace("Created new mutation id {}", id); + return id; + } + finally + { + shardLock.readLock().unlock(); + } + } + public void sentWriteRequest(Mutation mutation, IntHashSet toHostIds) { Preconditions.checkArgument(!mutation.id().isNone()); @@ -258,7 +294,7 @@ public void receivedWriteResponse(ShortMutationId mutationId, InetAddressAndPort } } - public void receivedActivationResponse(CoordinatedTransfer transfer, InetAddressAndPort fromHost) + public void receivedActivationResponse(AbstractCoordinatedBulkTransfer transfer, InetAddressAndPort fromHost) { shardLock.readLock().lock(); try @@ -287,7 +323,7 @@ public void retryFailedWrite(ShortMutationId mutationId, InetAddressAndPort onHo activeReconciler.schedule(mutationId, onHost, ActiveLogReconciler.Priority.REGULAR); } - public void retryFailedTransfer(CoordinatedTransfer transfer, InetAddressAndPort onHost, Throwable cause) + public void retryFailedTransfer(AbstractCoordinatedBulkTransfer transfer, InetAddressAndPort onHost, Throwable cause) { if (transfer.isCommitted()) { @@ -375,10 +411,10 @@ public void executeTransfers(String keyspace, Set sstables, Consi logger.info("Creating tracked bulk transfers for keyspace '{}' SSTables {}...", keyspace, sstables); KeyspaceShards shards = checkNotNull(keyspaceShards.get(keyspace)); - CoordinatedTransfers transfers = CoordinatedTransfers.create(keyspace, shards, sstables, cl); + TrackedImportTransfers transfers = TrackedImportTransfers.create(keyspace, shards, sstables, cl); logger.info("Split input SSTables into transfers {}", transfers); - for (CoordinatedTransfer transfer : transfers) + for (TrackedImportTransfer transfer : transfers) transfer.execute(); } finally @@ -398,12 +434,22 @@ void activateLocal(TransferActivation activation) PendingLocalTransfer pending = LocalTransfers.instance().getPendingTransfer(activation.planId); if (pending == null) throw new IllegalStateException(String.format("Cannot activate unknown local pending transfer %s", activation)); - pending.activate(activation); + + boolean committed; + try + { + committed = pending.activate(activation); + } + catch (Exception e) + { + logger.error("Local activation of {} failed due to error", activation, e); + throw e; + } shardLock.readLock().lock(); try { - if (activation.isCommit()) + if (committed) { keyspaceShards.get(pending.keyspace).lookUp(pending.range).finishActivation(pending, activation); incomingMutations.invokeListeners(activation.transferId); @@ -780,6 +826,70 @@ private void collectDurablyReconciledOffsets(Log2OffsetsMap.Mutable into) forEachKeyspace(keyspace -> keyspace.collectDurablyReconciledOffsets(into)); } + /** + * Splits the given ranges so that each resulting group of ranges falls within a single shard. + *

+ * This is used during repair to ensure that sync tasks operate within shard boundaries, + * which is required for tracked keyspaces where mutation tracking is shard-based. + * + * @param keyspace the keyspace name + * @param ranges the ranges to align with shard boundaries + * @return a list of range lists, where each inner list contains ranges within a single shard + */ + public List>> alignedToShardBoundaries(String keyspace, Collection> ranges) + { + KeyspaceShards ks = keyspaceShards.get(keyspace); + if (ks == null) + { + // No shards for this keyspace, return ranges as a single group + return Collections.singletonList(new ArrayList<>(ranges)); + } + + // Group ranges by the shard they fall into after splitting + Map, List>> rangesByShard = new LinkedHashMap<>(); + + for (Range range : ranges) + { + // Find all shard ranges that intersect with this range + for (Range shardRange : ks.shards.keySet()) + { + Set> intersections = range.intersectionWith(shardRange); + for (Range intersection : intersections) + { + rangesByShard.computeIfAbsent(shardRange, k -> new ArrayList<>()).add(intersection); + } + } + } + + return new ArrayList<>(rangesByShard.values()); + } + + public void alignToShardBoundaries(String keyspace, List tasks, SyncTasks into) + { + Keyspace ks = Keyspace.open(keyspace); + Preconditions.checkArgument(ks != null && ks.getMetadata().replicationStrategy.replicationType.isTracked()); + + KeyspaceShards shards = keyspaceShards.get(keyspace); + + Map> aligned = new HashMap<>(); + + for (SyncTask task : tasks) + { + Set intersectingShards = new HashSet<>(); + shards.forEachIntersectingShard(task.rangesToSync, intersectingShards::add); + for (Shard intersectingShard : intersectingShards) + aligned.computeIfAbsent(intersectingShard, key -> new HashSet<>()) + .add(task.withRanges(Collections.singleton(intersectingShard.range))); + } + + for (Map.Entry> entry : aligned.entrySet()) + { + Shard shard = entry.getKey(); + Collection syncTasks = entry.getValue(); + into.addAll(shard.nextId(), shard.participants, syncTasks); + } + } + public static class KeyspaceShards { private enum UpdateDecision @@ -993,6 +1103,14 @@ private void forEachIntersectingShard(AbstractBounds bounds, }); } + private void forEachIntersectingShard(Collection> ranges, Consumer consumer) + { + shards.forEach((range0, shard) -> { + if (shard.range.intersects(ranges)) + consumer.accept(shard); + }); + } + void collectShardReconciledOffsetsToBuilder(ReconciledLogSnapshot.Builder builder) { ReconciledKeyspaceOffsets.Builder keyspaceBuilder = builder.getKeyspaceBuilder(keyspace); @@ -1098,8 +1216,9 @@ static int loadHostLogIdFromSystemTable() private static class ReplicatedOffsetsBroadcaster { // TODO (later): a more intelligent heuristic for scheduling broadcasts - private static final long TRANSIENT_BROADCAST_INTERVAL_MILLIS = 200; - private static final long DURABLE_BROADCAST_INTERVAL_MILLIS = 60_000; + // TODO: Revert before merge, just increased frequency for test + private static final long TRANSIENT_BROADCAST_INTERVAL_MILLIS = 1_000; + private static final long DURABLE_BROADCAST_INTERVAL_MILLIS = 1_000; private volatile boolean isPaused = false; @@ -1148,11 +1267,13 @@ private void run(Shard shard, boolean durable) private static class LogStatePersister implements Runnable { // TODO (expected): consider a different interval - private static final long PERSIST_INTERVAL_MINUTES = 1; + // TODO: Revert before merge, just increased frequency for test + // private static final long PERSIST_INTERVAL_MILLIS = 60_000; + private static final long PERSIST_INTERVAL_MILLIS = 1_000; void start() { - executor.scheduleWithFixedDelay(this, PERSIST_INTERVAL_MINUTES, PERSIST_INTERVAL_MINUTES, TimeUnit.MINUTES); + executor.scheduleWithFixedDelay(this, PERSIST_INTERVAL_MILLIS, PERSIST_INTERVAL_MILLIS, TimeUnit.MILLISECONDS); } @Override @@ -1222,5 +1343,34 @@ public static KeyspaceShards getKeyspaceShards(MutationTrackingService service, { return service.keyspaceShards.get(keyspace); } + + /** + * Creates a test KeyspaceShards with the given shard ranges. + * The shards are created with minimal configuration suitable for testing. + */ + public static KeyspaceShards createTestKeyspaceShards(String keyspace, Set> shardRanges) + { + Map, Shard> shards = new HashMap<>(); + Map, VersionedEndpoints.ForRange> groups = new HashMap<>(); + + int localNodeId = 1; + LongSupplier logId = () -> CoordinatorLogId.asLong(localNodeId, 1); + Participants participants = new Participants(List.of(localNodeId)); + for (Range range : shardRanges) + { + shards.put(range, new Shard(localNodeId, keyspace, range, participants, logId, (s, l) -> {})); + groups.put(range, VersionedEndpoints.forRange(Epoch.EMPTY, EndpointsForRange.empty(range))); + } + + return new KeyspaceShards(keyspace, shards, new ReplicaGroups(groups)); + } + + /** + * Sets the keyspace shards for testing purposes. + */ + public static void setKeyspaceShards(MutationTrackingService service, String keyspace, KeyspaceShards shards) + { + service.keyspaceShards.put(keyspace, shards); + } } } diff --git a/src/java/org/apache/cassandra/replication/PendingLocalTransfer.java b/src/java/org/apache/cassandra/replication/PendingLocalTransfer.java index a2c0d833a7c0..3e9a225e6bf1 100644 --- a/src/java/org/apache/cassandra/replication/PendingLocalTransfer.java +++ b/src/java/org/apache/cassandra/replication/PendingLocalTransfer.java @@ -139,10 +139,10 @@ private boolean isFullReplica() * Synchronized to prevent a single activation from running multiple times if requested during read reconciliation * and in the background via {@link ActiveLogReconciler}. */ - public synchronized void activate(TransferActivation activation) + public synchronized boolean activate(TransferActivation activation) { if (activated) - return; + return false; Preconditions.checkState(isFullReplica()); @@ -155,7 +155,7 @@ public synchronized void activate(TransferActivation activation) if (activation.isPrepare()) { logger.info("{} Not adding SSTables to live set for dryRun {}", logPrefix(), activation); - return; + return false; } // Modify SSTables metadata to durably set transfer ID before importing @@ -166,7 +166,8 @@ public synchronized void activate(TransferActivation activation) // Ensure no lingering mutation IDs, only activation IDs for (SSTableReader sstable : sstables) { - Preconditions.checkState(sstable.getCoordinatorLogOffsets().mutations().isEmpty()); + // For TrackedImportTransfer, CLO should be empty, but for TrackedRepairSyncTransfer, can have CLOs + // Preconditions.checkState(sstable.getCoordinatorLogOffsets().mutations().isEmpty()); try { sstable.mutateCoordinatorLogOffsetsAndReload(logOffsets); @@ -211,6 +212,7 @@ public synchronized void activate(TransferActivation activation) logger.info("{} Finished activating transfer {} in {} ms", logPrefix(), this, finishedActivation - startedActivation); LocalTransfers.instance().scheduleCleanup(); + return true; } @Override diff --git a/src/java/org/apache/cassandra/replication/Shard.java b/src/java/org/apache/cassandra/replication/Shard.java index 5f7f1e7ee641..4e7b070a987e 100644 --- a/src/java/org/apache/cassandra/replication/Shard.java +++ b/src/java/org/apache/cassandra/replication/Shard.java @@ -185,7 +185,7 @@ void finishActivation(PendingLocalTransfer transfer, TransferActivation activati getOrCreate(activation.transferId).finishActivation(transfer, activation); } - void receivedActivationResponse(CoordinatedTransfer transfer, InetAddressAndPort onHost) + void receivedActivationResponse(AbstractCoordinatedBulkTransfer transfer, InetAddressAndPort onHost) { int onHostId = ClusterMetadata.current().directory.peerId(onHost).id(); getOrCreate(transfer.id()).receivedActivationResponse(transfer, onHostId); @@ -406,6 +406,17 @@ void collectShardReconciledOffsetsToBuilder(ReconciledKeyspaceOffsets.Builder ke logs.values().forEach(log -> keyspaceBuilder.put(log.logId, log.collectReconciledOffsets(), range)); } + @Override + public String toString() + { + return "Shard{" + + "participants=" + participants + + ", range=" + range + + ", keyspace='" + keyspace + '\'' + + ", localNodeId=" + localNodeId + + '}'; + } + public DebugInfo getDebugInfo() { SortedMap logDebugState = new TreeMap<>(Comparator.comparing(CoordinatorLogId::asLong)); diff --git a/src/java/org/apache/cassandra/replication/TrackedImportTransfer.java b/src/java/org/apache/cassandra/replication/TrackedImportTransfer.java new file mode 100644 index 000000000000..38711b4451b3 --- /dev/null +++ b/src/java/org/apache/cassandra/replication/TrackedImportTransfer.java @@ -0,0 +1,389 @@ +/* + * 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.replication; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Supplier; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.streaming.CassandraOutgoingFile; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.locator.AbstractReplicationStrategy; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.streaming.OutgoingStream; +import org.apache.cassandra.streaming.StreamException; +import org.apache.cassandra.streaming.StreamOperation; +import org.apache.cassandra.streaming.StreamPlan; +import org.apache.cassandra.streaming.StreamResultFuture; +import org.apache.cassandra.streaming.StreamState; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.utils.Throwables; +import org.apache.cassandra.utils.TimeUUID; +import org.apache.cassandra.utils.concurrent.Future; + +import static org.apache.cassandra.replication.AbstractCoordinatedBulkTransfer.SingleTransferResult.State.COMMITTED; +import static org.apache.cassandra.replication.AbstractCoordinatedBulkTransfer.SingleTransferResult.State.COMMITTING; +import static org.apache.cassandra.replication.AbstractCoordinatedBulkTransfer.SingleTransferResult.State.STREAM_COMPLETE; +import static org.apache.cassandra.replication.TransferActivation.Phase; + +/** + * Orchestrates the lifecycle of a tracked bulk data transfer for a single replica set, where the current instance is + * coordinating the transfer. + *

+ * The transfer proceeds through these phases: + *

    + *
  1. + * Streaming + * The coordinator streams SSTables to all replicas in parallel. Replicas store received data in a "pending" + * location where it's persisted to disk but not yet visible to reads. Once sufficient replicas have received + * their streams to meet the requested {@link ConsistencyLevel}, the SSTables are activated using a two-phase + * commit protocol, making them part of the live set and visible to reads. + *
  2. + *
  3. + * Activation {@link Phase#PREPARE} + * The coordinator sends PREPARE messages to verify replicas have the data persisted on disk and are ready for + * activation. + *
  4. + *
  5. + * Activation {@link Phase#COMMIT} + * After successful PREPARE, the coordinator sends COMMIT messages to replicas. Replicas atomically move data from + * pending to live sets, making it visible to reads with the proper transfer ID in metadata. If commit succeeds + * on some replicas but not others, the transfer will be activated later on via existing the existing + * reconciliation processes (read reconciliation and background reconciliation). + *
  6. + *
+ * + * For simplicity, the coordinator streams to itself rather than using direct file copy. This ensures we can use the + * same lifecycle management for crash-safety and atomic add. + *

+ * If a tracked data read is executed on a replica that's missing an activation, the read reconciliation process will + * apply the missing activation during reconciliation and a subsequent read will succeed. To minimize the gap between + * activations across replicas, avoid expensive operations like file copies or index builds during + * {@link TransferActivation#apply()}. + */ +public class TrackedImportTransfer extends AbstractCoordinatedBulkTransfer +{ + private static final Logger logger = LoggerFactory.getLogger(TrackedImportTransfer.class); + + private final String keyspace; + private final Range range; + final Collection sstables; + private final ConsistencyLevel cl; + // TODO: Refactor to new class PendingTransfers + final ConcurrentMap streamResults; + + @VisibleForTesting + TrackedImportTransfer(Range range, MutationId id) + { + super(id); + this.keyspace = null; + this.range = range; + this.sstables = Collections.emptyList(); + this.cl = null; + this.streamResults = new ConcurrentHashMap<>(); + } + + TrackedImportTransfer(String keyspace, Range range, Participants participants, Collection sstables, ConsistencyLevel cl, Supplier nextId) + { + super(nextId.get()); + this.keyspace = keyspace; + this.range = range; + this.sstables = sstables; + this.cl = cl; + + ClusterMetadata cm = ClusterMetadata.current(); + this.streamResults = new ConcurrentHashMap<>(participants.size()); + for (int i = 0; i < participants.size(); i++) + { + InetAddressAndPort addr = cm.directory.getNodeAddresses(new NodeId(participants.get(i))).broadcastAddress; + this.streamResults.put(addr, SingleTransferResult.Init()); + } + } + + void execute() + { + logger.debug("{} Executing tracked bulk transfer {}", logPrefix(), this); + LocalTransfers.instance().save(this); + stream(); + } + + private void stream() + { + // TODO: Don't stream multiple copies over the WAN, send one copy and indicate forwarding + List> streaming = new ArrayList<>(streamResults.size()); + for (InetAddressAndPort to : streamResults.keySet()) + { + Future stream = LocalTransfers.instance().executor.submit(() -> { + stream(to); + return null; + }); + streaming.add(stream); + } + + // Wait for all streams to complete, so we can clean up after failures. If we exit at the first failure, a + // future stream can complete. + LinkedList failures = null; + for (Future stream : streaming) + { + try + { + stream.get(); + } + catch (InterruptedException | ExecutionException e) + { + if (failures == null) + failures = new LinkedList<>(); + failures.add(e); + logger.error("{} Failed transfer due to", logPrefix(), e); + } + } + + if (failures != null && !failures.isEmpty()) + { + Throwable failure = failures.element(); + Throwable cause = failure instanceof ExecutionException ? failure.getCause() : failure; + maybeCleanupFailedStreams(cause); + + String msg = String.format("Failed streaming on %s instance(s): %s", failures.size(), failures); + throw new RuntimeException(msg, Throwables.unchecked(cause)); + } + + logger.info("{} All streaming completed successfully", logPrefix()); + } + + private boolean sufficient() + { + AbstractReplicationStrategy ars = Keyspace.open(keyspace).getReplicationStrategy(); + int blockFor = cl.blockFor(ars); + int responses = 0; + for (Map.Entry entry : streamResults.entrySet()) + { + if (entry.getValue().state == STREAM_COMPLETE) + responses++; + } + return responses >= blockFor; + } + + void stream(InetAddressAndPort to) + { + SingleTransferResult result; + try + { + result = streamTask(to); + } + catch (StreamException | ExecutionException | InterruptedException | TimeoutException e) + { + Throwable cause = e instanceof ExecutionException ? e.getCause() : e; + markStreamFailure(to, cause); + throw Throwables.unchecked(cause); + } + + try + { + streamComplete(to, result); + } + catch (ExecutionException | InterruptedException | TimeoutException e) + { + Throwable cause = e instanceof ExecutionException ? e.getCause() : e; + throw Throwables.unchecked(cause); + } + } + + private void markStreamFailure(InetAddressAndPort to, Throwable cause) + { + TimeUUID planId; + if (cause instanceof StreamException) + planId = ((StreamException) cause).finalState.planId; + else + planId = null; + streamResults.computeIfPresent(to, (peer, result) -> result.streamFailed(planId)); + } + + /** + * This shouldn't throw an exception, even if we fail to notify peers of the streaming failure. + */ + private void maybeCleanupFailedStreams(Throwable cause) + { + try + { + boolean purgeable = LocalTransfers.instance().purger.test(this); + if (!purgeable) + return; + + notifyFailure(); + LocalTransfers.instance().scheduleCleanup(); + } + catch (Throwable t) + { + if (cause != null) + t.addSuppressed(cause); + logger.error("{} Failed to notify peers of stream failure", logPrefix(), t); + } + } + + private void streamComplete(InetAddressAndPort to, SingleTransferResult result) throws ExecutionException, InterruptedException, TimeoutException + { + streamResults.put(to, result); + logger.info("{} Completed streaming to {}, {}", logPrefix(), to, this); + maybeActivate(); + } + + synchronized void maybeActivate() + { + // If any activations have already been sent out, send new activations to any received plans that have not yet + // been activated + boolean anyActivated = false; + Set awaitingActivation = new HashSet<>(); + for (Map.Entry entry : streamResults.entrySet()) + { + InetAddressAndPort peer = entry.getKey(); + SingleTransferResult result = entry.getValue(); + if (result.state == COMMITTING || result.state == COMMITTED) + { + anyActivated = true; + } + else if (result.state == STREAM_COMPLETE) + awaitingActivation.add(peer); + } + if (anyActivated && !awaitingActivation.isEmpty()) + { + logger.debug("{} Transfer already activated on some peers, sending activations to remaining: {}", logPrefix(), awaitingActivation); + activate(awaitingActivation); + return; + } + // If no activations have been sent out, check whether we have enough planIds back to meet the required CL + else if (sufficient()) + { + Set peers = new HashSet<>(); + for (Map.Entry entry : streamResults.entrySet()) + { + InetAddressAndPort peer = entry.getKey(); + SingleTransferResult result = entry.getValue(); + if (result.state == STREAM_COMPLETE) + peers.add(peer); + } + logger.debug("{} Transfer meets consistency level {}, sending activations to {}", logPrefix(), cl, peers); + activate(peers); + return; + } + + logger.debug("{} Nothing to activate", logPrefix()); + } + + public boolean isCommitted() + { + for (SingleTransferResult result : streamResults.values()) + { + if (result.state != COMMITTED) + return false; + } + return true; + } + + private SingleTransferResult streamTask(InetAddressAndPort to) throws StreamException, ExecutionException, InterruptedException, TimeoutException + { + StreamPlan plan = new StreamPlan(StreamOperation.TRACKED_TRANSFER); + + // No need to flush, only using non-live SSTables already on disk + plan.flushBeforeTransfer(false); + + for (SSTableReader sstable : sstables) + { + List> ranges = Collections.singletonList(range); + List positions = sstable.getPositionsForRanges(ranges); + long estimatedKeys = sstable.estimatedKeysForRanges(ranges); + OutgoingStream stream = new CassandraOutgoingFile(StreamOperation.TRACKED_TRANSFER, sstable.ref(), positions, ranges, estimatedKeys); + plan.transferStreams(to, Collections.singleton(stream)); + } + + long timeout = DatabaseDescriptor.getStreamTransferTaskTimeout().toMilliseconds(); + + logger.info("{} Starting streaming transfer {} to peer {}", logPrefix(), this, to); + StreamResultFuture execute = plan.execute(); + StreamState state; + try + { + state = execute.get(timeout, TimeUnit.MILLISECONDS); + logger.debug("{} Completed streaming transfer {} to peer {}", logPrefix(), this, to); + } + catch (InterruptedException | ExecutionException | TimeoutException e) + { + logger.error("Stream session failed with error", e); + throw e; + } + + if (state.hasFailedSession() || state.hasAbortedSession()) + throw new StreamException(state, "Stream failed due to failed or aborted sessions"); + + // If the SSTable doesn't contain any rows in the provided range, no streams delivered, nothing to activate + if (state.sessions().isEmpty()) + return SingleTransferResult.Noop(); + + return SingleTransferResult.StreamComplete(plan.planId()); + } + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) return false; + TrackedImportTransfer that = (TrackedImportTransfer) o; + return Objects.equals(keyspace, that.keyspace) && Objects.equals(range, that.range) && cl == that.cl && Objects.equals(streamResults, that.streamResults); + } + + @Override + public int hashCode() + { + return Objects.hash(keyspace, range, cl, streamResults); + } + + @Override + public String toString() + { + return "TrackedImportTransfer{" + + "keyspace='" + keyspace + '\'' + + ", range=" + range + + ", cl=" + cl + + ", streamResults=" + streamResults + + ", sstables=" + sstables + + ", streamResults=" + streamResults + + '}'; + } +} diff --git a/src/java/org/apache/cassandra/replication/CoordinatedTransfers.java b/src/java/org/apache/cassandra/replication/TrackedImportTransfers.java similarity index 76% rename from src/java/org/apache/cassandra/replication/CoordinatedTransfers.java rename to src/java/org/apache/cassandra/replication/TrackedImportTransfers.java index 2adb546fc5b3..250c240e0ae5 100644 --- a/src/java/org/apache/cassandra/replication/CoordinatedTransfers.java +++ b/src/java/org/apache/cassandra/replication/TrackedImportTransfers.java @@ -33,20 +33,20 @@ import org.apache.cassandra.utils.Interval; /** - * Factory and container for creating multiple {@link CoordinatedTransfer} instances from a collection + * Factory and container for creating multiple {@link TrackedImportTransfer} instances from a collection * of SSTables, partitioned by {@link MutationTrackingService.KeyspaceShards}, which are aligned to replica ownership * ranges. Each shard receives its own CoordinatedTransfer instance, which can be executed independently. */ -class CoordinatedTransfers implements Iterable +public class TrackedImportTransfers implements Iterable { - private final Collection transfers; + private final Collection transfers; - private CoordinatedTransfers(Collection transfers) + private TrackedImportTransfers(Collection transfers) { this.transfers = transfers; } - static CoordinatedTransfers create(String keyspace, MutationTrackingService.KeyspaceShards shards, Collection sstables, ConsistencyLevel cl) + static TrackedImportTransfers create(String keyspace, MutationTrackingService.KeyspaceShards shards, Collection sstables, ConsistencyLevel cl) { // Clean up incoming SSTables to remove any existing untrusted CoordinatorLogOffsets for (SSTableReader sstable : sstables) @@ -62,7 +62,7 @@ static CoordinatedTransfers create(String keyspace, MutationTrackingService.Keys } SSTableIntervalTree intervals = SSTableIntervalTree.buildSSTableIntervalTree(sstables); - List transfers = new ArrayList<>(); + List transfers = new ArrayList<>(); shards.forEachShard(shard -> { Range range = shard.tokenRange(); @@ -70,14 +70,14 @@ static CoordinatedTransfers create(String keyspace, MutationTrackingService.Keys if (sstablesForRange.isEmpty()) return; - CoordinatedTransfer transfer = new CoordinatedTransfer(keyspace, range, shard.participants, sstablesForRange, cl, shard::nextId); + TrackedImportTransfer transfer = new TrackedImportTransfer(keyspace, range, shard.participants, sstablesForRange, cl, shard::nextId); transfers.add(transfer); }); - return new CoordinatedTransfers(transfers); + return new TrackedImportTransfers(transfers); } @Override - public Iterator iterator() + public Iterator iterator() { return transfers.iterator(); } diff --git a/src/java/org/apache/cassandra/replication/TrackedRepairSyncTransfer.java b/src/java/org/apache/cassandra/replication/TrackedRepairSyncTransfer.java new file mode 100644 index 000000000000..9c50f3311566 --- /dev/null +++ b/src/java/org/apache/cassandra/replication/TrackedRepairSyncTransfer.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.replication; + +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.SyncStat; +import org.apache.cassandra.repair.SyncTask; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.NodeId; + +/** + * Repair sync tasks (that stream SSTable contents) must be integrated with Mutation Tracking's bulk transfer handling + * because any data that is not present on all instances must be expressed as unreconciled in the log, since read + * reconciliations depend on the log state to guarantee monotonicity of subsequent reads. Streaming sessions for full + * repair can complete on some instances before others, so we need to represent those completed sessions as unreconciled + * in the log. + *

+ * TODO: call LocalTransfers.instance().save(this) once the RepairJob starts streaming + */ +public class TrackedRepairSyncTransfer extends AbstractCoordinatedBulkTransfer +{ + private static final Logger logger = LoggerFactory.getLogger(TrackedRepairSyncTransfer.class); + + public TrackedRepairSyncTransfer(ShortMutationId id, Participants participants, Collection tasks) + { + super(id); + + Set replicaNodeIds = participants.asSet(); + Map> receivingReplicaToTasks = new HashMap<>(); + + for (SyncTask task : tasks) + receivingReplicaToTasks.computeIfAbsent(task.nodePair().peer, key -> new HashSet<>()).add(task); + + ClusterMetadata cm = ClusterMetadata.current(); + for (Integer replicaNodeId : replicaNodeIds) + { + InetAddressAndPort addr = cm.directory.endpoint(new NodeId(replicaNodeId)); + Collection syncTasks = receivingReplicaToTasks.get(addr); + // Need to activate on all replicas, not just ones with SyncTasks. For replicas that don't receive any data + // as part of a repair, they still need to activate the transfer ID as a no-op, to allow read reconciliations + // to complete. + if (syncTasks == null) + streamResults.put(addr, SingleTransferResult.Noop()); + else + streamResults.put(addr, SingleTransferResult.Init()); + } + } + + /** + * When all {@link SyncTask} for a repair have completed, follow the bulk transfer activation path to safely make + * the new data live and tracked in the log. This needs to include all replicas of {@link #id()}, even those that + * did not receive anything as part of the repair, otherwise any read reconciliation will fail to complete. + */ + public void activate(List syncs) + { + logger.debug("{} Activating {}: {}", logPrefix(), this, syncs); + + for (SyncStat sync : syncs) + { + Preconditions.checkNotNull(sync.planId); + streamResults.put(sync.nodes.peer, SingleTransferResult.StreamComplete(sync.planId)); + } + + activate(streamResults.keySet()); + } + + @Override + public String toString() + { + return "TrackedRepairSyncTransfer{" + + "streamResults=" + streamResults + + '}'; + } +} diff --git a/src/java/org/apache/cassandra/replication/TransferActivation.java b/src/java/org/apache/cassandra/replication/TransferActivation.java index ba0c918286d8..253e62f6b386 100644 --- a/src/java/org/apache/cassandra/replication/TransferActivation.java +++ b/src/java/org/apache/cassandra/replication/TransferActivation.java @@ -23,6 +23,9 @@ import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.exceptions.RequestFailureReason; import org.apache.cassandra.io.IVersionedSerializer; @@ -42,10 +45,12 @@ * reads) to live (visible to reads and compactions), by associating the streaming plan ID with a mutation ID, referred * to as the transfer ID. *

- * See {@link CoordinatedTransfer} for the lifecycle of a transfer and when a {@link TransferActivation} is sent. + * See {@link TrackedImportTransfer} for the lifecycle of a transfer and when a {@link TransferActivation} is sent. */ public class TransferActivation { + private static final Logger logger = LoggerFactory.getLogger(TransferActivation.class); + public final TimeUUID planId; public final ShortMutationId transferId; public final NodeId coordinatorId; @@ -78,11 +83,6 @@ static Phase from(int id) } } - public TransferActivation(CoordinatedTransfer transfer, InetAddressAndPort peer, Phase phase) - { - this(transfer.streamResults.get(peer).planId(), transfer.id(), ClusterMetadata.current().myNodeId(), phase); - } - TransferActivation(TimeUUID planId, ShortMutationId transferId, NodeId coordinatorId, Phase phase) { Preconditions.checkArgument(!transferId.isNone()); @@ -94,6 +94,27 @@ public TransferActivation(CoordinatedTransfer transfer, InetAddressAndPort peer, this.phase = phase; } + private TransferActivation(AbstractCoordinatedBulkTransfer transfer, InetAddressAndPort peer, Phase phase) + { + this(transfer.streamResults.get(peer).planId(), transfer.id(), ClusterMetadata.current().myNodeId(), phase); + } + + public static TransferActivation create(AbstractCoordinatedBulkTransfer transfer, InetAddressAndPort peer, Phase phase) + { + try + { + // TODO: If transfer is a TrackedRepairSyncTransfer, it's possible for the planId to be empty if that + // replica received no data as part of the repair. Need to support a TransferActivation that just adds a + // transferId to the log, without a planId. + return new TransferActivation(transfer, peer, phase); + } + catch (Throwable t) + { + logger.error("Could not create TransferActivation", t); + throw t; + } + } + ShortMutationId id() { return transferId; @@ -174,7 +195,7 @@ public void doVerb(Message msg) throws IOException public String toString() { return "TransferActivation{" + - ", planId=" + planId + + "planId=" + planId + ", transferId=" + transferId + ", coordinatorId=" + coordinatorId + ", phase=" + phase + diff --git a/src/java/org/apache/cassandra/replication/TransferFailed.java b/src/java/org/apache/cassandra/replication/TransferFailed.java index 6a956df87067..ef86ba8a778b 100644 --- a/src/java/org/apache/cassandra/replication/TransferFailed.java +++ b/src/java/org/apache/cassandra/replication/TransferFailed.java @@ -28,7 +28,7 @@ /** * Notification from coordinator to replicas when a bulk data transfer fails, triggering cleanup of the pending * transfer state. - * @see CoordinatedTransfer + * @see TrackedImportTransfer * @see PendingLocalTransfer */ public class TransferFailed diff --git a/src/java/org/apache/cassandra/replication/UnreconciledMutations.java b/src/java/org/apache/cassandra/replication/UnreconciledMutations.java index 22c15502306b..bbc3206ac9e9 100644 --- a/src/java/org/apache/cassandra/replication/UnreconciledMutations.java +++ b/src/java/org/apache/cassandra/replication/UnreconciledMutations.java @@ -274,10 +274,11 @@ static UnreconciledMutations loadFromJournal(Node2OffsetsMap witnessedOffsets, i result.addDirectly(mutation); continue; } - CoordinatedTransfer transfer = LocalTransfers.instance().getActivatedTransfer(id); + AbstractCoordinatedBulkTransfer transfer = LocalTransfers.instance().getActivatedTransfer(id); if (transfer != null) { - result.transfers.add(transfer.id(), transfer.sstables); + Preconditions.checkState(transfer instanceof TrackedImportTransfer); + result.transfers.add(transfer.id(), ((TrackedImportTransfer) transfer).sstables); continue; } diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 1c497544a70b..b504fd2664d3 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -188,6 +188,7 @@ import org.apache.cassandra.streaming.StreamManager; import org.apache.cassandra.streaming.StreamResultFuture; import org.apache.cassandra.streaming.StreamState; +import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.MultiStepOperation; @@ -3056,6 +3057,15 @@ public Pair> repair(String keyspace, Map repa public Pair> repair(String keyspace, RepairOption option, List listeners) { + KeyspaceMetadata ksm = Keyspace.open(keyspace).getMetadata(); + if (ksm.params.replicationType.isTracked()) + { + if (option.getPreviewKind() == PreviewKind.REPAIRED) + throw new IllegalArgumentException("Tracked keyspaces do not support validation repair"); + if (option.isIncremental()) + throw new IllegalArgumentException("Tracked keyspaces do not support incremental repair"); + } + // if ranges are not specified if (option.getRanges().isEmpty()) { diff --git a/src/java/org/apache/cassandra/streaming/StreamPlan.java b/src/java/org/apache/cassandra/streaming/StreamPlan.java index 4d249926264c..6da96f8afb1b 100644 --- a/src/java/org/apache/cassandra/streaming/StreamPlan.java +++ b/src/java/org/apache/cassandra/streaming/StreamPlan.java @@ -116,10 +116,8 @@ public StreamPlan requestRanges(InetAddressAndPort from, String keyspace, Ranges // TODO: add flag for fully reconciled data only if this is for a tracked keyspace session.addStreamRequest(keyspace, fullRanges, transientRanges, Arrays.asList(columnFamilies)); - // Automatically include mutation logs for tracked keyspaces - if (isTrackedReplicationEnabled(keyspace)) { + if (includeMutationLogs(keyspace, session)) session.addMutationLogRequest(keyspace, fullRanges, transientRanges); - } return this; } @@ -136,9 +134,7 @@ public StreamPlan requestRanges(InetAddressAndPort from, String keyspace, Ranges public StreamPlan transferRanges(InetAddressAndPort to, String keyspace, RangesAtEndpoint replicas, String... columnFamilies) { StreamSession session = coordinator.getOrCreateOutboundSession(to); - - // Automatically include mutation logs for tracked keyspaces - ReconciledKeyspaceOffsets reconciledKeyspaceOffsets = isTrackedReplicationEnabled(keyspace) + ReconciledKeyspaceOffsets reconciledKeyspaceOffsets = includeMutationLogs(keyspace, session) ? session.addMutationLogTransfer(keyspace, replicas) : null; @@ -147,6 +143,15 @@ public StreamPlan transferRanges(InetAddressAndPort to, String keyspace, RangesA return this; } + /** + * Do we want to include mutation logs for repair? Probably not, because we already flush before repair, and want + * repair to correct issues in the LSM, ignoring the log entirely. + */ + private boolean includeMutationLogs(String keyspace, StreamSession session) + { + return isTrackedReplicationEnabled(keyspace) && session.getStreamOperation() != StreamOperation.REPAIR; + } + /** * Add transfer task to send given streams * diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java index e93dec699f2b..8a600917866e 100644 --- a/src/java/org/apache/cassandra/streaming/StreamSession.java +++ b/src/java/org/apache/cassandra/streaming/StreamSession.java @@ -1020,6 +1020,15 @@ private void prepareLogTransferring(LogStreamManifest manifest) manifest.keyspaceRanges.forEach((keyspace, ranges) -> logTransfer.addKeyspaceRanges(keyspace, ranges)); } + + /** + * + */ + public boolean requiresTrackedActivation() + { + return streamOperation == StreamOperation.REPAIR || streamOperation == StreamOperation.TRACKED_TRANSFER; + } + /** * In the case where we have an error checking disk space we allow the Operation to continue. * In the case where we do _not_ have available space, this method raises a RTE. diff --git a/src/java/org/apache/cassandra/tcm/ownership/ReplicaGroups.java b/src/java/org/apache/cassandra/tcm/ownership/ReplicaGroups.java index dc43ae2ee10b..fa7f4a9bd8da 100644 --- a/src/java/org/apache/cassandra/tcm/ownership/ReplicaGroups.java +++ b/src/java/org/apache/cassandra/tcm/ownership/ReplicaGroups.java @@ -130,11 +130,11 @@ public VersionedEndpoints.ForRange matchToken(Token token) } /** - * This method is intended to be used on read/write path, not forRange. + * This method is intended to be used on read/write path, not forRange. Returns a {@link VersionedEndpoints.ForRange} + * with the full ownership range, which necessarily contains the input range but may be wider. */ public VersionedEndpoints.ForRange matchRange(Range range) { - EndpointsForRange.Builder builder = new EndpointsForRange.Builder(range); Epoch lastModified = Epoch.EMPTY; // find a range containing the *right* token for the given range - Range is start exclusive so if we looked for the // left one we could get the wrong range @@ -142,10 +142,9 @@ public VersionedEndpoints.ForRange matchRange(Range range) if (pos >= 0 && pos < ranges.size() && ranges.get(pos).contains(range)) { VersionedEndpoints.ForRange eps = endpoints.get(pos); - lastModified = eps.lastModified(); - builder.addAll(eps.get(), ReplicaCollection.Builder.Conflict.ALL); + return VersionedEndpoints.forRange(eps.lastModified(), eps.get()); } - return VersionedEndpoints.forRange(lastModified, builder.build()); + return VersionedEndpoints.forRange(lastModified, EndpointsForRange.empty(range)); } public VersionedEndpoints.ForRange forRange(Token token) diff --git a/src/java/org/apache/cassandra/utils/concurrent/Ref.java b/src/java/org/apache/cassandra/utils/concurrent/Ref.java index 6719948e3343..4d7d8c73035a 100644 --- a/src/java/org/apache/cassandra/utils/concurrent/Ref.java +++ b/src/java/org/apache/cassandra/utils/concurrent/Ref.java @@ -100,8 +100,8 @@ public final class Ref implements RefCounted { static final Logger logger = LoggerFactory.getLogger(Ref.class); - public static final boolean TRACE_ENABLED = TEST_DEBUG_REF_COUNT.getBoolean(); - public static final boolean DEBUG_EVENTS_ENABLED = TEST_DEBUG_REF_EVENTS.getBoolean(); + public static final boolean TRACE_ENABLED = TEST_DEBUG_REF_COUNT.getBoolean(true); + public static final boolean DEBUG_EVENTS_ENABLED = TEST_DEBUG_REF_EVENTS.getBoolean(true); static OnLeak ON_LEAK; @Shared(scope = SIMULATION) diff --git a/test/conf/logback-dtest.xml b/test/conf/logback-dtest.xml index ee79d68c874a..b07ade6dd08c 100644 --- a/test/conf/logback-dtest.xml +++ b/test/conf/logback-dtest.xml @@ -58,7 +58,7 @@ --> - + diff --git a/test/conf/logback-test.xml b/test/conf/logback-test.xml index 757806e35f2b..677e7569db50 100644 --- a/test/conf/logback-test.xml +++ b/test/conf/logback-test.xml @@ -60,9 +60,11 @@ + + - + 0 0 diff --git a/test/distributed/org/apache/cassandra/distributed/test/TrackedKeyspaceRepairSupportTest.java b/test/distributed/org/apache/cassandra/distributed/test/TrackedKeyspaceRepairSupportTest.java new file mode 100644 index 000000000000..b8c462285277 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/TrackedKeyspaceRepairSupportTest.java @@ -0,0 +1,519 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import org.junit.Test; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import net.bytebuddy.ByteBuddy; +import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; +import net.bytebuddy.implementation.bind.annotation.SuperCall; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.streaming.CassandraStreamReceiver; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.NodeToolResult; +import org.apache.cassandra.distributed.shared.AssertUtils; +import org.apache.cassandra.distributed.shared.ClusterUtils; +import org.apache.cassandra.distributed.test.tracking.BulkTransfersTest; +import org.apache.cassandra.distributed.test.tracking.MutationTrackingReadReconciliationTest; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.replication.TransferActivation; +import org.assertj.core.api.Assertions; +import org.awaitility.Awaitility; + +import static net.bytebuddy.implementation.MethodDelegation.to; +import static net.bytebuddy.matcher.ElementMatchers.named; +import static net.bytebuddy.matcher.ElementMatchers.takesNoArguments; +import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL; +import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM; +import static org.apache.cassandra.distributed.shared.AssertUtils.row; + +public class TrackedKeyspaceRepairSupportTest extends TestBaseImpl +{ + private static final Logger logger = LoggerFactory.getLogger(TrackedKeyspaceRepairSupportTest.class); + + private static final String KEYSPACE = "tracked_ks"; + private static final String TABLE = "tbl"; + private static final String KEYSPACE_TABLE = String.format("%s.%s", KEYSPACE, TABLE); + + @Test + public void testIncrementalRepairRejected() throws IOException + { + try (Cluster cluster = Cluster.build(3) + .withConfig(cfg -> cfg.with(Feature.NETWORK) + .with(Feature.GOSSIP) + .set("mutation_tracking_enabled", "true")) + .start()) + { + cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked';"); + cluster.schemaChange("CREATE TABLE " + KEYSPACE_TABLE + " (k INT PRIMARY KEY, v INT)"); + + NodeToolResult result = cluster.get(1).nodetoolResult("repair", KEYSPACE); + result.asserts().failure(); + Assertions.assertThat(result.getError()).hasMessageMatching("Tracked keyspaces do not support incremental repair"); + } + } + + @Test + public void testValidationRepairRejected() throws IOException + { + try (Cluster cluster = Cluster.build(3) + .withConfig(cfg -> cfg.with(Feature.NETWORK) + .with(Feature.GOSSIP) + .set("mutation_tracking_enabled", "true")) + .start()) + { + cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked';"); + cluster.schemaChange("CREATE TABLE " + KEYSPACE_TABLE + " (k INT PRIMARY KEY, v INT)"); + + NodeToolResult result = cluster.get(1).nodetoolResult("repair", "--validate", KEYSPACE); + result.asserts().failure(); + Assertions.assertThat(result.getError()).hasMessageMatching("Tracked keyspaces do not support validation repair"); + } + } + + @Test + public void testFullRepairPermitted() throws IOException + { + try (Cluster cluster = Cluster.build(3) + .withConfig(cfg -> cfg.with(Feature.NETWORK) + .with(Feature.GOSSIP) + .set("mutation_tracking_enabled", "true")) + .start()) + { + cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked';"); + NodeToolResult result = cluster.get(1).nodetoolResult("repair", "--full", KEYSPACE); + result.asserts().success(); + } + } + + @Test + public void testFullRepairPartiallyCompleteAnomaly() throws IOException, ExecutionException, InterruptedException, TimeoutException + { + try (Cluster cluster = Cluster.build(3) + .withInstanceInitializer(StreamReceiverFailureHelper::install) + .withConfig(cfg -> cfg + .with(Feature.NETWORK) + .with(Feature.GOSSIP) + .set("mutation_tracking_enabled", "true")) + .start()) + { + cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked';"); + String TABLE_SCHEMA_CQL = "CREATE TABLE " + KEYSPACE + '.' + TABLE + " (k INT PRIMARY KEY, v INT)"; + cluster.schemaChange(TABLE_SCHEMA_CQL); + + IInvokableInstance COORDINATING = cluster.get(1); + IInvokableInstance RECEIVING = cluster.get(2); + IInvokableInstance MISSING = cluster.get(3); + + /* + If we were to start this process with a normal write, that write would be added to the log. When repair + validation runs and finds the mismatching range, it streams the log. MISSING then receives the log and + applies the new mutation to the memtable, where it's visible to reads. + + We want to emulate a situation where one node has a mutation that's not present in the log, hence the + roundabout write path. Once the mutation has completed and is repaired on the coordinator, it's been + (durably) reconciled on all replicas. Then, drop this SSTable on the other peers and we should have a full + repair digest mismatch. + + The idea here is to emulate logical data corruption, where SSTables mismatch but the logs are in agreement. + */ + COORDINATING.coordinator().execute("INSERT INTO " + KEYSPACE_TABLE + " (k, v) " + "VALUES (?, ?)", ALL, 1, 1); + COORDINATING.flush(KEYSPACE); + Awaitility.waitAtMost(1, TimeUnit.MINUTES).pollDelay(1, TimeUnit.SECONDS) + .until(() -> { + boolean isRepaired = COORDINATING.callOnInstance(() -> { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, TABLE); + Set sstables = cfs.getLiveSSTables(); + if (sstables.size() != 1) + return false; + SSTableReader sstable = sstables.iterator().next(); + return sstable.isRepaired(); + }); + if (!isRepaired) + COORDINATING.forceCompact(KEYSPACE, TABLE); + return isRepaired; + }); + List.of(RECEIVING, MISSING).forEach(instance -> instance.runOnInstance(() -> { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, TABLE); + cfs.truncateBlockingWithoutSnapshot(); + })); + cluster.forEach(instance -> { + // Before repair, peers should have no data + Object[][] rows = instance.executeInternal("SELECT * FROM " + KEYSPACE + ".tbl WHERE k = 1"); + if (instance == COORDINATING) + AssertUtils.assertRows(rows, row(1, 1)); + else + AssertUtils.assertRows(rows); + }); + + // Prevent repair stream from completing + MISSING.runOnInstance(() -> StreamReceiverFailureHelper.shouldWait.set(true)); + + // Run full repair from COORDINATING + { + ExecutorService repairExecutor = Executors.newSingleThreadExecutor(); + Future repair = repairExecutor.submit(() -> COORDINATING.nodetoolResult("repair", "--full", KEYSPACE)); + Awaitility.waitAtMost(10, TimeUnit.SECONDS).pollDelay(1, TimeUnit.SECONDS) + .until(() -> { + int finished = StreamReceiverFailureHelper.getFinishedRepairs(RECEIVING); + return finished > 0; + }); + MISSING.runOnInstance(() -> StreamReceiverFailureHelper.shouldThrow.set(true)); + repair.get(10, TimeUnit.SECONDS).asserts().failure(); + repairExecutor.shutdown(); + } + + // Even after partial repair, RECEIVED should not move its SSTable to the live set, since the repair failed + cluster.forEach(instance -> { + Object[][] rows = instance.executeInternal("SELECT * FROM " + KEYSPACE + ".tbl WHERE k = 1"); + if (instance == COORDINATING) + AssertUtils.assertRows(rows, row(1, 1)); + else + AssertUtils.assertRows(rows); + }); + + /* + At this point, the repair is complete and partially applied. RECEIVED has an SSTable it received from + repair, and MISSING has no SSTables. If we were to do a tracked data read against RECEIVED, we'd have an + emptpy summary but rows, and if we were to execute the same read against MISSING we'd have an entirely + empty response. This would break monotonicity if a client executes a QUORUM read against RECEIVED then + against missing, because the empty summaries lead to no reconciliation happening. + + To provide monotonicity in this scenario, we integrate the full repair with bulk transfer machinery and + tag the SSTables with transfer IDs that can be included in summaries and reconciled. Then, the initial data + read against RECEIVED includes transfer IDs that are reconciled. Reconciliation detects that RECEIVED has an + SSTable that isn't present on MISSING and streams them, so the subsequent read against MISSING is up to + date. + */ + + { + MISSING.runOnInstance(() -> StreamReceiverFailureHelper.shouldWait.set(false)); + MISSING.runOnInstance(() -> StreamReceiverFailureHelper.shouldThrow.set(false)); + // Don't let coordinating act as a replica for the read + cluster.filters().inbound().to(ClusterUtils.instanceId(COORDINATING)).drop(); + MutationTrackingReadReconciliationTest.awaitNodeDead(RECEIVING, COORDINATING); + } + // Repair did not succeed sync, so it did not proceed to activation, so it's not visible on RECEIVING. + { + Object[][] rows = RECEIVING.coordinator().execute("SELECT * FROM " + KEYSPACE + ".tbl WHERE k = 1", QUORUM); + AssertUtils.assertRows(rows); // empty + } + cluster.filters().reset(); + MutationTrackingReadReconciliationTest.awaitNodeAlive(RECEIVING, COORDINATING); + + // Another repair succeeds, all peers should now agree on the local data + long mark = COORDINATING.logs().mark(); + COORDINATING.nodetoolResult("repair", "--full", KEYSPACE).asserts().success(); + List logs = COORDINATING.logs().grep(mark, "Activating .* on ").getResult(); + Assertions.assertThat(logs).isNotEmpty(); + cluster.forEach(instance -> { + Object[][] rows = instance.executeInternal("SELECT * FROM " + KEYSPACE + ".tbl WHERE k = 1"); + AssertUtils.assertRows(rows, row(1, 1)); + }); + + /* + This test will fail periodically because the commitlog has shut down but LogStatePersister wants to update + the system table. + */ + } + } + + public static class StreamReceiverFailureHelper + { + private static final Logger logger = LoggerFactory.getLogger(StreamReceiverFailureHelper.class); + + static AtomicBoolean shouldThrow = new AtomicBoolean(false); + static AtomicBoolean shouldWait = new AtomicBoolean(false); + static AtomicInteger count = new AtomicInteger(0); + + /** + * {@link CassandraStreamReceiver#finished} + */ + public static void install(ClassLoader classLoader, Integer instanceNum) + { + new ByteBuddy().rebase(CassandraStreamReceiver.class) + .method(named("finished").and(takesNoArguments())) + .intercept(to(StreamReceiverFailureHelper.class)) + .make() + .load(classLoader, ClassLoadingStrategy.Default.INJECTION); + } + + public static void run(@SuperCall Callable zuper) throws Exception + { + while (shouldWait.get()) + { + if (shouldThrow.get()) + throw new RuntimeException("Test: failing stream session"); + + logger.info("Test: blocking finish of stream session"); + Thread.sleep(1_000); + } + zuper.call(); + logger.info("Test: finished stream session"); + count.incrementAndGet(); + } + + private static int getFinishedRepairs(IInvokableInstance instance) + { + return instance.callOnInstance(() -> StreamReceiverFailureHelper.count.get()); + } + } + + @Test + public void testFullRepairCleanupOnFailure() throws IOException, ExecutionException, InterruptedException, TimeoutException + { + try (Cluster cluster = Cluster.build(3) + .withInstanceInitializer(StreamReceiverFailureHelper::install) + .withConfig(cfg -> cfg + .with(Feature.NETWORK) + .with(Feature.GOSSIP) + .set("mutation_tracking_enabled", "true")) + .start()) + { + cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked';"); + String TABLE_SCHEMA_CQL = "CREATE TABLE " + KEYSPACE + '.' + TABLE + " (k INT PRIMARY KEY, v INT)"; + cluster.schemaChange(TABLE_SCHEMA_CQL); + + IInvokableInstance COORDINATING = cluster.get(1); + IInvokableInstance RECEIVING = cluster.get(2); + IInvokableInstance MISSING = cluster.get(3); + + // Write a single row to COORDINATING node only + COORDINATING.executeInternal("INSERT INTO " + KEYSPACE_TABLE + " (k, v) VALUES (?, ?)", 1, 100); + + // Before repair, only COORDINATING has data + for (IInvokableInstance instance : cluster) + { + Object[][] rows = instance.executeInternal("SELECT * FROM " + KEYSPACE_TABLE + " WHERE k = 1"); + if (instance == COORDINATING) + AssertUtils.assertRows(rows, row(1, 100)); + else + AssertUtils.assertRows(rows); // empty + } + + // Prevent repair stream from completing + MISSING.runOnInstance(() -> StreamReceiverFailureHelper.shouldWait.set(true)); + + // Run full repair from COORDINATING + { + ExecutorService repairExecutor = Executors.newSingleThreadExecutor(); + Future repair = repairExecutor.submit(() -> COORDINATING.nodetoolResult("repair", "--full", KEYSPACE)); + Awaitility.waitAtMost(10, TimeUnit.SECONDS).pollDelay(1, TimeUnit.SECONDS) + .until(() -> { + int finished = StreamReceiverFailureHelper.getFinishedRepairs(RECEIVING); + return finished > 0; + }); + + // Repair completed against RECEIVING, so it should have pending SSTables + { + List pending = getPendingSSTablePaths(RECEIVING); + Assertions.assertThat(pending).isNotEmpty(); + } + + MISSING.runOnInstance(() -> StreamReceiverFailureHelper.shouldThrow.set(true)); + repair.get(10, TimeUnit.SECONDS).asserts().failure(); + repairExecutor.shutdown(); + } + + // No pending SSTables on COORDINATING because they're streamed from the live set + // No pending SSTables on MISSING because of stream failure injection + // No pending SSTables on RECEIVING because they were cleaned up when the repair failed + for (IInvokableInstance instance : cluster) + { + Object[][] rows = instance.executeInternal("SELECT * FROM " + KEYSPACE_TABLE + " WHERE k = 1"); + if (instance == COORDINATING) + AssertUtils.assertRows(rows, row(1, 100)); + else + AssertUtils.assertRows(rows); // empty + + List pending = getPendingSSTablePaths(instance); + Assertions.assertThat(pending).isEmpty(); + } + } + } + + private static List getPendingSSTablePaths(IInvokableInstance instance) + { + return instance.callOnInstance(() -> { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, TABLE); + Set pendingLocations = cfs.getDirectories().getPendingLocations(); + + List pendingUuidDirs = new ArrayList<>(); + for (File pendingDir : pendingLocations) + { + File[] uuidDirs = pendingDir.listUnchecked(File::isDirectory); + for (File dir : uuidDirs) + pendingUuidDirs.add(dir.absolutePath()); + } + return pendingUuidDirs; + }); + } + + // This should be aligned to a single shard: (-3074457345618258603,3074457345618258601] + private final static long TOKEN_VALUE = 1; + private final static Token TOKEN = new Murmur3Partitioner.LongToken(TOKEN_VALUE); + private final static ByteBuffer KEY = Murmur3Partitioner.LongToken.keyForToken(TOKEN.getLongValue()); + private final static Range SHARD_ALIGNED_RANGE = new Range<>(new Murmur3Partitioner.LongToken(TOKEN_VALUE - 10), new Murmur3Partitioner.LongToken(TOKEN_VALUE + 10)); + static + { + DecoratedKey reversed = Murmur3Partitioner.instance.decorateKey(TrackedKeyspaceRepairSupportTest.KEY); + Assertions.assertThat(reversed.getToken()).isEqualTo(TOKEN); + } + + @Test + public void testFullRepairShardAlignedRangeHappyPath() throws IOException + { + testFullRepair("repair", "--start-token", SHARD_ALIGNED_RANGE.left.toString(), "--end-token", SHARD_ALIGNED_RANGE.right.toString(), "--full", KEYSPACE); + } + + @Test + public void testFullRepairAcrossShardsHappyPath() throws IOException + { + testFullRepair("repair", "--full", KEYSPACE); + } + + public void testFullRepair(String... repairCommandAndArgs) throws IOException + { + try (Cluster cluster = Cluster.build(3) + .withConfig(cfg -> cfg.with(Feature.NETWORK) + .with(Feature.GOSSIP) + .set("mutation_tracking_enabled", "true")) + .start()) + { + cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked';"); + cluster.schemaChange("CREATE TABLE " + KEYSPACE_TABLE + " (pk BLOB PRIMARY KEY, v INT)"); + + IInvokableInstance coordinator = cluster.get(1); + coordinator.executeInternal("INSERT INTO " + KEYSPACE_TABLE + " (pk, v) VALUES (?, 1)", KEY); + + // Write should only be present on instance 1 + cluster.forEach(instance -> { + Object[][] rows = instance.executeInternal("SELECT * FROM " + KEYSPACE_TABLE + " WHERE pk = ?", KEY); + if (ClusterUtils.instanceId(instance) == 1) + AssertUtils.assertRows(rows, row(KEY, 1)); + else + AssertUtils.assertRows(rows); // empty + }); + + long mark = coordinator.logs().mark(); + NodeToolResult result = coordinator.nodetoolResult(repairCommandAndArgs); + result.asserts().success(); + List logs = coordinator.logs().grep(mark, "Created 2 sync tasks based on 3 merkle tree responses").getResult(); + Assertions.assertThat(logs).isNotEmpty(); + + // Write visible on all instances after repair + cluster.forEach(instance -> { + Object[][] rows = instance.executeInternal("SELECT * FROM " + KEYSPACE_TABLE + " WHERE pk = ?", KEY); + AssertUtils.assertRows(rows, row(KEY, 1)); + }); + } + } + + @Test + public void testRepairFailsOnMissedActivation() throws IOException + { + int MISSED_ACTIVATION = 3; + try (Cluster cluster = Cluster.build(3) + .withInstanceInitializer(BulkTransfersTest.ByteBuddyInjections.SkipActivation.install(MISSED_ACTIVATION)) + .withConfig(cfg -> cfg.with(Feature.NETWORK) + .with(Feature.GOSSIP) + .set("mutation_tracking_enabled", "true") + .set("write_request_timeout", "1000ms") + .set("repair_request_timeout", "2s") + .set("stream_transfer_task_timeout", "10s")) + .start()) + { + cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked';"); + cluster.schemaChange("CREATE TABLE " + KEYSPACE_TABLE + " (k INT PRIMARY KEY, v INT)"); + + IInvokableInstance COORDINATING = cluster.get(1); + IInvokableInstance MISSING = cluster.get(MISSED_ACTIVATION); + + COORDINATING.executeInternal("INSERT INTO " + KEYSPACE_TABLE + " (k, v) VALUES (?, ?)", 1, 100); + + // Before repair, only instance 1 has data + for (IInvokableInstance instance : cluster) + { + Object[][] rows = instance.executeInternal("SELECT * FROM " + KEYSPACE_TABLE + " WHERE k = 1"); + if (instance == COORDINATING) + AssertUtils.assertRows(rows, row(1, 100)); + else + AssertUtils.assertRows(rows); // empty + } + + // Repair fails because MISSING blocked activation + BulkTransfersTest.ByteBuddyInjections.SkipActivation.setup(cluster, TransferActivation.Phase.COMMIT, true); + { + NodeToolResult repair = COORDINATING.nodetoolResult("repair", "--full", KEYSPACE); + repair.asserts().failure(); + } + for (IInvokableInstance instance : cluster) + { + Object[][] rows = instance.executeInternal("SELECT * FROM " + KEYSPACE_TABLE + " WHERE k = 1"); + List pending = getPendingSSTablePaths(instance); + if (instance == MISSING) + { + AssertUtils.assertRows(rows); // empty + Assertions.assertThat(pending).isNotEmpty(); + } + else + { + AssertUtils.assertRows(rows, row(1, 100)); + Assertions.assertThat(pending).isEmpty(); + } + } + + // Re-enable activation; read reconciliation at ALL should activate the pending SSTables on MISSING + BulkTransfersTest.ByteBuddyInjections.SkipActivation.setup(cluster, null); + COORDINATING.coordinator().execute("SELECT * FROM " + KEYSPACE_TABLE + " WHERE k = 1", ALL); + for (IInvokableInstance instance : cluster) + { + Object[][] rows = instance.executeInternal("SELECT * FROM " + KEYSPACE_TABLE + " WHERE k = 1"); + List pending = getPendingSSTablePaths(instance); + AssertUtils.assertRows(rows, row(1, 100)); + Assertions.assertThat(pending).isEmpty(); + } + } + } +} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/distributed/test/tracking/BulkTransfersTest.java b/test/distributed/org/apache/cassandra/distributed/test/tracking/BulkTransfersTest.java index 2c85dd81b961..60e6b2a7d13c 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/tracking/BulkTransfersTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/tracking/BulkTransfersTest.java @@ -623,6 +623,8 @@ public static class SkipActivation // null to not skip public static volatile TransferActivation.Phase phase; + public static volatile boolean throwOnActivation = false; + public static IInstanceInitializer install(int...nodes) { return (ClassLoader cl, ThreadGroup tg, int num, int generation) -> { @@ -639,8 +641,16 @@ public static IInstanceInitializer install(int...nodes) // Need to set phase in each instance's classloader, otherwise assignment won't be visible to injected method body public static void setup(Cluster cluster, TransferActivation.Phase phase) { - logger.debug("Setting up phase {}", phase); - cluster.forEach(instance -> instance.runOnInstance(() -> ByteBuddyInjections.SkipActivation.phase = phase)); + setup(cluster, phase, false); + } + + public static void setup(Cluster cluster, TransferActivation.Phase phase, boolean throwOnActivation) + { + logger.debug("Setting up phase {}, throwOnActivation {}", phase, throwOnActivation); + cluster.forEach(instance -> instance.runOnInstance(() -> { + SkipActivation.phase = phase; + SkipActivation.throwOnActivation = throwOnActivation; + })); } @SuppressWarnings("unused") @@ -648,8 +658,17 @@ public static void doVerb(Message msg, @SuperCall Callable future = new CompletableFuture<>(); session.registerSyncCompleteCallback(future::get); - ListenableFuture> syncResults = job.executeTasks(syncTasks); + ListenableFuture> syncResults = job.executeTasks(SyncTasks.untracked(syncTasks)); // Immediately following execution the internal execution queue should still retain the trees long sizeDuringExecution = ObjectSizes.measureDeep(session); @@ -903,7 +903,7 @@ private void interceptRepairMessages(Map mockTr break; case SYNC_REQ: SyncRequest syncRequest = (SyncRequest) message.payload; - session.syncComplete(sessionJobDesc, Message.builder(Verb.SYNC_RSP, new SyncResponse(sessionJobDesc, new SyncNodePair(syncRequest.src, syncRequest.dst), true, Collections.emptyList())).from(to).build()); + session.syncComplete(sessionJobDesc, Message.builder(Verb.SYNC_RSP, new SyncResponse(sessionJobDesc, new SyncNodePair(syncRequest.src, syncRequest.dst), true, Collections.emptyList(), null)).from(to).build()); break; default: break; diff --git a/test/unit/org/apache/cassandra/repair/SymmetricRemoteSyncTaskTest.java b/test/unit/org/apache/cassandra/repair/SymmetricRemoteSyncTaskTest.java index 37d4e09e78a1..ff840391c2a7 100644 --- a/test/unit/org/apache/cassandra/repair/SymmetricRemoteSyncTaskTest.java +++ b/test/unit/org/apache/cassandra/repair/SymmetricRemoteSyncTaskTest.java @@ -41,7 +41,7 @@ private static class InstrumentedSymmetricRemoteSyncTask extends SymmetricRemote { public InstrumentedSymmetricRemoteSyncTask(InetAddressAndPort e1, InetAddressAndPort e2) { - super(SharedContext.Global.instance, DESC, e1, e2, RANGE_LIST, PreviewKind.NONE); + super(SharedContext.Global.instance, DESC, e1, e2, RANGE_LIST, PreviewKind.NONE, null); } RepairMessage sentMessage = null; diff --git a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java index f15b2a618265..8f460f1367b6 100644 --- a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java +++ b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java @@ -183,7 +183,7 @@ public void syncCompleteMessage() throws IOException Lists.newArrayList(new StreamSummary(TableId.fromUUID(UUID.randomUUID()), emptyList(), 5, 100)), Lists.newArrayList(new StreamSummary(TableId.fromUUID(UUID.randomUUID()), emptyList(), 500, 10)) )); - SyncResponse msg = new SyncResponse(buildRepairJobDesc(), new SyncNodePair(src, dst), true, summaries); + SyncResponse msg = new SyncResponse(buildRepairJobDesc(), new SyncNodePair(src, dst), true, summaries, null); serializeRoundTrip(msg, SyncResponse.serializer); } diff --git a/test/unit/org/apache/cassandra/replication/AlignedToShardBoundariesTest.java b/test/unit/org/apache/cassandra/replication/AlignedToShardBoundariesTest.java new file mode 100644 index 000000000000..f52b91bd51e8 --- /dev/null +++ b/test/unit/org/apache/cassandra/replication/AlignedToShardBoundariesTest.java @@ -0,0 +1,218 @@ +/* + * 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.replication; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.dht.ByteOrderedPartitioner; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.utils.ByteBufferUtil; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class AlignedToShardBoundariesTest +{ + private static final String TEST_KEYSPACE = "test_ks"; + + @BeforeClass + public static void setUp() throws IOException + { + SchemaLoader.prepareServer(); + } + + private static Token tk(String key) + { + return new ByteOrderedPartitioner.BytesToken(ByteBufferUtil.bytes(key)); + } + + private static Range range(String left, String right) + { + return new Range<>(tk(left), tk(right)); + } + + @Test + public void testNoShardsReturnsRangesAsSingleGroup() + { + MutationTrackingService service = MutationTrackingService.TestAccess.create(); + + // No shards set for keyspace + Collection> inputRanges = Arrays.asList( + range("a", "m"), + range("n", "z") + ); + + List>> result = service.alignedToShardBoundaries(TEST_KEYSPACE, inputRanges); + + // Should return all ranges as a single group + assertEquals(1, result.size()); + assertEquals(2, result.get(0).size()); + assertTrue(result.get(0).contains(range("a", "m"))); + assertTrue(result.get(0).contains(range("n", "z"))); + } + + @Test + public void testSingleRangeWithinSingleShard() + { + MutationTrackingService service = MutationTrackingService.TestAccess.create(); + + // Create a single shard covering a-z + Set> shardRanges = new HashSet<>(); + shardRanges.add(range("a", "z")); + MutationTrackingService.KeyspaceShards shards = + MutationTrackingService.TestAccess.createTestKeyspaceShards(TEST_KEYSPACE, shardRanges); + MutationTrackingService.TestAccess.setKeyspaceShards(service, TEST_KEYSPACE, shards); + + // Input range completely within the shard + Collection> inputRanges = Collections.singletonList(range("d", "m")); + + List>> result = service.alignedToShardBoundaries(TEST_KEYSPACE, inputRanges); + + // Should return the range unchanged in a single group + assertEquals(1, result.size()); + assertEquals(1, result.get(0).size()); + assertEquals(range("d", "m"), result.get(0).get(0)); + } + + @Test + public void testSingleRangeSpanningMultipleShards() + { + MutationTrackingService service = MutationTrackingService.TestAccess.create(); + + // Create two shards + Set> shardRanges = new HashSet<>(); + shardRanges.add(range("a", "m")); + shardRanges.add(range("m", "z")); + MutationTrackingService.KeyspaceShards shards = + MutationTrackingService.TestAccess.createTestKeyspaceShards(TEST_KEYSPACE, shardRanges); + MutationTrackingService.TestAccess.setKeyspaceShards(service, TEST_KEYSPACE, shards); + + // Input range spans both shards + Collection> inputRanges = Collections.singletonList(range("d", "s")); + + List>> result = service.alignedToShardBoundaries(TEST_KEYSPACE, inputRanges); + + // Should be split into two groups, one per shard + assertEquals(2, result.size()); + + // Collect all split ranges + List> allRanges = new ArrayList<>(); + result.forEach(allRanges::addAll); + assertEquals(2, allRanges.size()); + + // Should contain the two split pieces + assertTrue(allRanges.contains(range("d", "m"))); + assertTrue(allRanges.contains(range("m", "s"))); + } + + @Test + public void testMultipleRangesWithinSameShard() + { + MutationTrackingService service = MutationTrackingService.TestAccess.create(); + + // Create a single shard + Set> shardRanges = new HashSet<>(); + shardRanges.add(range("a", "z")); + MutationTrackingService.KeyspaceShards shards = + MutationTrackingService.TestAccess.createTestKeyspaceShards(TEST_KEYSPACE, shardRanges); + MutationTrackingService.TestAccess.setKeyspaceShards(service, TEST_KEYSPACE, shards); + + // Multiple ranges all within the same shard + Collection> inputRanges = Arrays.asList( + range("b", "d"), + range("e", "g"), + range("h", "j") + ); + + List>> result = service.alignedToShardBoundaries(TEST_KEYSPACE, inputRanges); + + // All ranges should be in a single group (same shard) + assertEquals(1, result.size()); + assertEquals(3, result.get(0).size()); + assertTrue(result.get(0).contains(range("b", "d"))); + assertTrue(result.get(0).contains(range("e", "g"))); + assertTrue(result.get(0).contains(range("h", "j"))); + } + + @Test + public void testEmptyRangesReturnsEmptyOrSingleEmptyGroup() + { + MutationTrackingService service = MutationTrackingService.TestAccess.create(); + + // Create a shard + Set> shardRanges = new HashSet<>(); + shardRanges.add(range("a", "z")); + MutationTrackingService.KeyspaceShards shards = + MutationTrackingService.TestAccess.createTestKeyspaceShards(TEST_KEYSPACE, shardRanges); + MutationTrackingService.TestAccess.setKeyspaceShards(service, TEST_KEYSPACE, shards); + + // Empty input + Collection> inputRanges = Collections.emptyList(); + + List>> result = service.alignedToShardBoundaries(TEST_KEYSPACE, inputRanges); + + // Should return empty result (no groups with ranges) + assertTrue(result.isEmpty() || (result.size() == 1 && result.get(0).isEmpty())); + } + + @Test + public void testMultipleRangesAcrossMultipleShards() + { + MutationTrackingService service = MutationTrackingService.TestAccess.create(); + + // Create three shards + Set> shardRanges = new HashSet<>(); + shardRanges.add(range("a", "h")); + shardRanges.add(range("h", "p")); + shardRanges.add(range("p", "z")); + MutationTrackingService.KeyspaceShards shards = + MutationTrackingService.TestAccess.createTestKeyspaceShards(TEST_KEYSPACE, shardRanges); + MutationTrackingService.TestAccess.setKeyspaceShards(service, TEST_KEYSPACE, shards); + + // Multiple ranges, some spanning shards + Collection> inputRanges = Arrays.asList( + range("b", "e"), // Within shard 1 + range("f", "j"), // Spans shard 1 and 2 + range("q", "s") // Within shard 3 + ); + + List>> result = service.alignedToShardBoundaries(TEST_KEYSPACE, inputRanges); + + // Should have 3 groups (one per shard) + assertEquals(3, result.size()); + + // Collect all split ranges + List> allRanges = new ArrayList<>(); + result.forEach(allRanges::addAll); + + // Should have: b-e, f-h (shard 1), h-j (shard 2), q-s (shard 3) = 4 ranges + assertEquals(4, allRanges.size()); + } +} diff --git a/test/unit/org/apache/cassandra/replication/LocalTransfersTest.java b/test/unit/org/apache/cassandra/replication/LocalTransfersTest.java index 465f280c4928..807ee198869c 100644 --- a/test/unit/org/apache/cassandra/replication/LocalTransfersTest.java +++ b/test/unit/org/apache/cassandra/replication/LocalTransfersTest.java @@ -33,13 +33,13 @@ import org.apache.cassandra.utils.TimeUUID; import org.assertj.core.api.Assertions; -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.COMMITTED; -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.COMMITTING; -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.PREPARE_FAILED; -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.PREPARING; -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.STREAM_COMPLETE; -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.STREAM_FAILED; -import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.STREAM_NOOP; +import static org.apache.cassandra.replication.AbstractCoordinatedBulkTransfer.SingleTransferResult.State.COMMITTED; +import static org.apache.cassandra.replication.AbstractCoordinatedBulkTransfer.SingleTransferResult.State.COMMITTING; +import static org.apache.cassandra.replication.AbstractCoordinatedBulkTransfer.SingleTransferResult.State.PREPARE_FAILED; +import static org.apache.cassandra.replication.AbstractCoordinatedBulkTransfer.SingleTransferResult.State.PREPARING; +import static org.apache.cassandra.replication.AbstractCoordinatedBulkTransfer.SingleTransferResult.State.STREAM_COMPLETE; +import static org.apache.cassandra.replication.AbstractCoordinatedBulkTransfer.SingleTransferResult.State.STREAM_FAILED; +import static org.apache.cassandra.replication.AbstractCoordinatedBulkTransfer.SingleTransferResult.State.STREAM_NOOP; import static org.apache.cassandra.utils.TimeUUID.Generator.nextTimeUUID; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -66,17 +66,17 @@ public void setUp() throws Exception planId = nextTimeUUID(); } - private CoordinatedTransfer coordinatedTransfer(ShortMutationId transferId) + private TrackedImportTransfer coordinatedTransfer(ShortMutationId transferId) { return coordinatedTransfer(transferId, new Range<>(tk(0), tk(1000))); } - private CoordinatedTransfer coordinatedTransfer(ShortMutationId transferId, Range range) + private TrackedImportTransfer coordinatedTransfer(ShortMutationId transferId, Range range) { MutationId mutationId = transferId != null ? new MutationId(transferId.logId(), transferId.offset(), (int) System.currentTimeMillis()) : null; - return new CoordinatedTransfer(range, mutationId); + return new TrackedImportTransfer(range, mutationId); } private PendingLocalTransfer pendingTransfer(TimeUUID planId) @@ -95,12 +95,12 @@ private static Token tk(long token) @Test public void testSaveCoordinatedTransfer() { - CoordinatedTransfer transfer = coordinatedTransfer(transferId); + TrackedImportTransfer transfer = coordinatedTransfer(transferId); localTransfers.save(transfer); localTransfers.activating(transfer); - CoordinatedTransfer loaded = localTransfers.getActivatedTransfer(transferId); + AbstractCoordinatedBulkTransfer loaded = localTransfers.getActivatedTransfer(transferId); Assertions.assertThat(loaded).isEqualTo(transfer); assertThatThrownBy(() -> localTransfers.save(transfer)) @@ -110,12 +110,12 @@ public void testSaveCoordinatedTransfer() @Test public void testActivatingTransfer() { - CoordinatedTransfer transfer = coordinatedTransfer(transferId); + TrackedImportTransfer transfer = coordinatedTransfer(transferId); localTransfers.save(transfer); localTransfers.activating(transfer); - CoordinatedTransfer retrieved = localTransfers.getActivatedTransfer(transferId); + AbstractCoordinatedBulkTransfer retrieved = localTransfers.getActivatedTransfer(transferId); assertThat(retrieved).isEqualTo(transfer); } @@ -146,17 +146,17 @@ public void testGetPendingTransferNotFound() @Test public void testGetActivatedTransferNotFound() { - CoordinatedTransfer retrieved = localTransfers.getActivatedTransfer(transferId); + AbstractCoordinatedBulkTransfer retrieved = localTransfers.getActivatedTransfer(transferId); assertThat(retrieved).isNull(); } @Test public void testPurgingTransferNotStarted() { - CoordinatedTransfer transfer = coordinatedTransfer(transferId); + TrackedImportTransfer transfer = coordinatedTransfer(transferId); // All streams in INIT state - should NOT be purgeable (stream hasn't started yet) - CoordinatedTransfer.SingleTransferResult result = CoordinatedTransfer.SingleTransferResult.Init(); + TrackedImportTransfer.SingleTransferResult result = TrackedImportTransfer.SingleTransferResult.Init(); transfer.streamResults.put(mock(InetAddressAndPort.class), result); Assertions.assertThat(localTransfers.purger.test(transfer)).isFalse(); @@ -165,11 +165,11 @@ public void testPurgingTransferNotStarted() @Test public void testPurgingTransferAllStreamsComplete() { - CoordinatedTransfer transfer = coordinatedTransfer(transferId); + TrackedImportTransfer transfer = coordinatedTransfer(transferId); // All streams in STREAM_COMPLETE state - should NOT be purgeable (no failures) - CoordinatedTransfer.SingleTransferResult result1 = CoordinatedTransfer.SingleTransferResult.StreamComplete(nextTimeUUID()); - CoordinatedTransfer.SingleTransferResult result2 = CoordinatedTransfer.SingleTransferResult.StreamComplete(nextTimeUUID()); + TrackedImportTransfer.SingleTransferResult result1 = TrackedImportTransfer.SingleTransferResult.StreamComplete(nextTimeUUID()); + TrackedImportTransfer.SingleTransferResult result2 = TrackedImportTransfer.SingleTransferResult.StreamComplete(nextTimeUUID()); transfer.streamResults.put(mock(InetAddressAndPort.class), result1); transfer.streamResults.put(mock(InetAddressAndPort.class), result2); @@ -180,10 +180,10 @@ public void testPurgingTransferAllStreamsComplete() @Test public void testPurgingTransferPrepareFailed() { - CoordinatedTransfer transfer = coordinatedTransfer(transferId); + TrackedImportTransfer transfer = coordinatedTransfer(transferId); - CoordinatedTransfer.SingleTransferResult result1 = new CoordinatedTransfer.SingleTransferResult(PREPARE_FAILED, planId); - CoordinatedTransfer.SingleTransferResult result2 = new CoordinatedTransfer.SingleTransferResult(PREPARING, planId); + TrackedImportTransfer.SingleTransferResult result1 = new TrackedImportTransfer.SingleTransferResult(PREPARE_FAILED, planId); + TrackedImportTransfer.SingleTransferResult result2 = new TrackedImportTransfer.SingleTransferResult(PREPARING, planId); transfer.streamResults.put(mock(InetAddressAndPort.class), result1); transfer.streamResults.put(mock(InetAddressAndPort.class), result2); @@ -194,11 +194,11 @@ public void testPurgingTransferPrepareFailed() @Test public void testPurgingTransferAllActivationCommitted() { - CoordinatedTransfer transfer = coordinatedTransfer(transferId); + TrackedImportTransfer transfer = coordinatedTransfer(transferId); // All streams in ACTIVATE_COMMITTED state - should be purgeable (allComplete = true) - CoordinatedTransfer.SingleTransferResult result1 = new CoordinatedTransfer.SingleTransferResult(COMMITTED, planId); - CoordinatedTransfer.SingleTransferResult result2 = new CoordinatedTransfer.SingleTransferResult(COMMITTED, planId); + TrackedImportTransfer.SingleTransferResult result1 = new TrackedImportTransfer.SingleTransferResult(COMMITTED, planId); + TrackedImportTransfer.SingleTransferResult result2 = new TrackedImportTransfer.SingleTransferResult(COMMITTED, planId); transfer.streamResults.put(mock(InetAddressAndPort.class), result1); transfer.streamResults.put(mock(InetAddressAndPort.class), result2); @@ -209,11 +209,11 @@ public void testPurgingTransferAllActivationCommitted() @Test public void testPurgingTransferMixedCommittedAndNoop() { - CoordinatedTransfer transfer = coordinatedTransfer(transferId); + TrackedImportTransfer transfer = coordinatedTransfer(transferId); // Mix of ACTIVATE_COMMITTED and STREAM_NOOP - should be purgeable (allComplete = true) - CoordinatedTransfer.SingleTransferResult result1 = new CoordinatedTransfer.SingleTransferResult(COMMITTED, planId); - CoordinatedTransfer.SingleTransferResult result2 = new CoordinatedTransfer.SingleTransferResult(STREAM_NOOP, null); + TrackedImportTransfer.SingleTransferResult result1 = new TrackedImportTransfer.SingleTransferResult(COMMITTED, planId); + TrackedImportTransfer.SingleTransferResult result2 = new TrackedImportTransfer.SingleTransferResult(STREAM_NOOP, null); transfer.streamResults.put(mock(InetAddressAndPort.class), result1); transfer.streamResults.put(mock(InetAddressAndPort.class), result2); @@ -224,11 +224,11 @@ public void testPurgingTransferMixedCommittedAndNoop() @Test public void testPurgingTransferActivationPartialCommitted() { - CoordinatedTransfer transfer = coordinatedTransfer(transferId); + TrackedImportTransfer transfer = coordinatedTransfer(transferId); // One stream in ACTIVATE_PREPARING - should NOT be purgeable - CoordinatedTransfer.SingleTransferResult result1 = new CoordinatedTransfer.SingleTransferResult(PREPARING, planId); - CoordinatedTransfer.SingleTransferResult result2 = new CoordinatedTransfer.SingleTransferResult(COMMITTING, planId); + TrackedImportTransfer.SingleTransferResult result1 = new TrackedImportTransfer.SingleTransferResult(PREPARING, planId); + TrackedImportTransfer.SingleTransferResult result2 = new TrackedImportTransfer.SingleTransferResult(COMMITTING, planId); transfer.streamResults.put(mock(InetAddressAndPort.class), result1); transfer.streamResults.put(mock(InetAddressAndPort.class), result2); @@ -239,11 +239,11 @@ public void testPurgingTransferActivationPartialCommitted() @Test public void testPurgingTransferAllStreamsFailed() { - CoordinatedTransfer transfer = coordinatedTransfer(transferId); + TrackedImportTransfer transfer = coordinatedTransfer(transferId); // All streams in STREAM_FAILED state - should be purgeable (noneActivated = true) - CoordinatedTransfer.SingleTransferResult result1 = new CoordinatedTransfer.SingleTransferResult(STREAM_FAILED, planId); - CoordinatedTransfer.SingleTransferResult result2 = new CoordinatedTransfer.SingleTransferResult(STREAM_FAILED, planId); + TrackedImportTransfer.SingleTransferResult result1 = new TrackedImportTransfer.SingleTransferResult(STREAM_FAILED, planId); + TrackedImportTransfer.SingleTransferResult result2 = new TrackedImportTransfer.SingleTransferResult(STREAM_FAILED, planId); transfer.streamResults.put(mock(InetAddressAndPort.class), result1); transfer.streamResults.put(mock(InetAddressAndPort.class), result2); @@ -254,11 +254,11 @@ public void testPurgingTransferAllStreamsFailed() @Test public void testPurgingTransferMixedInitAndFailed() { - CoordinatedTransfer transfer = coordinatedTransfer(transferId); + TrackedImportTransfer transfer = coordinatedTransfer(transferId); // Mix of INIT and STREAM_FAILED - should be purgeable (has failure, none activated) - CoordinatedTransfer.SingleTransferResult result1 = CoordinatedTransfer.SingleTransferResult.Init(); - CoordinatedTransfer.SingleTransferResult result2 = CoordinatedTransfer.SingleTransferResult.Init().streamFailed(nextTimeUUID()); + TrackedImportTransfer.SingleTransferResult result1 = TrackedImportTransfer.SingleTransferResult.Init(); + TrackedImportTransfer.SingleTransferResult result2 = TrackedImportTransfer.SingleTransferResult.Init().streamFailed(nextTimeUUID()); transfer.streamResults.put(mock(InetAddressAndPort.class), result1); transfer.streamResults.put(mock(InetAddressAndPort.class), result2); @@ -269,11 +269,11 @@ public void testPurgingTransferMixedInitAndFailed() @Test public void testPurgingTransferMixedCompleteAndFailed() { - CoordinatedTransfer transfer = coordinatedTransfer(transferId); + TrackedImportTransfer transfer = coordinatedTransfer(transferId); // Mix of STREAM_COMPLETE and STREAM_FAILED - should be purgeable (has failure, none activated) - CoordinatedTransfer.SingleTransferResult result1 = CoordinatedTransfer.SingleTransferResult.StreamComplete(nextTimeUUID()); - CoordinatedTransfer.SingleTransferResult result2 = CoordinatedTransfer.SingleTransferResult.Init().streamFailed(nextTimeUUID()); + TrackedImportTransfer.SingleTransferResult result1 = TrackedImportTransfer.SingleTransferResult.StreamComplete(nextTimeUUID()); + TrackedImportTransfer.SingleTransferResult result2 = TrackedImportTransfer.SingleTransferResult.Init().streamFailed(nextTimeUUID()); transfer.streamResults.put(mock(InetAddressAndPort.class), result1); transfer.streamResults.put(mock(InetAddressAndPort.class), result2); @@ -284,12 +284,12 @@ public void testPurgingTransferMixedCompleteAndFailed() @Test public void testPurgingTransferMixedStreamingCompleteAndPreparing() { - CoordinatedTransfer transfer = coordinatedTransfer(transferId); + TrackedImportTransfer transfer = coordinatedTransfer(transferId); // Mix of STREAM_COMPLETE and ACTIVATE_PREPARING - should NOT be purgeable // (noneActivated = false because of ACTIVATE_PREPARING, allComplete = false) - CoordinatedTransfer.SingleTransferResult result1 = new CoordinatedTransfer.SingleTransferResult(STREAM_COMPLETE, planId); - CoordinatedTransfer.SingleTransferResult result2 = new CoordinatedTransfer.SingleTransferResult(PREPARING, planId); + TrackedImportTransfer.SingleTransferResult result1 = new TrackedImportTransfer.SingleTransferResult(STREAM_COMPLETE, planId); + TrackedImportTransfer.SingleTransferResult result2 = new TrackedImportTransfer.SingleTransferResult(PREPARING, planId); transfer.streamResults.put(mock(InetAddressAndPort.class), result1); transfer.streamResults.put(mock(InetAddressAndPort.class), result2); @@ -300,10 +300,10 @@ public void testPurgingTransferMixedStreamingCompleteAndPreparing() @Test public void testPurgingTransferMixedCommittingCommitted() { - CoordinatedTransfer transfer = coordinatedTransfer(transferId); + TrackedImportTransfer transfer = coordinatedTransfer(transferId); - CoordinatedTransfer.SingleTransferResult result1 = new CoordinatedTransfer.SingleTransferResult(COMMITTING, planId); - CoordinatedTransfer.SingleTransferResult result2 = new CoordinatedTransfer.SingleTransferResult(COMMITTED, planId); + TrackedImportTransfer.SingleTransferResult result1 = new TrackedImportTransfer.SingleTransferResult(COMMITTING, planId); + TrackedImportTransfer.SingleTransferResult result2 = new TrackedImportTransfer.SingleTransferResult(COMMITTED, planId); transfer.streamResults.put(mock(InetAddressAndPort.class), result1); transfer.streamResults.put(mock(InetAddressAndPort.class), result2); @@ -314,11 +314,11 @@ public void testPurgingTransferMixedCommittingCommitted() @Test public void testPurgingTransferWithNullTransferId() { - CoordinatedTransfer transfer = coordinatedTransfer(null); + TrackedImportTransfer transfer = coordinatedTransfer(null); // All streams complete but transferId is null - should NOT be purgeable - CoordinatedTransfer.SingleTransferResult result1 = new CoordinatedTransfer.SingleTransferResult(STREAM_COMPLETE, null); - CoordinatedTransfer.SingleTransferResult result2 = new CoordinatedTransfer.SingleTransferResult(STREAM_COMPLETE, null); + TrackedImportTransfer.SingleTransferResult result1 = new TrackedImportTransfer.SingleTransferResult(STREAM_COMPLETE, null); + TrackedImportTransfer.SingleTransferResult result2 = new TrackedImportTransfer.SingleTransferResult(STREAM_COMPLETE, null); transfer.streamResults.put(mock(InetAddressAndPort.class), result1); transfer.streamResults.put(mock(InetAddressAndPort.class), result2); @@ -330,11 +330,11 @@ public void testPurgingTransferWithNullTransferId() @Test public void testPurgingTransferNoopOnly() { - CoordinatedTransfer transfer = coordinatedTransfer(transferId); + TrackedImportTransfer transfer = coordinatedTransfer(transferId); // All streams in STREAM_NOOP - should be purgeable (both noneActivated and allComplete are true) - CoordinatedTransfer.SingleTransferResult result1 = CoordinatedTransfer.SingleTransferResult.Noop(); - CoordinatedTransfer.SingleTransferResult result2 = CoordinatedTransfer.SingleTransferResult.Noop(); + TrackedImportTransfer.SingleTransferResult result1 = TrackedImportTransfer.SingleTransferResult.Noop(); + TrackedImportTransfer.SingleTransferResult result2 = TrackedImportTransfer.SingleTransferResult.Noop(); transfer.streamResults.put(mock(InetAddressAndPort.class), result1); transfer.streamResults.put(mock(InetAddressAndPort.class), result2); diff --git a/test/unit/org/apache/cassandra/service/SerializationsTest.java b/test/unit/org/apache/cassandra/service/SerializationsTest.java index 00864f5a3434..288e528c2063 100644 --- a/test/unit/org/apache/cassandra/service/SerializationsTest.java +++ b/test/unit/org/apache/cassandra/service/SerializationsTest.java @@ -228,9 +228,9 @@ private void testSyncCompleteWrite() throws IOException Lists.newArrayList(new StreamSummary(TABLE_ID, emptyList(), 5, 100)), Lists.newArrayList(new StreamSummary(TABLE_ID, emptyList(), 500, 10)) )); - SyncResponse success = new SyncResponse(DESC, src, dest, true, summaries); + SyncResponse success = new SyncResponse(DESC, src, dest, true, summaries, null); // sync fail - SyncResponse fail = new SyncResponse(DESC, src, dest, false, emptyList()); + SyncResponse fail = new SyncResponse(DESC, src, dest, false, emptyList(), null); testRepairMessageWrite("service.SyncComplete.bin", SyncResponse.serializer, success, fail); } From 673e86f779b9b910162edc4e908abbac9d260eda Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe Date: Fri, 23 Jan 2026 18:20:49 -0600 Subject: [PATCH 2/2] Clean up the streamResults duplication between TrackedImportTransfer and AbstractCoordinatedBulkTransfer --- .../org/apache/cassandra/repair/SyncTask.java | 1 - .../AbstractCoordinatedBulkTransfer.java | 6 ++++++ .../replication/TrackedImportTransfer.java | 19 ++----------------- 3 files changed, 8 insertions(+), 18 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/SyncTask.java b/src/java/org/apache/cassandra/repair/SyncTask.java index f11752c60598..843ee66e69e9 100644 --- a/src/java/org/apache/cassandra/repair/SyncTask.java +++ b/src/java/org/apache/cassandra/repair/SyncTask.java @@ -36,7 +36,6 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.repair.messages.RepairMessage; import org.apache.cassandra.repair.messages.SyncRequest; -import org.apache.cassandra.replication.MutationId; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.TimeUUID; diff --git a/src/java/org/apache/cassandra/replication/AbstractCoordinatedBulkTransfer.java b/src/java/org/apache/cassandra/replication/AbstractCoordinatedBulkTransfer.java index 9978e4b6d20e..89c6af829fde 100644 --- a/src/java/org/apache/cassandra/replication/AbstractCoordinatedBulkTransfer.java +++ b/src/java/org/apache/cassandra/replication/AbstractCoordinatedBulkTransfer.java @@ -65,6 +65,12 @@ public AbstractCoordinatedBulkTransfer(ShortMutationId id) this.streamResults = new ConcurrentHashMap<>(); } + public AbstractCoordinatedBulkTransfer(ShortMutationId id, Participants participants) + { + this.id = id; + this.streamResults = new ConcurrentHashMap<>(participants.size()); + } + ShortMutationId id() { return id; diff --git a/src/java/org/apache/cassandra/replication/TrackedImportTransfer.java b/src/java/org/apache/cassandra/replication/TrackedImportTransfer.java index 38711b4451b3..a6a39bc52bd5 100644 --- a/src/java/org/apache/cassandra/replication/TrackedImportTransfer.java +++ b/src/java/org/apache/cassandra/replication/TrackedImportTransfer.java @@ -27,8 +27,6 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -107,8 +105,6 @@ public class TrackedImportTransfer extends AbstractCoordinatedBulkTransfer private final Range range; final Collection sstables; private final ConsistencyLevel cl; - // TODO: Refactor to new class PendingTransfers - final ConcurrentMap streamResults; @VisibleForTesting TrackedImportTransfer(Range range, MutationId id) @@ -118,19 +114,18 @@ public class TrackedImportTransfer extends AbstractCoordinatedBulkTransfer this.range = range; this.sstables = Collections.emptyList(); this.cl = null; - this.streamResults = new ConcurrentHashMap<>(); } TrackedImportTransfer(String keyspace, Range range, Participants participants, Collection sstables, ConsistencyLevel cl, Supplier nextId) { - super(nextId.get()); + super(nextId.get(), participants); this.keyspace = keyspace; this.range = range; this.sstables = sstables; this.cl = cl; ClusterMetadata cm = ClusterMetadata.current(); - this.streamResults = new ConcurrentHashMap<>(participants.size()); + for (int i = 0; i < participants.size(); i++) { InetAddressAndPort addr = cm.directory.getNodeAddresses(new NodeId(participants.get(i))).broadcastAddress; @@ -308,16 +303,6 @@ else if (sufficient()) logger.debug("{} Nothing to activate", logPrefix()); } - public boolean isCommitted() - { - for (SingleTransferResult result : streamResults.values()) - { - if (result.state != COMMITTED) - return false; - } - return true; - } - private SingleTransferResult streamTask(InetAddressAndPort to) throws StreamException, ExecutionException, InterruptedException, TimeoutException { StreamPlan plan = new StreamPlan(StreamOperation.TRACKED_TRANSFER);