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..843ee66e69e9 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; @@ -36,6 +38,7 @@ import org.apache.cassandra.repair.messages.SyncRequest; 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 +53,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 +67,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..89c6af829fde --- /dev/null +++ b/src/java/org/apache/cassandra/replication/AbstractCoordinatedBulkTransfer.java @@ -0,0 +1,404 @@ +/* + * 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<>(); + } + + public AbstractCoordinatedBulkTransfer(ShortMutationId id, Participants participants) + { + this.id = id; + this.streamResults = new ConcurrentHashMap<>(participants.size()); + } + + 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..a6a39bc52bd5 --- /dev/null +++ b/src/java/org/apache/cassandra/replication/TrackedImportTransfer.java @@ -0,0 +1,374 @@ +/* + * 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.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; + + @VisibleForTesting + TrackedImportTransfer(Range range, MutationId id) + { + super(id); + this.keyspace = null; + this.range = range; + this.sstables = Collections.emptyList(); + this.cl = null; + } + + TrackedImportTransfer(String keyspace, Range range, Participants participants, Collection sstables, ConsistencyLevel cl, Supplier nextId) + { + super(nextId.get(), participants); + this.keyspace = keyspace; + this.range = range; + this.sstables = sstables; + this.cl = cl; + + ClusterMetadata cm = ClusterMetadata.current(); + + 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()); + } + + 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); }