diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index 8dcc01a0e007..7b3917439575 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -914,7 +914,7 @@ public List importNewSSTables(Set srcPaths, boolean resetLevel, .build()); } - Descriptor getUniqueDescriptorFor(Descriptor descriptor, File targetDirectory) + public Descriptor getUniqueDescriptorFor(Descriptor descriptor, File targetDirectory) { Descriptor newDescriptor; do diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java index 67b87d14e97c..e415c7737576 100644 --- a/src/java/org/apache/cassandra/db/Directories.java +++ b/src/java/org/apache/cassandra/db/Directories.java @@ -73,6 +73,7 @@ import org.apache.cassandra.service.snapshot.SnapshotManifest; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.TimeUUID; import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; @@ -115,6 +116,7 @@ public class Directories public static final String BACKUPS_SUBDIR = "backups"; public static final String SNAPSHOT_SUBDIR = "snapshots"; + public static final String PENDING_SUBDIR = "pending"; public static final String TMP_SUBDIR = "tmp"; public static final String SECONDARY_INDEX_NAME_SEPARATOR = "."; @@ -316,13 +318,8 @@ public File getLocationForDisk(DataDirectory dataDirectory) { if (dataDirectory != null) for (File dir : dataPaths) - { - // Note that we must compare absolute paths (not canonical) here since keyspace directories might be symlinks - Path dirPath = dir.toAbsolute().toPath(); - Path locationPath = dataDirectory.location.toAbsolute().toPath(); - if (dirPath.startsWith(locationPath)) + if (dataDirectory.contains(dir)) return dir; - } return null; } @@ -727,6 +724,33 @@ public static File getSnapshotSchemaFile(File snapshotDir) return new File(snapshotDir, "schema.cql"); } + @VisibleForTesting + public Set getPendingLocations() + { + Set result = new HashSet<>(); + for (DataDirectory dataDirectory : dataDirectories.getAllDirectories()) + { + for (File dir : dataPaths) + { + if (!dataDirectory.contains(dir)) + continue; + result.add(getOrCreate(dir, PENDING_SUBDIR)); + } + } + return result; + } + + public File getPendingLocationForDisk(DataDirectory dataDirectory, TimeUUID planId) + { + for (File dir : dataPaths) + { + if (!dataDirectory.contains(dir)) + continue; + return getOrCreate(dir, PENDING_SUBDIR, planId.toString()); + } + throw new RuntimeException("Could not find pending location"); + } + public static File getBackupsDirectory(Descriptor desc) { return getBackupsDirectory(desc.directory); @@ -815,6 +839,13 @@ public DataDirectory(Path location) this.location = new File(location); } + public boolean contains(File file) + { + // Note that we must compare absolute paths (not canonical) here since keyspace directories might be symlinks + Path path = file.toAbsolute().toPath(); + return path.startsWith(location.toAbsolute().toPath()); + } + public long getAvailableSpace() { long availableSpace = PathUtils.tryGetSpace(location.toPath(), FileStore::getUsableSpace) - DatabaseDescriptor.getMinFreeSpacePerDriveInBytes(); diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java index d56deb86b5bb..41ca0bcab43f 100644 --- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java +++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java @@ -376,6 +376,8 @@ protected void recordLatency(TableMetrics metric, long latencyNanos) public UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadExecutionController controller) { ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange())); + if (cfs.metadata().replicationType().isTracked()) + controller.addActivationIds(view.sstables); Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().partitionKeyType)); // fetch data from current memtable, historical memtables, and SSTables in the correct order. diff --git a/src/java/org/apache/cassandra/db/ReadExecutionController.java b/src/java/org/apache/cassandra/db/ReadExecutionController.java index 85b17d469077..cb5b78e7a4df 100644 --- a/src/java/org/apache/cassandra/db/ReadExecutionController.java +++ b/src/java/org/apache/cassandra/db/ReadExecutionController.java @@ -18,21 +18,37 @@ package org.apache.cassandra.db; import java.nio.ByteBuffer; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; import java.util.concurrent.TimeUnit; +import javax.annotation.concurrent.NotThreadSafe; + import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.ColumnFamilyStore.ViewFragment; import org.apache.cassandra.db.filter.DataLimits; import org.apache.cassandra.index.Index; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.replication.ActivatedTransfers; +import org.apache.cassandra.replication.ShortMutationId; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.MonotonicClock; import org.apache.cassandra.utils.concurrent.OpOrder; import static org.apache.cassandra.utils.MonotonicClock.Global.preciseTime; +@NotThreadSafe public class ReadExecutionController implements AutoCloseable { + private static final Logger logger = LoggerFactory.getLogger(ReadExecutionController.class); + private static final long NO_SAMPLING = Long.MIN_VALUE; // For every reads @@ -50,6 +66,13 @@ public class ReadExecutionController implements AutoCloseable private final RepairedDataInfo repairedDataInfo; private long oldestUnrepairedTombstone = Long.MAX_VALUE; + /** + * Track bulk transfers involved in the read, so we can do read reconciliation. These come from the + * {@link ViewFragment}, not the SSTable read path, so bloom filters and short-circuiting SSTable scans will still + * include the total set of relevant bulk transfers. + */ + private Set activationIds = null; + ReadExecutionController(ReadCommand command, OpOrder.Group baseOp, TableMetadata baseMetadata, @@ -243,4 +266,29 @@ private void addSample() if (cfs != null) cfs.metric.topLocalReadQueryTime.addSample(cql, timeMicros); } + + public void addActivationIds(Iterable sstables) + { + Preconditions.checkState(metadata().replicationType().isTracked()); + if (activationIds == null) + activationIds = new HashSet<>(); + for (SSTableReader sstable : sstables) + { + ActivatedTransfers transfers = sstable.getCoordinatorLogOffsets().transfers(); + if (transfers.isEmpty()) + continue; + logger.trace("Adding overlapping IDs to read keyRange {}", command.dataRange.keyRange); + transfers.forEachIntersecting(command.dataRange.keyRange, id -> { + String msg = String.format("Found overlapping activation ID %s for queried range %s", id, command.dataRange.keyRange); + Tracing.trace(msg); + logger.debug(msg); + activationIds.add(id); + }); + } + } + + public Iterator getActivationIds() + { + return activationIds == null ? null : activationIds.iterator(); + } } diff --git a/src/java/org/apache/cassandra/db/SSTableImporter.java b/src/java/org/apache/cassandra/db/SSTableImporter.java index 1eb301ca4269..1725dcbe15e8 100644 --- a/src/java/org/apache/cassandra/db/SSTableImporter.java +++ b/src/java/org/apache/cassandra/db/SSTableImporter.java @@ -44,6 +44,7 @@ import org.apache.cassandra.io.sstable.format.SSTableFormat.Components; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.File; +import org.apache.cassandra.replication.MutationTrackingService; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.service.StorageService; @@ -80,11 +81,8 @@ synchronized List importNewSSTables(Options options) UUID importID = UUID.randomUUID(); logger.info("[{}] Loading new SSTables for {}/{}: {}", importID, cfs.getKeyspaceName(), cfs.getTableName(), options); - // This will be supported in the future TableMetadata metadata = cfs.metadata(); - if (metadata.replicationType() != null && metadata.replicationType().isTracked()) - throw new IllegalStateException("Can't import into tables with mutation tracking enabled"); - + boolean isTracked = metadata.replicationType().isTracked(); List> listers = getSSTableListers(options.srcPaths); Set currentDescriptors = new HashSet<>(); @@ -183,7 +181,10 @@ synchronized List importNewSSTables(Options options) Descriptor newDescriptor = cfs.getUniqueDescriptorFor(entry.getKey(), targetDir); maybeMutateMetadata(entry.getKey(), options); movedSSTables.add(new MovedSSTable(newDescriptor, entry.getKey(), entry.getValue())); - SSTableReader sstable = SSTableReader.moveAndOpenSSTable(cfs, entry.getKey(), newDescriptor, entry.getValue(), options.copyData); + // Don't move tracked SSTables, since that will move them to the live set on bounce + SSTableReader sstable = isTracked + ? SSTableReader.open(cfs, oldDescriptor, metadata.ref) + : SSTableReader.moveAndOpenSSTable(cfs, oldDescriptor, newDescriptor, entry.getValue(), options.copyData); newSSTablesPerDirectory.add(sstable); } catch (Throwable t) @@ -233,12 +234,14 @@ synchronized List importNewSSTables(Options options) if (!cfs.indexManager.validateSSTableAttachedIndexes(newSSTables, false, options.validateIndexChecksum)) cfs.indexManager.buildSSTableAttachedIndexesBlocking(newSSTables); - cfs.getTracker().addSSTables(newSSTables); + if (isTracked) + TrackedBulkTransfer.execute(cfs.keyspace.getName(), newSSTables); + else + cfs.getTracker().addSSTables(newSSTables); + for (SSTableReader reader : newSSTables) - { if (options.invalidateCaches && cfs.isRowCacheEnabled()) invalidateCachesForSSTable(reader); - } } catch (Throwable t) { @@ -250,6 +253,17 @@ 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/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java index 5b71ae33a23d..604a41302ddb 100644 --- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java +++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java @@ -758,6 +758,8 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ReadableView view, Co Tracing.trace("Acquiring sstable references"); List sstables = view.sstables(); sstables.sort(SSTableReader.maxTimestampDescending); + if (cfs.metadata().replicationType().isTracked()) + controller.addActivationIds(sstables); ClusteringIndexFilter filter = clusteringIndexFilter(); long minTimestamp = Long.MAX_VALUE; long mostRecentPartitionTombstone = Long.MIN_VALUE; @@ -997,7 +999,6 @@ private boolean queriesMulticellType() private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ReadableView view, ColumnFamilyStore cfs, ClusteringIndexNamesFilter filter, ReadExecutionController controller) { Tracing.trace("Acquiring sstable references"); - ImmutableBTreePartition result = null; SSTableReadMetricsCollector metricsCollector = new SSTableReadMetricsCollector(); @@ -1020,6 +1021,11 @@ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ReadableV /* add the SSTables on disk */ List sstables = view.sstables(); sstables.sort(SSTableReader.maxTimestampDescending); + if (cfs.metadata().replicationType().isTracked()) + { + logger.trace("Executing read against SSTables {}", sstables); + controller.addActivationIds(view.sstables()); + } // read sorted sstables for (SSTableReader sstable : sstables) { diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java index 5229fbb2c550..d45f16e578de 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java @@ -49,6 +49,7 @@ import org.apache.cassandra.io.sstable.metadata.MetadataCollector; import org.apache.cassandra.io.util.File; import org.apache.cassandra.replication.ImmutableCoordinatorLogOffsets; +import org.apache.cassandra.replication.MutationTrackingService; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.service.snapshot.SnapshotOptions; @@ -397,6 +398,7 @@ public static ImmutableCoordinatorLogOffsets getCoordinatorLogOffsets(Set sstables) public void addSSTables(Collection sstables) { + Preconditions.checkState(!cfstore.metadata().replicationType().isTracked()); + addSSTablesInternal(sstables, false, true, true); + } + + public void addSSTablesTracked(Collection sstables) + { + Preconditions.checkState(cfstore.metadata().replicationType().isTracked()); + for (SSTableReader sstable : sstables) + { + ImmutableCoordinatorLogOffsets logOffsets = sstable.getCoordinatorLogOffsets(); + Preconditions.checkState(logOffsets.mutations().isEmpty()); + Preconditions.checkState(!logOffsets.transfers().isEmpty()); + } + addSSTablesInternal(sstables, false, true, true); } diff --git a/src/java/org/apache/cassandra/db/lifecycle/View.java b/src/java/org/apache/cassandra/db/lifecycle/View.java index c6fd7ed52af2..6772b77166b0 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/View.java +++ b/src/java/org/apache/cassandra/db/lifecycle/View.java @@ -402,4 +402,4 @@ public boolean apply(T t) } }; } -} \ No newline at end of file +} diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java index 1f178a482bd8..535c5ff7defd 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java @@ -48,6 +48,7 @@ import org.apache.cassandra.streaming.StreamReceiver; import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.streaming.messages.StreamMessageHeader; +import org.apache.cassandra.utils.FBUtilities; import static java.lang.String.format; import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemory; @@ -139,7 +140,7 @@ public SSTableMultiWriter read(DataInputPlus in) throws IOException UnaryOperator transform = stats -> stats.mutateLevel(header.sstableLevel) .mutateRepairedMetadata(messageHeader.repairedAt, messageHeader.pendingRepair); - String description = String.format("level %s and repairedAt time %s and pendingRepair %s", + String description = format("level %s and repairedAt time %s and pendingRepair %s", header.sstableLevel, messageHeader.repairedAt, messageHeader.pendingRepair); writer.descriptor().getMetadataSerializer().mutate(writer.descriptor(), description, transform); return writer; @@ -159,9 +160,13 @@ public SSTableMultiWriter read(DataInputPlus in) throws IOException private File getDataDir(ColumnFamilyStore cfs, long totalSize) throws IOException { + boolean isTracked = cfs.metadata().replicationType().isTracked(); + Directories.DataDirectory localDir = cfs.getDirectories().getWriteableLocation(totalSize); if (localDir == null) - throw new IOException(format("Insufficient disk space to store %s", prettyPrintMemory(totalSize))); + throw new IOException(format("Insufficient disk space to store %s", FBUtilities.prettyPrintMemory(totalSize))); + if (isTracked) + return cfs.getDirectories().getPendingLocationForDisk(localDir, session.planId()); File dir = cfs.getDirectories().getLocationForDisk(cfs.getDiskBoundaries().getCorrectDiskForKey(header.firstKey)); diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java index b90ea5c69ccb..579fc9b29b0e 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java @@ -46,6 +46,7 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.UnknownColumnException; +import org.apache.cassandra.io.sstable.RangeAwarePendingSSTableWriter; import org.apache.cassandra.io.sstable.RangeAwareSSTableWriter; import org.apache.cassandra.io.sstable.SSTableMultiWriter; import org.apache.cassandra.io.sstable.SSTableSimpleIterator; @@ -61,6 +62,7 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.ProgressInfo; +import org.apache.cassandra.streaming.StreamOperation; import org.apache.cassandra.streaming.StreamReceivedOutOfTokenRangeException; import org.apache.cassandra.streaming.StreamReceiver; import org.apache.cassandra.streaming.StreamSession; @@ -187,7 +189,16 @@ protected SSTableTxnSingleStreamWriter createWriter(ColumnFamilyStore cfs, long StreamReceiver streamReceiver = session.getAggregator(tableId); Preconditions.checkState(streamReceiver instanceof CassandraStreamReceiver); ILifecycleTransaction txn = createTxn(); - RangeAwareSSTableWriter writer = new RangeAwareSSTableWriter(cfs, estimatedKeys, repairedAt, pendingRepair, coordinatorLogOffsets, format, sstableLevel, totalSize, txn, getHeader(cfs.metadata())); + + RangeAwareSSTableWriter writer; + if (session.streamOperation() == StreamOperation.TRACKED_TRANSFER) + { + Preconditions.checkState(cfs.metadata().replicationType().isTracked()); + writer = new RangeAwarePendingSSTableWriter(cfs, estimatedKeys, repairedAt, pendingRepair, coordinatorLogOffsets, format, sstableLevel, totalSize, txn, getHeader(cfs.metadata()), session.planId()); + } + else + writer = new RangeAwareSSTableWriter(cfs, estimatedKeys, repairedAt, pendingRepair, coordinatorLogOffsets, format, sstableLevel, totalSize, txn, getHeader(cfs.metadata())); + return new SSTableTxnSingleStreamWriter(txn, writer); } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java index 73d599fe04fe..e934d22334ab 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java @@ -52,7 +52,10 @@ import org.apache.cassandra.service.accord.AccordTopology; import org.apache.cassandra.service.accord.IAccordService; import org.apache.cassandra.service.accord.TimeOnlyRequestBookkeeping.LatencyRequestBookkeeping; +import org.apache.cassandra.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; @@ -132,6 +135,13 @@ public synchronized void received(IncomingStream stream) txn.update(finished); sstables.addAll(finished); receivedEntireSSTable = file.isEntireSSTable(); + + if (session.streamOperation() == StreamOperation.TRACKED_TRANSFER) + { + Preconditions.checkState(cfs.metadata().replicationType().isTracked()); + PendingLocalTransfer transfer = new PendingLocalTransfer(cfs.metadata().id, session.planId(), sstables); + MutationTrackingService.instance.received(transfer); + } } @Override @@ -256,6 +266,11 @@ public void finished() // add sstables (this will build non-SSTable-attached secondary indexes too, see CASSANDRA-10130) 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) + return; + cfs.addSSTables(readers); //invalidate row and counter cache diff --git a/src/java/org/apache/cassandra/io/sstable/RangeAwarePendingSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/RangeAwarePendingSSTableWriter.java new file mode 100644 index 000000000000..232cf0af621d --- /dev/null +++ b/src/java/org/apache/cassandra/io/sstable/RangeAwarePendingSSTableWriter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.io.sstable; + +import java.io.IOException; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; +import org.apache.cassandra.io.sstable.format.SSTableFormat; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.replication.ImmutableCoordinatorLogOffsets; +import org.apache.cassandra.utils.TimeUUID; + +public class RangeAwarePendingSSTableWriter extends RangeAwareSSTableWriter +{ + private final TimeUUID planId; + + public RangeAwarePendingSSTableWriter(ColumnFamilyStore cfs, long estimatedKeys, long repairedAt, TimeUUID pendingRepair, ImmutableCoordinatorLogOffsets coordinatorLogOffsets, SSTableFormat format, int sstableLevel, long totalSize, ILifecycleTransaction txn, SerializationHeader header, TimeUUID planId) throws IOException + { + super(cfs, estimatedKeys, repairedAt, pendingRepair, coordinatorLogOffsets, format, sstableLevel, totalSize, txn, header); + this.planId = planId; + } + + @Override + protected File currentFile() + { + return cfs.getDirectories().getPendingLocationForDisk(directories.get(currentIndex), planId); + } +} diff --git a/src/java/org/apache/cassandra/io/sstable/RangeAwareSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/RangeAwareSSTableWriter.java index f94ec94a77b9..aa08d5b39292 100644 --- a/src/java/org/apache/cassandra/io/sstable/RangeAwareSSTableWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/RangeAwareSSTableWriter.java @@ -32,6 +32,7 @@ import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.io.sstable.format.SSTableFormat; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.File; import org.apache.cassandra.replication.ImmutableCoordinatorLogOffsets; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.utils.FBUtilities; @@ -40,7 +41,7 @@ public class RangeAwareSSTableWriter implements SSTableMultiWriter { private final List boundaries; - private final List directories; + protected final List directories; private final int sstableLevel; private final long estimatedKeys; private final long repairedAt; @@ -49,7 +50,7 @@ public class RangeAwareSSTableWriter implements SSTableMultiWriter private final SSTableFormat format; private final SerializationHeader header; private final ILifecycleTransaction txn; - private int currentIndex = -1; + protected int currentIndex = -1; public final ColumnFamilyStore cfs; private final List finishedWriters = new ArrayList<>(); private SSTableMultiWriter currentWriter = null; @@ -79,7 +80,7 @@ public RangeAwareSSTableWriter(ColumnFamilyStore cfs, long estimatedKeys, long r } } - private void maybeSwitchWriter(DecoratedKey key) + protected void maybeSwitchWriter(DecoratedKey key) { if (boundaries == null) return; @@ -96,11 +97,16 @@ private void maybeSwitchWriter(DecoratedKey key) if (currentWriter != null) finishedWriters.add(currentWriter); - Descriptor desc = cfs.newSSTableDescriptor(cfs.getDirectories().getLocationForDisk(directories.get(currentIndex)), format); + Descriptor desc = cfs.newSSTableDescriptor(currentFile(), format); currentWriter = cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, pendingRepair, coordinatorLogOffsets, null, sstableLevel, header, txn); } } + protected File currentFile() + { + return cfs.getDirectories().getLocationForDisk(directories.get(currentIndex)); + } + public void append(UnfilteredRowIterator partition) { maybeSwitchWriter(partition.partitionKey()); diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java index 0caf2aecbf24..326ab6a189a8 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java @@ -1377,6 +1377,18 @@ public void mutateRepairedAndReload(long newRepairedAt, TimeUUID newPendingRepai } } + /** + * Mutate {@link ImmutableCoordinatorLogOffsets} with a lock to avoid racing with entire-sstable-streaming and then reload sstable metadata + */ + public void mutateCoordinatorLogOffsetsAndReload(ImmutableCoordinatorLogOffsets logOffsets) throws IOException + { + synchronized (tidy.global) + { + descriptor.getMetadataSerializer().mutateCoordinatorLogOffsets(descriptor, logOffsets); + reloadSSTableMetadata(); + } + } + /** * Reloads the sstable metadata from disk. *

diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataSerializer.java index e8299466c763..1b99222bdeb0 100644 --- a/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataSerializer.java +++ b/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataSerializer.java @@ -26,6 +26,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.Version; import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.replication.ImmutableCoordinatorLogOffsets; import org.apache.cassandra.utils.TimeUUID; /** @@ -96,6 +97,11 @@ public interface IMetadataSerializer */ public void mutateRepairMetadata(Descriptor descriptor, long newRepairedAt, TimeUUID newPendingRepair) throws IOException; + /** + * Replace mutation tracking metadata. + */ + public void mutateCoordinatorLogOffsets(Descriptor descriptor, ImmutableCoordinatorLogOffsets logOffsets) throws IOException; + /** * Replace the sstable metadata file ({@code -Statistics.db}) with the given components. */ diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java index ac8aa08c96e8..7a40e50a2c76 100644 --- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java +++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java @@ -45,6 +45,7 @@ import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileDataInput; import org.apache.cassandra.io.util.RandomAccessReader; +import org.apache.cassandra.replication.ImmutableCoordinatorLogOffsets; import org.apache.cassandra.utils.TimeUUID; import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt; @@ -248,6 +249,15 @@ public void mutateRepairMetadata(Descriptor descriptor, long newRepairedAt, Time mutate(descriptor, stats -> stats.mutateRepairedMetadata(newRepairedAt, newPendingRepair)); } + @Override + public void mutateCoordinatorLogOffsets(Descriptor descriptor, ImmutableCoordinatorLogOffsets logOffsets) throws IOException + { + if (logger.isTraceEnabled()) + logger.trace("Mutating {} to {}", descriptor.fileFor(Components.STATS), logOffsets); + + mutate(descriptor, stats -> stats.mutateCoordinatorLogOffsets(logOffsets)); + } + private void mutate(Descriptor descriptor, UnaryOperator transform) throws IOException { Map currentComponents = deserialize(descriptor, EnumSet.allOf(MetadataType.class)); diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java index fa7a4491f37a..cdd53eec5c45 100644 --- a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java +++ b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java @@ -242,6 +242,35 @@ public StatsMetadata mutateRepairedMetadata(long newRepairedAt, TimeUUID newPend lastKey); } + public StatsMetadata mutateCoordinatorLogOffsets(ImmutableCoordinatorLogOffsets newLogOffsets) + { + return new StatsMetadata(estimatedPartitionSize, + estimatedCellPerPartitionCount, + commitLogIntervals, + minTimestamp, + maxTimestamp, + minLocalDeletionTime, + maxLocalDeletionTime, + minTTL, + maxTTL, + compressionRatio, + estimatedTombstoneDropTime, + sstableLevel, + clusteringTypes, + coveredClustering, + hasLegacyCounterShards, + repairedAt, + totalColumnsSet, + totalRows, + tokenSpaceCoverage, + originatingHostId, + pendingRepair, + hasPartitionLevelDeletions, + newLogOffsets, + firstKey, + lastKey); + } + @Override public boolean equals(Object o) { diff --git a/src/java/org/apache/cassandra/net/Verb.java b/src/java/org/apache/cassandra/net/Verb.java index c26e3af09097..6719e71e2772 100644 --- a/src/java/org/apache/cassandra/net/Verb.java +++ b/src/java/org/apache/cassandra/net/Verb.java @@ -76,8 +76,11 @@ import org.apache.cassandra.repair.messages.ValidationResponse; import org.apache.cassandra.replication.BroadcastLogOffsets; import org.apache.cassandra.replication.ForwardedWrite; +import org.apache.cassandra.replication.LocalTransfers; import org.apache.cassandra.replication.PullMutationsRequest; import org.apache.cassandra.replication.PushMutationRequest; +import org.apache.cassandra.replication.TransferActivation; +import org.apache.cassandra.replication.TransferFailed; import org.apache.cassandra.schema.SchemaMutationsSerializer; import org.apache.cassandra.schema.SchemaPullVerbHandler; import org.apache.cassandra.schema.SchemaPushVerbHandler; @@ -336,6 +339,10 @@ public enum Verb TRACKED_SUMMARY_RSP (910, P2, readTimeout, REQUEST_RESPONSE, () -> TrackedSummaryResponse.serializer, () -> TrackedSummaryResponse.verbHandler ), TRACKED_SUMMARY_REQ (911, P3, readTimeout, READ, () -> TrackedRead.SummaryRequest.serializer, () -> TrackedRead.verbHandler, TRACKED_SUMMARY_RSP ), + TRACKED_TRANSFER_ACTIVATE_RSP (912, P1, repairTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, RESPONSE_HANDLER), + TRACKED_TRANSFER_ACTIVATE_REQ (913, P1, repairTimeout, ANTI_ENTROPY, () -> TransferActivation.serializer, () -> TransferActivation.verbHandler, TRACKED_TRANSFER_ACTIVATE_RSP), + TRACKED_TRANSFER_FAILED_RSP (914, P1, repairTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, RESPONSE_HANDLER), + TRACKED_TRANSFER_FAILED_REQ (915, P1, repairTimeout, ANTI_ENTROPY, () -> TransferFailed.serializer, () -> LocalTransfers.verbHandler, TRACKED_TRANSFER_FAILED_RSP), // accord ACCORD_SIMPLE_RSP (119, P2, writeTimeout, IMMEDIATE, () -> accordEmbedded(EnumSerializer.simpleReply), AccordService::responseHandlerOrNoop ), diff --git a/src/java/org/apache/cassandra/replication/ActivatedTransfers.java b/src/java/org/apache/cassandra/replication/ActivatedTransfers.java new file mode 100644 index 000000000000..a918ba28cb50 --- /dev/null +++ b/src/java/org/apache/cassandra/replication/ActivatedTransfers.java @@ -0,0 +1,279 @@ +/* + * 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.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.function.Consumer; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.Comparators; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; + +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Bounds; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.utils.CollectionSerializers; + +public class ActivatedTransfers implements Iterable +{ + public static final ActivatedTransfers EMPTY = new ActivatedTransfers(); + + // This could be IntervalTree, but we're expecting to have _very_ few transfers (typically 0) + private final Set transfers; + + public ActivatedTransfers() + { + this(new HashSet<>(1)); + } + + private ActivatedTransfers(Collection transfers) + { + this.transfers = new HashSet<>(transfers); + } + + public static ActivatedTransfers copyOf(ActivatedTransfers other) + { + return other == null ? new ActivatedTransfers() : new ActivatedTransfers(other.transfers); + } + + @VisibleForTesting + static final class ActivatedTransfer + { + final ShortMutationId id; + final Bounds bounds; + + @VisibleForTesting + ActivatedTransfer(ShortMutationId id, Bounds bounds) + { + this.id = id; + this.bounds = bounds; + } + + private ActivatedTransfer(ShortMutationId id, Collection sstables) + { + this(id, covering(sstables)); + } + + public static final IVersionedSerializer serializer = new IVersionedSerializer<>() + { + @Override + public void serialize(ActivatedTransfer transfer, DataOutputPlus out, int version) throws IOException + { + ShortMutationId.serializer.serialize(transfer.id, out, version); + Token.serializer.serialize(transfer.bounds.left, out, version); + Token.serializer.serialize(transfer.bounds.right, out, version); + } + + @Override + public ActivatedTransfer deserialize(DataInputPlus in, int version) throws IOException + { + ShortMutationId id = ShortMutationId.serializer.deserialize(in, version); + Token left = Token.serializer.deserialize(in, version); + Token right = Token.serializer.deserialize(in, version); + return new ActivatedTransfer(id, new Bounds(left, right)); + } + + @Override + public long serializedSize(ActivatedTransfer transfer, int version) + { + long size = 0; + size += ShortMutationId.serializer.serializedSize(transfer.id, version); + size += Token.serializer.serializedSize(transfer.bounds.left, version); + size += Token.serializer.serializedSize(transfer.bounds.right, version); + return size; + } + }; + + @Override + public String toString() + { + return "ActivatedTransfer{" + + "id=" + id + + ", bounds=" + bounds + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) return false; + ActivatedTransfer that = (ActivatedTransfer) o; + return Objects.equals(id, that.id) && Objects.equals(bounds, that.bounds); + } + + @Override + public int hashCode() + { + return Objects.hash(id, bounds); + } + } + + public void removeOffset(int offset) + { + transfers.removeIf(transfer -> transfer.id.offset() == offset); + } + + @VisibleForTesting + public void add(ShortMutationId activationId, Bounds bounds) + { + transfers.add(new ActivatedTransfer(activationId, bounds)); + } + + public void add(ShortMutationId activationId, Collection sstables) + { + transfers.add(new ActivatedTransfer(activationId, sstables)); + } + + public void addAll(ActivatedTransfers other) + { + transfers.addAll(other.transfers); + } + + public void forEachIntersecting(AbstractBounds range, Consumer consumer) + { + for (ActivatedTransfer transfer : transfers) + if (intersects(transfer.bounds, range)) + consumer.accept(transfer.id); + } + + public void forEachIntersecting(Token token, Consumer consumer) + { + for (ActivatedTransfer transfer : transfers) + if (transfer.bounds.contains(token)) + consumer.accept(transfer.id); + } + + @Override + public Iterator iterator() + { + return Iterators.transform(transfers.iterator(), transfer -> transfer.id); + } + + public boolean isEmpty() + { + return transfers.isEmpty(); + } + + private static Bounds covering(Collection sstables) + { + Preconditions.checkArgument(!sstables.isEmpty()); + Iterator iter = sstables.iterator(); + SSTableReader next = iter.next(); + Token left = next.getFirst().getToken(); + Token right = next.getLast().getToken(); + while (iter.hasNext()) + { + next = iter.next(); + left = Comparators.min(left, next.getFirst().getToken()); + right = Comparators.max(right, next.getLast().getToken()); + } + return new Bounds<>(left, right); + } + + private static boolean intersects(Bounds bounds, AbstractBounds range) + { + Preconditions.checkArgument(!AbstractBounds.strictlyWrapsAround(bounds.left, bounds.right)); + if (range instanceof Range && ((Range) range).isTrulyWrapAround()) + { + List> unwrapped = range.unwrap(); + return Iterables.any(unwrapped, unwrap -> intersects(bounds, unwrap)); + } + + if (range.right.getToken().isMinimum()) + { + /* + bounds: [] + range: ?----| + */ + boolean overlapsPastBoundary = bounds.right.compareTo(range.left.getToken()) > 0; + /* + bounds: [] + range: [----| + */ + boolean overlapsAtBoundary = bounds.right.equals(range.left.getToken()) && range.inclusiveLeft(); + return overlapsPastBoundary || overlapsAtBoundary; + } + + if ((range.left.getToken().compareTo(bounds.right) < 0) && (bounds.left.compareTo(range.right.getToken()) < 0)) + return true; + + if (range.inclusiveLeft() && bounds.contains(range.left.getToken())) + return true; + if (range.inclusiveRight() && bounds.contains(range.right.getToken())) + return true; + return false; + } + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) return false; + ActivatedTransfers that = (ActivatedTransfers) o; + return Objects.equals(transfers, that.transfers); + } + + @Override + public int hashCode() + { + return Objects.hashCode(transfers); + } + + @Override + public String toString() + { + return "ActivatedTransfers{" + + "transfers=" + transfers + + '}'; + } + + public static final IVersionedSerializer serializer = new IVersionedSerializer<>() + { + @Override + public void serialize(ActivatedTransfers transfers, DataOutputPlus out, int version) throws IOException + { + CollectionSerializers.serializeCollection(transfers.transfers, out, version, ActivatedTransfer.serializer); + } + + @Override + public ActivatedTransfers deserialize(DataInputPlus in, int version) throws IOException + { + return new ActivatedTransfers(CollectionSerializers.deserializeSet(in, version, ActivatedTransfer.serializer)); + } + + @Override + public long serializedSize(ActivatedTransfers transfers, int version) + { + return CollectionSerializers.serializedCollectionSize(transfers.transfers, version, ActivatedTransfer.serializer); + } + }; +} diff --git a/src/java/org/apache/cassandra/replication/ActiveLogReconciler.java b/src/java/org/apache/cassandra/replication/ActiveLogReconciler.java index 54ded3766c75..8667c766018e 100644 --- a/src/java/org/apache/cassandra/replication/ActiveLogReconciler.java +++ b/src/java/org/apache/cassandra/replication/ActiveLogReconciler.java @@ -17,11 +17,15 @@ */ package org.apache.cassandra.replication; +import java.util.Collections; import java.util.concurrent.TimeUnit; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.agrona.concurrent.ManyToOneConcurrentLinkedQueue; import org.apache.cassandra.concurrent.Interruptible; import org.apache.cassandra.concurrent.Shutdownable; @@ -46,6 +50,8 @@ // TODO (expected): handle temporarily down nodes public final class ActiveLogReconciler implements Shutdownable { + private static final Logger logger = LoggerFactory.getLogger(ActiveLogReconciler.class); + public enum Priority { HIGH, REGULAR } // prioritised delivery of mutations that are needed by reads; @@ -71,7 +77,7 @@ public enum Priority { HIGH, REGULAR } */ void schedule(ShortMutationId mutationId, InetAddressAndPort toHost, Priority priority) { - queue(priority).offer(new Task(mutationId, toHost)); + queue(priority).offer(Task.from(mutationId, toHost)); haveWork.release(1); } @@ -82,7 +88,7 @@ void schedule(ShortMutationId mutationId, InetAddressAndPort toHost, Priority pr void schedule(Offsets offsets, InetAddressAndPort toHost, Priority priority) { ManyToOneConcurrentLinkedQueue queue = queue(priority); - offsets.forEach(id -> queue.offer(new Task(id, toHost))); + offsets.forEach(id -> queue.offer(Task.from(id, toHost))); haveWork.release(1); } @@ -114,12 +120,26 @@ public void run(Interruptible.State state) throws InterruptedException } } - private static final class Task implements RequestCallback + private static abstract class Task implements RequestCallback + { + private static Task from(ShortMutationId id, InetAddressAndPort toHost) + { + CoordinatedTransfer transfer = LocalTransfers.instance().getActivatedTransfer(id); + if (transfer != null) + return new TransferTask(transfer, toHost); + else + return new MutationTask(id, toHost); + } + + abstract void send(); + } + + private static final class MutationTask extends Task { private final ShortMutationId mutationId; private final InetAddressAndPort toHost; - Task(ShortMutationId mutationId, InetAddressAndPort toHost) + MutationTask(ShortMutationId mutationId, InetAddressAndPort toHost) { this.mutationId = mutationId; this.toHost = toHost; @@ -156,6 +176,59 @@ void send() } } + private static final class TransferTask extends Task + { + private final CoordinatedTransfer transfer; + private final InetAddressAndPort toHost; + + TransferTask(CoordinatedTransfer transfer, InetAddressAndPort toHost) + { + this.transfer = transfer; + this.toHost = toHost; + } + + @Override + public boolean invokeOnFailure() + { + return true; + } + + @Override + public void onResponse(Message msg) + { + logger.debug("Received activation ack for TransferTask from {}", toHost); + MutationTrackingService.instance.receivedActivationResponse(transfer, toHost); + } + + @Override + public void onFailure(InetAddressAndPort from, RequestFailure failureReason) + { + onFailure(failureReason.failure); + } + + public void onFailure(Throwable cause) + { + logger.debug("Received activation failure for TransferTask from {} due to", toHost, cause); + MutationTrackingService.instance.retryFailedTransfer(transfer, toHost, cause); + } + + void send() + { + logger.debug("Sending activation to {}", toHost); + LocalTransfers.instance().executor.submit(() -> { + try + { + transfer.activateOn(Collections.singleton(toHost)); + onResponse(null); + } + catch (Throwable t) + { + onFailure(t); + } + }); + } + } + private volatile boolean isShutdown = false; private volatile boolean isPaused = false; diff --git a/src/java/org/apache/cassandra/replication/CoordinatedTransfer.java b/src/java/org/apache/cassandra/replication/CoordinatedTransfer.java new file mode 100644 index 000000000000..9820ecc91176 --- /dev/null +++ b/src/java/org/apache/cassandra/replication/CoordinatedTransfer.java @@ -0,0 +1,587 @@ +/* + * 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.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Supplier; + +import com.google.common.annotations.VisibleForTesting; +import 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.Throwables; +import org.apache.cassandra.utils.TimeUUID; +import org.apache.cassandra.utils.concurrent.AsyncFuture; +import org.apache.cassandra.utils.concurrent.Future; +import org.apache.cassandra.utils.concurrent.FutureCombiner; + +/** + * A tracked bulk transfer for a single replica set. + *

+ * For simplicity, streaming from coordinator to itself instead of copying files. This has some perks: + * (1) it allows us to import out-of-range SSTables using the same paths, and + * (2) it uses the existing lifecycle management to handle crash-safety, so don't need to deal with atomic multi-file + * copy. + *

+ * A transfer happens in a few steps. First, the coordinator streams the SSTables to each replica. Replicas store the + * streamed transfer in a "pending" location on the filesystem, where it isn't visible to reads. Once the coordinator + * receives acknowledgements of completed streams from sufficient replicas, the coordinator assigns an "activation ID" + * for the transfer, and notifies replicas that the pending stream has been activated with that ID. Replicas then move + * the pending SSTables into the live set, where they're visible to reads, and include the "activation ID" in mutation + * tracking summaries for reads that would include the new SSTables. + */ +public class CoordinatedTransfer +{ + private static final Logger logger = LoggerFactory.getLogger(CoordinatedTransfer.class); + + String logPrefix() + { + return String.format("[CoordinatedTransfer #%s]", transferId); + } + + final TimeUUID transferId; + + // TODO(expected): Add epoch at time of creation + final String keyspace; + public final Range range; + + final ConcurrentMap streams; + + final Collection sstables; + + final ConsistencyLevel cl; + + final Supplier getActivationId; + volatile MutationId activationId = null; + + CoordinatedTransfer(String keyspace, Range range, Participants participants, Collection sstables, ConsistencyLevel cl, Supplier getActivationId) + { + this(TimeUUID.Generator.nextTimeUUID(), keyspace, range, participants, sstables, cl, getActivationId); + } + + @VisibleForTesting + CoordinatedTransfer(TimeUUID transferId, Range range, MutationId activationId) + { + this.transferId = transferId; + this.keyspace = null; + this.range = range; + this.sstables = Collections.emptyList(); + this.cl = null; + this.getActivationId = () -> activationId; + this.activationId = activationId; + this.streams = new ConcurrentHashMap<>(); + } + + private CoordinatedTransfer(TimeUUID transferId, String keyspace, Range range, Participants participants, Collection sstables, ConsistencyLevel cl, Supplier getActivationId) + { + this.transferId = transferId; + this.keyspace = keyspace; + this.range = range; + this.sstables = sstables; + this.cl = cl; + this.getActivationId = getActivationId; + + ClusterMetadata cm = ClusterMetadata.current(); + this.streams = new ConcurrentHashMap<>(participants.size()); + for (int i = 0; i < participants.size(); i++) + { + InetAddressAndPort addr = cm.directory.getNodeAddresses(new NodeId(participants.get(i))).broadcastAddress; + this.streams.put(addr, SingleTransferResult.Init()); + } + } + + void execute() + { + logger.debug("Executing tracked bulk transfer {}", 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<>(streams.size()); + for (InetAddressAndPort to : streams.keySet()) + { + Future stream = LocalTransfers.instance().executor.submit(() -> { + stream(to); + return null; + }); + streaming.add(stream); + } + + Future> future = FutureCombiner.allOf(streaming); + try + { + future.get(); + } + catch (InterruptedException | ExecutionException e) + { + logger.error("{} Failed due to", logPrefix(), e); + Throwable cause = e instanceof ExecutionException ? e.getCause() : e; + throw Throwables.unchecked(cause); + } + } + + private boolean sufficient() + { + AbstractReplicationStrategy ars = Keyspace.open(keyspace).getReplicationStrategy(); + int blockFor = cl.blockFor(ars); + int responses = 0; + for (Map.Entry entry : streams.entrySet()) + { + if (entry.getValue().state == SingleTransferResult.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; + cleanupFailedStream(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 Future notifyFailure() + { + class NotifyFailure extends AsyncFuture implements RequestCallbackWithFailure + { + final Set responses = ConcurrentHashMap.newKeySet(streams.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 : streams.entrySet()) + { + InetAddressAndPort to = entry.getKey(); + SingleTransferResult result = entry.getValue(); + if (result.planId == null) + continue; + + logger.debug("Notifying {} of transfer failure for plan {}", 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); + } + return notifyFailure; + } + + // This shouldn't throw an exception, even if we fail to notify peers of the streaming failure + private void cleanupFailedStream(InetAddressAndPort to, Throwable cause) + { + LocalTransfers.instance().scheduleCleanup(); + + if (cause instanceof StreamException) + streams.get(to).streamFailed(((StreamException) cause).finalState.planId); + else + streams.get(to).streamFailed(null); + + Future notify = notifyFailure(); + try + { + notify.get(); + } + catch (Throwable t) + { + if (cause != null) + t.addSuppressed(cause); + logger.error("Failed to notify peers of stream failure", t); + } + } + + private void streamComplete(InetAddressAndPort to, SingleTransferResult result) throws ExecutionException, InterruptedException, TimeoutException + { + streams.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 : streams.entrySet()) + { + InetAddressAndPort peer = entry.getKey(); + SingleTransferResult result = entry.getValue(); + if (result.state == SingleTransferResult.State.ACTIVATE_COMPLETE) + { + anyActivated = true; + } + else if (result.state == SingleTransferResult.State.STREAM_COMPLETE) + awaitingActivation.add(peer); + } + if (anyActivated && !awaitingActivation.isEmpty()) + { + logger.debug("{} Transfer already activated on peers, sending activations to {}", 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 : streams.entrySet()) + { + InetAddressAndPort peer = entry.getKey(); + SingleTransferResult result = entry.getValue(); + if (result.state == SingleTransferResult.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"); + } + + void activateOn(Collection peers) + { + Preconditions.checkState(!peers.isEmpty()); + logger.debug("{} Activating {} on {}", logPrefix(), this, peers); + + if (activationId == null) + { + activationId = getActivationId.get(); + logger.info("{} Assigned activationId {}", logPrefix(), activationId); + } + 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); + cleanupFailedStream(from, failure.failure); + 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, TransferActivation.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); + SingleTransferResult result = CoordinatedTransfer.this.streams.get(msg.from()); + if (result != null) + result.sentActivation(); + } + 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 {}", msg.from()); + SingleTransferResult result = CoordinatedTransfer.this.streams.get(msg.from()); + if (result != null) + result.completedActivation(); + + 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 {}", 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, TransferActivation.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); + } + + 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); + } + + static class SingleTransferResult + { + enum State + { + INIT, + STREAM_NOOP, + STREAM_FAILED, + STREAM_COMPLETE, + ACTIVATE_START, + ACTIVATE_COMPLETE, + UNKNOWN + } + + volatile State state; + private volatile TimeUUID planId; + + private SingleTransferResult(State state, TimeUUID planId) + { + this.state = state; + this.planId = planId; + } + + public static SingleTransferResult Init() + { + return new SingleTransferResult(State.INIT, null); + } + + private static SingleTransferResult Complete(TimeUUID planId) + { + return new SingleTransferResult(State.STREAM_COMPLETE, planId); + } + + private static SingleTransferResult Noop() + { + return new SingleTransferResult(State.STREAM_NOOP, null); + } + + public void streamFailed(TimeUUID planId) + { + this.state = State.STREAM_FAILED; + // Don't overwrite if the stream succeeded but PREPARE failed, so we can clean up later + if (planId != null) + this.planId = planId; + } + + public void sentActivation() + { + state = State.ACTIVATE_START; + } + + public void completedActivation() + { + state = State.ACTIVATE_COMPLETE; + } + + 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.Complete(plan.planId()); + } + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) return false; + CoordinatedTransfer transfer = (CoordinatedTransfer) o; + return Objects.equals(transferId, transfer.transferId) && Objects.equals(keyspace, transfer.keyspace) && Objects.equals(range, transfer.range) && Objects.equals(streams, transfer.streams) && Objects.equals(sstables, transfer.sstables) && cl == transfer.cl && Objects.equals(getActivationId, transfer.getActivationId) && Objects.equals(activationId, transfer.activationId); + } + + @Override + public int hashCode() + { + return Objects.hash(transferId, keyspace, range, streams, sstables, cl, getActivationId, activationId); + } + + @Override + public String toString() + { + return "CoordinatedTransfer{" + + "transferId=" + transferId + + ", range=" + range + + ", streams=" + streams + + ", sstables=" + sstables + + ", activationId=" + activationId + + '}'; + } +} diff --git a/src/java/org/apache/cassandra/replication/CoordinatedTransfers.java b/src/java/org/apache/cassandra/replication/CoordinatedTransfers.java new file mode 100644 index 000000000000..47de1120b004 --- /dev/null +++ b/src/java/org/apache/cassandra/replication/CoordinatedTransfers.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.replication; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; + +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.lifecycle.SSTableIntervalTree; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.utils.Interval; + +class CoordinatedTransfers implements Iterable +{ + private final Collection transfers; + + private CoordinatedTransfers(Collection transfers) + { + this.transfers = transfers; + } + + static CoordinatedTransfers create(String keyspace, MutationTrackingService.KeyspaceShards shards, Collection sstables, ConsistencyLevel cl) + { + // Clean up incoming SSTables to remove any existing CoordinatorLogOffsets, can't be trusted + for (SSTableReader sstable : sstables) + { + try + { + sstable.mutateCoordinatorLogOffsetsAndReload(ImmutableCoordinatorLogOffsets.NONE); + } + catch (IOException e) + { + throw new UncheckedIOException(e); + } + } + + // Expensive - add a metric? + SSTableIntervalTree intervals = SSTableIntervalTree.buildSSTableIntervalTree(sstables); + List transfers = new ArrayList<>(); + + shards.forEachShard(shard -> { + Range range = shard.tokenRange(); + Collection sstablesForRange = intervals.search(Interval.create(range.left.minKeyBound(), range.right.maxKeyBound())); + + CoordinatedTransfer transfer = new CoordinatedTransfer(keyspace, range, shard.participants, sstablesForRange, cl, shard::nextId); + if (transfer.sstables.isEmpty()) + return; + transfers.add(transfer); + }); + return new CoordinatedTransfers(transfers); + } + + @Override + public Iterator iterator() + { + return transfers.iterator(); + } + + @Override + public String toString() + { + return "CoordinatedTransfers{transfers=" + transfers + '}'; + } +} diff --git a/src/java/org/apache/cassandra/replication/CoordinatorLog.java b/src/java/org/apache/cassandra/replication/CoordinatorLog.java index 6f4a4aed4cd9..a0c7add0e120 100644 --- a/src/java/org/apache/cassandra/replication/CoordinatorLog.java +++ b/src/java/org/apache/cassandra/replication/CoordinatorLog.java @@ -18,6 +18,7 @@ package org.apache.cassandra.replication; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -148,37 +149,34 @@ CoordinatorLog withParticipants(Participants newParticipants) // retroactively un-reconciling previously reconciled offsets for the other replicas. offsets.addAll(reconciledOffsets); } - Offsets.Mutable persisted = participants.contains(participantId) ? persistedOffsets.get(participantId) : new Offsets.Mutable(logId); - passivelyReconciled = passivelyReconciled != null ? Offsets.Immutable.intersection(passivelyReconciled, offsets) : offsets; - newWitnessedOffsets.add(participantId, offsets); newPersistedOffsets.add(participantId, persisted); } - UnreconciledMutations newUnreconciled; + UnreconciledMutations newUnreconciledMutations; passivelyReconciled = Offsets.Immutable.difference(passivelyReconciled, reconciledOffsets); if (!passivelyReconciled.isEmpty()) { logger.debug("Toplogy change implicitly reconciled offsets: {}", passivelyReconciled); - newUnreconciled = unreconciledMutations.copy(); - passivelyReconciled.forEach(id -> newUnreconciled.remove(id.offset)); + newUnreconciledMutations = unreconciledMutations.copy(); + passivelyReconciled.forEach(id -> newUnreconciledMutations.remove(id.offset)); } else { - newUnreconciled = unreconciledMutations; + newUnreconciledMutations = unreconciledMutations; } if (logger.isTraceEnabled()) logger.trace("Updating coordinator log {} participants: {} -> {}. Passively reconciled: {}", logId, participants, newParticipants, passivelyReconciled); - return withUpdatedParticipants(newParticipants, newWitnessedOffsets, newPersistedOffsets, newUnreconciled); + return withUpdatedParticipants(newParticipants, newWitnessedOffsets, newPersistedOffsets, newUnreconciledMutations); } finally { @@ -218,6 +216,7 @@ private void updateWitnessedReplicatedOffsets(Offsets offsets, int onNodeId) reconciledOffsets.add(offset); unreconciledMutations.remove(offset); } + logger.trace("done applying WRO, now {}", witnessedOffsets); } }); } @@ -225,7 +224,9 @@ private void updateWitnessedReplicatedOffsets(Offsets offsets, int onNodeId) private void updatePersistedReplicatedOffsets(Offsets offsets, int onNodeId) { persistedOffsets.get(onNodeId).addAll(offsets); + logger.debug("done applying PO, now {}", persistedOffsets); reconciledPersistedOffsets.addAll(persistedOffsets.intersection()); + logger.debug("done applying PRO, now {}", reconciledPersistedOffsets); } public void recordFullyReconciledOffsets(Offsets.Immutable reconciled) @@ -331,17 +332,80 @@ protected boolean remoteReplicasWitnessed(int offset) return othersWitnessed(offset, localNodeId); } + /* + - On local replicas after they've completed activation (onHostId == me) + */ + void finishActivation(PendingLocalTransfer transfer, TransferActivation activation) + { + logger.trace("witnessed local transfer {}", activation.id()); + + lock.writeLock().lock(); + try + { + int offset = activation.id().offset(); + // we've raced with another write, no need to do anything else + if (!witnessedOffsets.get(localNodeId).add(offset)) + return; + + // This is the only difference with finishWriting - can we consolidate these methods? + unreconciledMutations.activatedTransfer(activation.id(), transfer.sstables); + + if (remoteReplicasWitnessed(offset)) + { + reconciledOffsets.add(offset); + unreconciledMutations.remove(offset); + } + } + finally + { + lock.writeLock().unlock(); + } + } + + /* + - 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) + { + MutationId activationId = transfer.activationId; + Preconditions.checkArgument(!activationId.isNone()); + logger.trace("witnessed transfer activation ack {} from {}", activationId, onHostId); + lock.writeLock().lock(); + try + { + if (!witnessedOffsets.get(onHostId).add(activationId.offset())) + return; // already witnessed; very uncommon but possible path + + if (!witnessedOffsets.get(localNodeId).contains(activationId.offset())) + return; // local host hasn't witnessed yet -> no cleanup needed + + if (remoteReplicasWitnessed(activationId.offset())) + { + logger.trace("marking transfer {} as fully reconciled", activationId); + // if all replicas have now witnessed the id, remove it from the index + unreconciledMutations.remove(activationId.offset()); + reconciledOffsets.add(activationId.offset()); + } + } + finally + { + logger.trace("after receivedActivationAck {} witnessed by: {}", activationId, witnessedOffsets); + lock.writeLock().unlock(); + } + } + /** * Look up unreconciled sequence ids of mutations witnessed by this host in this coordinataor log. * Adds the ids to the supplied collection, so it can be reused to aggregate lookups for multiple logs. */ - boolean collectOffsetsFor(Token token, TableId tableId, boolean includePending, Offsets.OffsetReciever unreconciledInto, Offsets.OffsetReciever reconciledInto) + void collectOffsetsFor(Token token, TableId tableId, boolean includePending, Offsets.OffsetReciever unreconciledInto, Offsets.OffsetReciever reconciledInto) { lock.readLock().lock(); try { reconciledInto.addAll(reconciledOffsets); - return unreconciledMutations.collect(token, tableId, includePending, unreconciledInto); + unreconciledMutations.collect(token, tableId, includePending, unreconciledInto); } finally { @@ -353,13 +417,13 @@ boolean collectOffsetsFor(Token token, TableId tableId, boolean includePending, * Look up unreconciled sequence ids of mutations witnessed by this host in this coordinataor log. * Adds the ids to the supplied collection, so it can be reused to aggregate lookups for multiple logs. */ - boolean collectOffsetsFor(AbstractBounds range, TableId tableId, boolean includePending, Offsets.OffsetReciever unreconciledInto, Offsets.OffsetReciever reconciledInto) + void collectOffsetsFor(AbstractBounds range, TableId tableId, boolean includePending, Offsets.OffsetReciever unreconciledInto, Offsets.OffsetReciever reconciledInto) { lock.readLock().lock(); try { reconciledInto.addAll(reconciledOffsets); - return unreconciledMutations.collect(range, tableId, includePending, unreconciledInto); + unreconciledMutations.collect(range, tableId, includePending, unreconciledInto); } finally { @@ -406,14 +470,50 @@ void collectDurablyReconciledOffsets(Log2OffsetsMap.Mutable into) into.add(reconciledPersistedOffsets); } + boolean isDurablyReconciled(ShortMutationId id) + { + lock.readLock().lock(); + try + { + boolean contains = reconciledPersistedOffsets.contains(id.offset); + if (!contains) + logger.debug("Offset {} is not contained in durably reconciled offsets {}", id.offset, reconciledPersistedOffsets); + return contains; + } + finally + { + lock.readLock().unlock(); + } + } + + private boolean isDurablyReconciled(Iterator ids) + { + if (ids == null) + return true; + while (ids.hasNext()) + { + ShortMutationId id = ids.next(); + if (id.logId() != logId.asLong()) + continue; + if (!isDurablyReconciled(id)) + return false; + } + return true; + } + boolean isDurablyReconciled(CoordinatorLogOffsets logOffsets) { lock.readLock().lock(); try { Offsets.RangeIterator durablyReconciled = reconciledPersistedOffsets.rangeIterator(); - Offsets.RangeIterator difference = Offsets.difference(logOffsets.offsets(logId.asLong()).rangeIterator(), durablyReconciled); - return !difference.tryAdvance(); + // Mutations only + Offsets.RangeIterator offsets = logOffsets.mutations().offsets(logId.asLong()).rangeIterator(); + Offsets.RangeIterator unreconciledMutations = Offsets.difference(offsets, durablyReconciled); + + // Transfers + boolean transfersReconciled = isDurablyReconciled(logOffsets.transfers().iterator()); + return transfersReconciled && !unreconciledMutations.tryAdvance(); } finally { @@ -447,8 +547,6 @@ static class CoordinatorLogPrimary extends CoordinatorLog super(keyspace, range, localNodeId, logId, participants); } - - @Override CoordinatorLog withUpdatedParticipants(Participants newParticipants, Node2OffsetsMap witnessedOffsets, Node2OffsetsMap persistedOffsets, UnreconciledMutations unreconciledMutations) { diff --git a/src/java/org/apache/cassandra/replication/CoordinatorLogOffsets.java b/src/java/org/apache/cassandra/replication/CoordinatorLogOffsets.java index dfe34f580050..5a394fc08df9 100644 --- a/src/java/org/apache/cassandra/replication/CoordinatorLogOffsets.java +++ b/src/java/org/apache/cassandra/replication/CoordinatorLogOffsets.java @@ -22,18 +22,35 @@ /** * Mutation ID offsets present in this SSTable for each coordinator log, to determine whether an SSTable is reconciled - * or not. + * or not. This includes "plain" mutation IDs for regular writes, and "activation" mutation IDs for bulk transfers. + * Bulk transfer IDs are kept separately because we expect to have very few of them, and they're materialized for fast + * access on the read path. *

* Note that peers may have reconciled all mutations included in an SSTable, but {@link StatsMetadata#repairedAt} is * dependent on compaction timing, so "nodetool repair --validate" may report temporary disagreements on the repaired * set. - *

- * Iterable over {@link CoordinatorLogId}. */ -public interface CoordinatorLogOffsets extends Iterable +public interface CoordinatorLogOffsets { - O offsets(long logId); - int size(); + /** + * Iterable over {@link CoordinatorLogId}. + */ + interface Mutations extends Iterable + { + O offsets(long logId); + int size(); + default boolean isEmpty() + { + return size() == 0; + } + } + + Mutations mutations(); + + default ActivatedTransfers transfers() + { + return ActivatedTransfers.EMPTY; + } ImmutableCoordinatorLogOffsets NONE = new ImmutableCoordinatorLogOffsets.Builder(0).build(); } diff --git a/src/java/org/apache/cassandra/replication/ImmutableCoordinatorLogOffsets.java b/src/java/org/apache/cassandra/replication/ImmutableCoordinatorLogOffsets.java index 7d946ec9f737..fdeda33cbf7e 100644 --- a/src/java/org/apache/cassandra/replication/ImmutableCoordinatorLogOffsets.java +++ b/src/java/org/apache/cassandra/replication/ImmutableCoordinatorLogOffsets.java @@ -19,16 +19,24 @@ package org.apache.cassandra.replication; import java.io.IOException; +import java.util.Collection; import java.util.Iterator; import java.util.Map; import java.util.Objects; -import java.util.function.BiConsumer; +import java.util.function.Predicate; import javax.annotation.concurrent.NotThreadSafe; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterators; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.agrona.collections.Long2ObjectHashMap; +import org.apache.cassandra.dht.Bounds; +import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.net.MessagingService; @@ -36,73 +44,73 @@ public class ImmutableCoordinatorLogOffsets implements CoordinatorLogOffsets { - private final Long2ObjectHashMap ids; + private static final Logger logger = LoggerFactory.getLogger(ImmutableCoordinatorLogOffsets.class); - @Override - public Offsets.Immutable offsets(long logId) - { - Offsets.Immutable offsets = ids.get(logId); - if (offsets == null) - return new Offsets.Immutable(new CoordinatorLogId(logId)); - return offsets; - } + private final ImmutableMutations mutations; + private final ActivatedTransfers transfers; - @Override - public int size() + private ImmutableCoordinatorLogOffsets(Builder builder) { - return ids.size(); + // Important to set shouldAvoidAllocation=false, otherwise iterators are cached and not thread safe, even when immutable and read-only + Long2ObjectHashMap ids = new Long2ObjectHashMap<>(builder.ids.size(), 0.9f, false); + + for (Map.Entry entry : builder.ids.entrySet()) + ids.put(entry.getKey(), entry.getValue().build()); + + this.mutations = new ImmutableMutations(ids); + this.transfers = ActivatedTransfers.copyOf(builder.transfers); } - public boolean isEmpty() + @Override + public Mutations mutations() { - return size() == 0; + return mutations; } @Override - public Iterator iterator() + public ActivatedTransfers transfers() { - return Iterators.unmodifiableIterator(ids.keySet().iterator()); + return transfers == null ? ActivatedTransfers.EMPTY : transfers; } public Iterable> entries() { - return ids.entrySet(); + return mutations.ids.entrySet(); } + public boolean isEmpty() + { + return mutations().isEmpty() && transfers().isEmpty(); + } @Override public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; - ImmutableCoordinatorLogOffsets longs = (ImmutableCoordinatorLogOffsets) o; - return Objects.equals(ids, longs.ids); + ImmutableCoordinatorLogOffsets other = (ImmutableCoordinatorLogOffsets) o; + return Objects.equals(mutations, other.mutations) && Objects.equals(transfers, other.transfers); } @Override public int hashCode() { - return Objects.hashCode(ids); + return Objects.hash(mutations, transfers); } - public ImmutableCoordinatorLogOffsets(Builder builder) - { - // Important to set shouldAvoidAllocation=false, otherwise iterators are cached and not thread safe, even when - // immutable and read-only - this.ids = new Long2ObjectHashMap<>(builder.ids.size(), 0.9f, false); - - for (Map.Entry entry : builder.ids.entrySet()) - ids.put(entry.getKey(), entry.getValue().build()); - } - - public void forEach(BiConsumer consumer) + @Override + public String toString() { - ids.forEach((logId, offsets) -> consumer.accept(new CoordinatorLogId(logId), offsets)); + return "ImmutableCoordinatorLogOffsets{" + + "mutations=" + mutations + + ", transfers=" + transfers + + '}'; } @NotThreadSafe public static class Builder { private final Long2ObjectHashMap ids; + private ActivatedTransfers transfers; public Builder() { @@ -112,6 +120,7 @@ public Builder() public Builder(int size) { this.ids = new Long2ObjectHashMap<>(size, 0.9f, false); + this.transfers = null; } public Builder add(MutationId mutationId) @@ -123,17 +132,28 @@ public Builder add(MutationId mutationId) return this; } - public Builder addAll(CoordinatorLogOffsets logOffsets) + private Builder addAll(CoordinatorLogOffsets.Mutations mutations) { - for (long log : logOffsets) + for (long log : mutations) { - Offsets offsets = logOffsets.offsets(log); + Offsets offsets = mutations.offsets(log); ids.computeIfAbsent(log, logId -> new Offsets.Immutable.Builder(new CoordinatorLogId(logId))) .addAll(offsets); } return this; } + public Builder addAll(CoordinatorLogOffsets logOffsets) + { + addAll(logOffsets.mutations()); + ActivatedTransfers newTransfers = logOffsets.transfers(); + if (transfers == null) + transfers = newTransfers; + else + transfers.addAll(newTransfers); + return this; + } + public Builder addAll(Offsets.Immutable offsets) { ids.computeIfAbsent(offsets.logId.asLong(), logId -> new Offsets.Immutable.Builder(new CoordinatorLogId(logId))) @@ -141,6 +161,62 @@ public Builder addAll(Offsets.Immutable offsets) return this; } + @VisibleForTesting + public Builder addTransfer(ShortMutationId activationId, Bounds bounds) + { + if (activationId.isNone()) + return this; + if (transfers == null) + transfers = new ActivatedTransfers(); + transfers.add(activationId, bounds); + return this; + } + + public Builder addTransfer(ShortMutationId activationId, Collection sstables) + { + if (activationId.isNone()) + return this; + if (transfers == null) + transfers = new ActivatedTransfers(); + transfers.add(activationId, sstables); + return this; + } + + public Builder addTransfers(ActivatedTransfers other) + { + if (other.isEmpty()) + return this; + if (transfers == null) + transfers = other; + else + transfers.addAll(other); + return this; + } + + /** + * Removes expired transfers + */ + public void purgeTransfers(Predicate predicate) + { + int purged = 0; + if (transfers != null) + { + Iterator iter = transfers.iterator(); + while (iter.hasNext()) + { + ShortMutationId id = iter.next(); + if (predicate.test(id)) + { + iter.remove(); + purged++; + logger.debug("Purging activation {}", id); + } + } + } + if (purged > 0) + logger.info("Purged {} transfers", purged); + } + public ImmutableCoordinatorLogOffsets build() { return new ImmutableCoordinatorLogOffsets(this); @@ -154,9 +230,8 @@ public void serialize(ImmutableCoordinatorLogOffsets logOffsets, DataOutputPlus { if (version < MessagingService.VERSION_52) return; - out.writeUnsignedVInt32(logOffsets.size()); - for (long logId : logOffsets) - Offsets.serializer.serialize(logOffsets.offsets(logId), out, version); + ImmutableMutations.serializer.serialize(logOffsets.mutations, out, version); + ActivatedTransfers.serializer.serialize(logOffsets.transfers(), out, version); } @Override @@ -164,13 +239,12 @@ public ImmutableCoordinatorLogOffsets deserialize(DataInputPlus in, int version) { if (version < MessagingService.VERSION_52) return ImmutableCoordinatorLogOffsets.NONE; - int size = in.readUnsignedVInt32(); - ImmutableCoordinatorLogOffsets.Builder builder = new ImmutableCoordinatorLogOffsets.Builder(size); - for (int i = 0; i < size; i++) - { - Offsets.Immutable offsets = Offsets.serializer.deserialize(in, version); - builder.addAll(offsets); - } + Builder builder = new Builder(); + ImmutableMutations mutations = ImmutableMutations.serializer.deserialize(in, version); + mutations.ids.forEach((id, offsets) -> builder.addAll(offsets)); + ActivatedTransfers transfers = ActivatedTransfers.serializer.deserialize(in, version); + if (!transfers.isEmpty()) + builder.addTransfers(transfers); return builder.build(); } @@ -180,12 +254,97 @@ public long serializedSize(ImmutableCoordinatorLogOffsets logOffsets, int versio if (version < MessagingService.VERSION_52) return 0; long size = 0; - size += VIntCoding.computeUnsignedVIntSize(logOffsets.size()); - for (long logId : logOffsets) - size += Offsets.serializer.serializedSize(logOffsets.offsets(logId), version); + size += ImmutableMutations.serializer.serializedSize(logOffsets.mutations, version); + size += ActivatedTransfers.serializer.serializedSize(logOffsets.transfers(), version); return size; } } public static final Serializer serializer = new Serializer(); + + public static class ImmutableMutations implements Mutations + { + final private Long2ObjectHashMap ids; + + private ImmutableMutations(Long2ObjectHashMap ids) + { + this.ids = ids; + } + + @Override + public Offsets.Immutable offsets(long logId) + { + Offsets.Immutable offsets = ids.get(logId); + if (offsets == null) + return new Offsets.Immutable(new CoordinatorLogId(logId)); + return offsets; + } + + @Override + public int size() + { + return ids.size(); + } + + @Override + public Iterator iterator() + { + return Iterators.unmodifiableIterator(ids.keySet().iterator()); + } + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) return false; + ImmutableMutations longs = (ImmutableMutations) o; + return Objects.equals(ids, longs.ids); + } + + @Override + public int hashCode() + { + return Objects.hashCode(ids); + } + + @Override + public String toString() + { + return "ImmutableMutations{" + + "ids=" + ids + + '}'; + } + + private static final IVersionedSerializer serializer = new IVersionedSerializer<>() + { + @Override + public void serialize(ImmutableMutations mutations, DataOutputPlus out, int version) throws IOException + { + out.writeUnsignedVInt32(mutations.size()); + for (long logId : mutations) + Offsets.serializer.serialize(mutations.offsets(logId), out, version); + } + + @Override + public ImmutableMutations deserialize(DataInputPlus in, int version) throws IOException + { + int size = in.readUnsignedVInt32(); + Long2ObjectHashMap ids = new Long2ObjectHashMap<>(size, 0.9f, false); + for (int i = 0; i < size; i++) + { + Offsets.Immutable offsets = Offsets.serializer.deserialize(in, version); + ids.put(offsets.logId.asLong(), offsets); + } + return new ImmutableMutations(ids); + } + + @Override + public long serializedSize(ImmutableMutations mutations, int version) + { + long size = VIntCoding.computeUnsignedVIntSize(mutations.size()); + for (long logId : mutations) + size += Offsets.serializer.serializedSize(mutations.offsets(logId), version); + return size; + } + }; + } } diff --git a/src/java/org/apache/cassandra/replication/LocalTransfers.java b/src/java/org/apache/cassandra/replication/LocalTransfers.java new file mode 100644 index 000000000000..f6dca0079314 --- /dev/null +++ b/src/java/org/apache/cassandra/replication/LocalTransfers.java @@ -0,0 +1,287 @@ +/* + * 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.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.concurrent.ExecutorPlus; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.net.IVerbHandler; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.NoPayload; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.TimeUUID; + +import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; +import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State; +import static org.apache.cassandra.replication.CoordinatedTransfer.SingleTransferResult.State.ACTIVATE_COMPLETE; +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; + +/** + * Stores coordinated and received transfers. + *

+ * TODO: Make changes to pending set durable with SystemKeyspace.savePendingLocalTransfer(transfer)? + */ +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 coordinatingActivated = new ConcurrentHashMap<>(); + private final Map received = new ConcurrentHashMap<>(); + + final ExecutorPlus executor = executorFactory().pooled("LocalTrackedTransfers", Integer.MAX_VALUE); + + private static final LocalTransfers instance = new LocalTransfers(); + static LocalTransfers instance() + { + return instance; + } + + void save(CoordinatedTransfer transfer) + { + lock.writeLock().lock(); + try + { + CoordinatedTransfer existing = coordinating.put(transfer.transferId, transfer); + Preconditions.checkState(existing == null); + } + finally + { + lock.writeLock().unlock(); + } + } + + void activating(CoordinatedTransfer transfer) + { + Preconditions.checkNotNull(transfer.activationId); + lock.writeLock().lock(); + try + { + coordinatingActivated.put(transfer.activationId, transfer); + } + finally + { + lock.writeLock().unlock(); + } + } + + void received(PendingLocalTransfer transfer) + { + lock.writeLock().lock(); + try + { + logger.debug("received: {}", transfer); + Preconditions.checkState(!transfer.sstables.isEmpty()); + + PendingLocalTransfer existing = received.put(transfer.planId, transfer); + Preconditions.checkState(existing == null); + } + finally + { + lock.writeLock().unlock(); + } + } + + @VisibleForTesting + Purger purger = new Purger(); + + static class Purger + { + boolean test(CoordinatedTransfer transfer) + { + logger.trace("Checking whether we can purge {}", transfer); + + // Safe to purge a transfer if it failed before activation anywhere (INIT, STREAM_NOOP, STREAM_COMPLETE), or + // if all activation is complete (ACTIVATE_COMPLETE, STREAM_NOOP) since those leave no opportunity for a + // peer to request the transfer during reconciliation + boolean failedBeforeActivation = false; + boolean noneActivated = true; + boolean allComplete = true; + for (CoordinatedTransfer.SingleTransferResult result : transfer.streams.values()) + { + State state = result.state; + if (state == STREAM_FAILED) + failedBeforeActivation = true; + + if (state != INIT && state != STREAM_NOOP && state != STREAM_COMPLETE && state != STREAM_FAILED) + noneActivated = false; + + if (state != ACTIVATE_COMPLETE && state != STREAM_NOOP) + allComplete = false; + } + + return (failedBeforeActivation && noneActivated) || (allComplete && transfer.activationId != null); + } + + boolean test(PendingLocalTransfer transfer) + { + return transfer.activated; + } + } + + private void cleanup() + { + lock.writeLock().lock(); + try + { + for (PendingLocalTransfer transfer : received.values()) + if (purger.test(transfer)) + purge(transfer); + + for (CoordinatedTransfer transfer : coordinating.values()) + if (purger.test(transfer)) + purge(transfer); + } + finally + { + lock.writeLock().unlock(); + } + } + + private void purge(TransferFailed failed) + { + lock.writeLock().lock(); + try + { + PendingLocalTransfer pending = received.get(failed.planId); + if (pending == null) + { + logger.warn("Cannot purge unknown local pending transfer {}", failed); + return; + } + purge(pending); + } + finally + { + lock.writeLock().unlock(); + } + } + + private void purge(PendingLocalTransfer transfer) + { + logger.info("Cleaning up activated pending transfer: {}", transfer); + + lock.writeLock().lock(); + try + { + // Delete the entire pending transfer directory /pending// + if (!transfer.sstables.isEmpty()) + { + SSTableReader sstable = transfer.sstables.iterator().next(); + File pendingDir = sstable.descriptor.directory; + + if (pendingDir.exists()) + { + Preconditions.checkState(pendingDir.absolutePath().contains(transfer.planId.toString())); + logger.debug("Deleting pending transfer directory: {}", pendingDir); + pendingDir.deleteRecursive(); + } + } + } + finally + { + lock.writeLock().unlock(); + } + } + + private void purge(CoordinatedTransfer transfer) + { + logger.info("Cleaning up completed coordinated transfer: {}", transfer); + + lock.writeLock().lock(); + try + { + coordinating.remove(transfer.transferId); + + if (transfer.activationId != null) + coordinatingActivated.remove(transfer.activationId); + + CoordinatedTransfer.SingleTransferResult localPending = transfer.streams.get(FBUtilities.getBroadcastAddressAndPort()); + PendingLocalTransfer localTransfer; + TimeUUID planId; + if (localPending != null && (planId = localPending.planId()) != null && (localTransfer = received.get(planId)) != null) + purge(localTransfer); + } + finally + { + lock.writeLock().unlock(); + } + } + + void scheduleCleanup() + { + executor.submit(() -> { + try + { + cleanup(); + } + catch (Throwable t) + { + logger.error("Cleanup failed", t); + } + }); + } + + @Nullable PendingLocalTransfer getPendingTransfer(TimeUUID planId) + { + lock.readLock().lock(); + try + { + return received.get(planId); + } + finally + { + lock.readLock().unlock(); + } + } + + @Nullable CoordinatedTransfer getActivatedTransfer(ShortMutationId activationId) + { + lock.readLock().lock(); + try + { + return coordinatingActivated.get(activationId); + } + finally + { + lock.readLock().unlock(); + } + } + + public static IVerbHandler verbHandler = message -> { + LocalTransfers.instance().purge(message.payload); + MessagingService.instance().respond(NoPayload.noPayload, message); + }; +} diff --git a/src/java/org/apache/cassandra/replication/MutableCoordinatorLogOffsets.java b/src/java/org/apache/cassandra/replication/MutableCoordinatorLogOffsets.java index e47e6245d90e..9ecb389554d1 100644 --- a/src/java/org/apache/cassandra/replication/MutableCoordinatorLogOffsets.java +++ b/src/java/org/apache/cassandra/replication/MutableCoordinatorLogOffsets.java @@ -28,11 +28,11 @@ public interface MutableCoordinatorLogOffsets extends CoordinatorLogOffsets mutations) { - for (long logId : from) + for (long logId : mutations) { - Offsets offsets = from.offsets(logId); + Offsets offsets = mutations.offsets(logId); offsets.forEach(this::add); } } diff --git a/src/java/org/apache/cassandra/replication/MutationTrackingService.java b/src/java/org/apache/cassandra/replication/MutationTrackingService.java index 4009dbd39a88..f19daa4736ac 100644 --- a/src/java/org/apache/cassandra/replication/MutationTrackingService.java +++ b/src/java/org/apache/cassandra/replication/MutationTrackingService.java @@ -38,10 +38,12 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; import org.agrona.collections.IntArrayList; import org.agrona.collections.IntHashSet; import org.apache.cassandra.concurrent.ScheduledExecutorPlus; +import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.cql3.UntypedResultSet; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Mutation; @@ -53,6 +55,7 @@ import org.apache.cassandra.dht.Token; 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.InetAddressAndPort; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; @@ -75,6 +78,7 @@ import org.slf4j.LoggerFactory; import static java.lang.String.format; +import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; import static org.apache.cassandra.concurrent.ExecutorFactory.SimulatorSemantics.NORMAL; import static org.apache.cassandra.cql3.QueryProcessor.executeInternal; @@ -88,9 +92,12 @@ public class MutationTrackingService /** * Split ranges into this many shards. *

+ * REVIEW: Reset back to 1 because for transfers, replicas need to know each others' shards, since transfers are + * sliced to fit within shards. Can we achieve sharding via split range ownership, instead of it being local-only? + *

* TODO (expected): ability to rebalance / change this constant */ - private static final int SHARD_MULTIPLIER = 8; + private static final int SHARD_MULTIPLIER = 1; private static final Logger logger = LoggerFactory.getLogger(MutationTrackingService.class); public static final MutationTrackingService instance = new MutationTrackingService(); @@ -250,6 +257,29 @@ public void receivedWriteResponse(ShortMutationId mutationId, InetAddressAndPort } } + public void receivedActivationResponse(CoordinatedTransfer transfer, InetAddressAndPort fromHost) + { + shardLock.readLock().lock(); + try + { + logger.debug("{} receivedActivationAck from {}", transfer.logPrefix(), fromHost); + MutationId activationId = transfer.activationId; + Preconditions.checkArgument(!activationId.isNone()); + + // REVIEW: This will be called with ShortMutationId, which overrides hashCode from CoordinatorLogId, but map + // is updated with CoordinatorLogId; shouldn't call this with a ShortMutationId, not sure why that's working + // elsewhere + Shard shard = getShardNullable(new CoordinatorLogId(activationId.logId())); + // Local activation acknowledged in MutationTrackingService.activateLocal + if (shard != null && !fromHost.equals(FBUtilities.getBroadcastAddressAndPort())) + shard.receivedActivationResponse(transfer, fromHost); + } + finally + { + shardLock.readLock().unlock(); + } + } + public void retryFailedWrite(ShortMutationId mutationId, InetAddressAndPort onHost, RequestFailure reason) { Preconditions.checkArgument(!mutationId.isNone()); @@ -257,6 +287,14 @@ public void retryFailedWrite(ShortMutationId mutationId, InetAddressAndPort onHo activeReconciler.schedule(mutationId, onHost, ActiveLogReconciler.Priority.REGULAR); } + public void retryFailedTransfer(CoordinatedTransfer transfer, InetAddressAndPort onHost, Throwable cause) + { + logger.debug("Retrying failed transfer {} to {} with exception", transfer, onHost, cause); + MutationId id = transfer.activationId; + Preconditions.checkArgument(!id.isNone()); + activeReconciler.schedule(id, onHost, ActiveLogReconciler.Priority.REGULAR); + } + public void updateReplicatedOffsets(String keyspace, Range range, List offsets, boolean durable, InetAddressAndPort onHost) { shardLock.readLock().lock(); @@ -325,6 +363,57 @@ public boolean registerMutationCallback(ShortMutationId mutationId, IncomingMuta return incomingMutations.subscribe(mutationId, callback); } + public void executeTransfers(String keyspace, Set sstables, ConsistencyLevel cl) + { + shardLock.readLock().lock(); + try + { + 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); + logger.info("Split input SSTables into transfers {}", transfers); + + for (CoordinatedTransfer transfer : transfers) + transfer.execute(); + } + finally + { + shardLock.readLock().unlock(); + } + } + + public void received(PendingLocalTransfer transfer) + { + logger.debug("Received pending transfer for tracked table {}", transfer); + LocalTransfers.instance().received(transfer); + } + + void activateLocal(TransferActivation activation) + { + shardLock.readLock().lock(); + try + { + PendingLocalTransfer pending = LocalTransfers.instance().getPendingTransfer(activation.planId); + if (pending == null) + { + logger.error("Cannot activate unknown local pending transfer {}", activation); + return; + } + pending.activate(activation); + + if (activation.isCommit()) + { + keyspaceShards.get(pending.keyspace).lookUp(pending.range).finishActivation(pending, activation); + incomingMutations.invokeListeners(activation.activationId); + } + } + finally + { + shardLock.readLock().unlock(); + } + } + public MutationSummary createSummaryForKey(DecoratedKey key, TableId tableId, boolean includePending) { shardLock.readLock().lock(); @@ -459,14 +548,33 @@ private int nextHostLogId() } private int prevHostLogId; + public boolean isDurablyReconciled(ShortMutationId id) + { + shardLock.readLock().lock(); + try + { + long logId = id.logId(); + Shard shard = getShardNullable(new CoordinatorLogId(logId)); + if (shard == null) + throw new IllegalStateException("Could not find shard for logId " + logId); + + return shard.isDurablyReconciled(id); + } + finally + { + shardLock.readLock().unlock(); + } + } + public boolean isDurablyReconciled(ImmutableCoordinatorLogOffsets logOffsets) { shardLock.readLock().lock(); try { - // Could pass through SSTable bounds to exclude shards for non-overlapping ranges, but this will mostly be - // called on flush for L0 SSTables with wide bounds. - for (Long logId : logOffsets) + Iterable mutations = logOffsets.mutations(); + Iterable transfers = Iterables.transform(logOffsets.transfers(), ShortMutationId::logId); + Iterable logIds = Iterables.concat(mutations, transfers); + for (Long logId : logIds) { Shard shard = getShardNullable(new CoordinatorLogId(logId)); if (shard == null) @@ -889,6 +997,11 @@ Shard lookUp(Token token) return shards.get(groups.forRange(token).range()); } + Shard lookUp(Range range) + { + return shards.get(groups.matchRange(range).range()); + } + void persistToSystemTables() { for (Shard shard : shards.values()) shard.persistToSystemTables(); diff --git a/src/java/org/apache/cassandra/replication/Node2OffsetsMap.java b/src/java/org/apache/cassandra/replication/Node2OffsetsMap.java index 1001d855cd40..ac6fcc0dafae 100644 --- a/src/java/org/apache/cassandra/replication/Node2OffsetsMap.java +++ b/src/java/org/apache/cassandra/replication/Node2OffsetsMap.java @@ -122,4 +122,12 @@ public boolean equals(Object o) Node2OffsetsMap that = (Node2OffsetsMap) o; return this.offsetsMap.equals(that.offsetsMap); } + + @Override + public String toString() + { + return "Node2OffsetsMap{" + + "offsetsMap=" + offsetsMap + + '}'; + } } diff --git a/src/java/org/apache/cassandra/replication/NonBlockingCoordinatorLogOffsets.java b/src/java/org/apache/cassandra/replication/NonBlockingCoordinatorLogOffsets.java index 857d6fc999e0..3f54ad847b08 100644 --- a/src/java/org/apache/cassandra/replication/NonBlockingCoordinatorLogOffsets.java +++ b/src/java/org/apache/cassandra/replication/NonBlockingCoordinatorLogOffsets.java @@ -71,18 +71,31 @@ public void add(ShortMutationId mutationId) } @Override - public Offsets.Mutable offsets(long logId) + public Mutations mutations() { - E logOffsets = get(logId); - if (logOffsets == null) - return new Offsets.Mutable(new CoordinatorLogId(logId)); - return logOffsets.offsets(); - } + return new Mutations<>() + { + @Override + public Offsets.Mutable offsets(long logId) + { + E logOffsets = get(logId); + if (logOffsets == null) + return new Offsets.Mutable(new CoordinatorLogId(logId)); + return logOffsets.offsets(); + } - @Override - public Iterator iterator() - { - return Iterators.unmodifiableIterator(keys().asIterator()); + @Override + public int size() + { + return NonBlockingCoordinatorLogOffsets.super.size(); + } + + @Override + public Iterator iterator() + { + return Iterators.unmodifiableIterator(keys().asIterator()); + } + }; } public static class Exclusive extends NonBlockingCoordinatorLogOffsets diff --git a/src/java/org/apache/cassandra/replication/PendingLocalTransfer.java b/src/java/org/apache/cassandra/replication/PendingLocalTransfer.java new file mode 100644 index 000000000000..ebd7d8c97fab --- /dev/null +++ b/src/java/org/apache/cassandra/replication/PendingLocalTransfer.java @@ -0,0 +1,220 @@ +/* + * 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.Collection; +import java.util.Objects; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.streaming.CassandraStreamReceiver; +import org.apache.cassandra.dht.AbstractBounds; +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.EndpointsForRange; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ownership.ReplicaGroups; +import org.apache.cassandra.utils.TimeUUID; + +import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; + +/** + * A transfer on a replica, once present on disk. + */ +public class PendingLocalTransfer +{ + private static final Logger logger = LoggerFactory.getLogger(PendingLocalTransfer.class); + + private String logPrefix() + { + return String.format("[PendingLocalTransfer #%s]", planId); + } + + final TimeUUID planId; + final TableId tableId; + final Collection sstables; + final long createdAt = currentTimeMillis(); + transient String keyspace; + transient Range range; + + volatile boolean activated = false; + + public PendingLocalTransfer(TableId tableId, TimeUUID planId, Collection sstables) + { + Preconditions.checkState(!sstables.isEmpty()); + this.tableId = tableId; + this.planId = planId; + this.sstables = sstables; + this.keyspace = Objects.requireNonNull(ColumnFamilyStore.getIfExists(tableId)).keyspace.getName(); + this.range = shardRange(keyspace, sstables); + } + + @VisibleForTesting + PendingLocalTransfer(TimeUUID planId, Collection sstables) + { + Preconditions.checkState(!sstables.isEmpty()); + this.planId = planId; + this.tableId = null; + this.sstables = sstables; + this.keyspace = null; + this.range = null; + } + + /** + * Pending transfers should be within a single shard, which are aligned to natural ranges. + * See ({@link MutationTrackingService.KeyspaceShards#make}). + */ + private static Range shardRange(String keyspace, Collection sstables) + { + ClusterMetadata cm = ClusterMetadata.current(); + ReplicaGroups writes = cm.placements.get(Keyspace.open(keyspace).getMetadata().params.replication).writes; + Range range = null; + for (SSTableReader sstable : sstables) + { + if (range == null) + { + Token first = sstable.getFirst().getToken(); + range = writes.forRange(first).range(); + } + else + { + AbstractBounds bounds = sstable.getBounds(); + Preconditions.checkState(!range.isTrulyWrapAround()); + Preconditions.checkState(range.contains(bounds.left)); + Preconditions.checkState(range.contains(bounds.right)); + } + } + + Preconditions.checkNotNull(range); + return range; + } + + private boolean isFullReplica() + { + ClusterMetadata cm = ClusterMetadata.current(); + Keyspace ks = Keyspace.open(keyspace); + ReplicaGroups writes = cm.placements.get(ks.getMetadata().params.replication).writes; + EndpointsForRange replicas = writes.forRange(range.right).get(); + return replicas.containsSelf() && replicas.selfIfPresent().isFull(); + } + + /** + * Safely move a transfer into the live set. This must be crash-safe, and the primary invariant we need to + * preserve is a transfer is only added to the live set iff the transfer ID is present in its mutation summaries. + * + * We don't validate checksums here, mostly because a transfer can be activated during a read, if one replica + * missed the TransferActivation. Transfers should not be pending for very long, and should be protected by + * internode integrity checks provided by TLS. + * + * TODO: Clear out the row cache and counter cache, like {@link CassandraStreamReceiver#finished}. + */ + public void activate(TransferActivation activation) + { + if (activated) + return; + + Preconditions.checkState(isFullReplica()); + + logger.info("{} Activating transfer {}, {} ms since pending", logPrefix(), this, currentTimeMillis() - createdAt); + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(tableId); + Preconditions.checkNotNull(cfs); + Preconditions.checkState(!sstables.isEmpty()); + + if (activation.isPropose()) + { + logger.info("{} Not adding SSTables to live set for dryRun {}", logPrefix(), activation); + return; + } + + // Modify SSTables metadata to durably set transfer ID before importing + ImmutableCoordinatorLogOffsets logOffsets = new ImmutableCoordinatorLogOffsets.Builder() + .addTransfer(activation.activationId, sstables) + .build(); + + // Ensure no lingering mutation IDs, only activation IDs + for (SSTableReader sstable : sstables) + { + Preconditions.checkState(sstable.getCoordinatorLogOffsets().mutations().isEmpty()); + try + { + sstable.mutateCoordinatorLogOffsetsAndReload(logOffsets); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + + Preconditions.checkState(sstable.getCoordinatorLogOffsets().mutations().isEmpty()); + ActivatedTransfers transfers = sstable.getCoordinatorLogOffsets().transfers(); + Preconditions.checkState(!transfers.isEmpty()); + } + + File dst = cfs.getDirectories().getDirectoryForNewSSTables(); + + // Retain the original SSTables in pending/ dir on the coordinator, so future streams can get the originals, and + // we don't need to isolate activated SSTables during compaction + boolean isCoordinator = activation.activationId.hostId == ClusterMetadata.current().myNodeId().id(); + logger.debug("{} {} pending SSTables for activation to {}", isCoordinator ? "Copying" : "Moving", logPrefix(), dst); + + dst.createFileIfNotExists(); + Collection moved = new ArrayList<>(sstables.size()); + for (SSTableReader sstable : sstables) + moved.add(SSTableReader.moveAndOpenSSTable(cfs, sstable.descriptor, cfs.getUniqueDescriptorFor(sstable.descriptor, dst), sstable.getComponents(), isCoordinator)); + + // Add all SSTables atomically + cfs.getTracker().addSSTablesTracked(moved); + + activated = true; + LocalTransfers.instance().scheduleCleanup(); + } + + @Override + public String toString() + { + return "PendingLocalTransfer{" + + "planId=" + planId + + ", tableId=" + tableId + + ", sstables=" + sstables + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) return false; + PendingLocalTransfer transfer = (PendingLocalTransfer) o; + return Objects.equals(planId, transfer.planId) && Objects.equals(tableId, transfer.tableId) && Objects.equals(sstables, transfer.sstables); + } + + @Override + public int hashCode() + { + return Objects.hash(planId, tableId, sstables); + } +} diff --git a/src/java/org/apache/cassandra/replication/PullMutationsRequest.java b/src/java/org/apache/cassandra/replication/PullMutationsRequest.java index ec5632ab5ba0..d8e6f0febb98 100644 --- a/src/java/org/apache/cassandra/replication/PullMutationsRequest.java +++ b/src/java/org/apache/cassandra/replication/PullMutationsRequest.java @@ -72,4 +72,12 @@ public void doVerb(Message message) MutationTrackingService.instance.requestMissingMutations(offsets, forHost); } }; + + @Override + public String toString() + { + return "PullMutationsRequest{" + + "offsets=" + offsets + + '}'; + } } diff --git a/src/java/org/apache/cassandra/replication/Shard.java b/src/java/org/apache/cassandra/replication/Shard.java index 7274127fcb74..825d35af7dcc 100644 --- a/src/java/org/apache/cassandra/replication/Shard.java +++ b/src/java/org/apache/cassandra/replication/Shard.java @@ -139,7 +139,7 @@ Shard withParticipants(Participants newParticipants) { CoordinatorLog newLog = log.withParticipants(newParticipants); newLogs.put(newLog.logId.asLong(), newLog); - + if (log == currentLocalLog) newCurrentLocalLog = (CoordinatorLog.CoordinatorLogPrimary) newLog; } @@ -153,9 +153,10 @@ Shard withParticipants(Participants newParticipants) MutationId nextId() { MutationId nextId = currentLocalLog.nextId(); - if (nextId != null) - return nextId; - return maybeRotateLocalLogAndGetNextId(); + if (nextId == null) + nextId = maybeRotateLocalLogAndGetNextId(); + logger.trace("Issuing next MutationId {}", nextId); + return nextId; } // if ids overflow, we need to rotate the local log @@ -176,6 +177,17 @@ void receivedWriteResponse(ShortMutationId mutationId, InetAddressAndPort fromHo getOrCreate(mutationId).receivedWriteResponse(mutationId, fromHostId); } + void finishActivation(PendingLocalTransfer transfer, TransferActivation activation) + { + getOrCreate(activation.activationId).finishActivation(transfer, activation); + } + + void receivedActivationResponse(CoordinatedTransfer transfer, InetAddressAndPort onHost) + { + int onHostId = ClusterMetadata.current().directory.peerId(onHost).id(); + getOrCreate(transfer.activationId).receivedActivationResponse(transfer, onHostId); + } + void updateReplicatedOffsets(List offsets, boolean durable, InetAddressAndPort onHost) { int onHostId = ClusterMetadata.current().directory.peerId(onHost).id(); @@ -255,6 +267,11 @@ List remoteReplicas() return replicas; } + boolean isDurablyReconciled(ShortMutationId id) + { + return logs.get(id.logId()).isDurablyReconciled(id); + } + boolean isDurablyReconciled(long logId, CoordinatorLogOffsets logOffsets) { return logs.get(logId).isDurablyReconciled(logOffsets); diff --git a/src/java/org/apache/cassandra/replication/TransferActivation.java b/src/java/org/apache/cassandra/replication/TransferActivation.java new file mode 100644 index 000000000000..c247c9fc0c46 --- /dev/null +++ b/src/java/org/apache/cassandra/replication/TransferActivation.java @@ -0,0 +1,186 @@ +/* + * 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.Objects; + +import com.google.common.base.Preconditions; + +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.IVerbHandler; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.NoPayload; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.utils.TimeUUID; + +public class TransferActivation +{ + public final TimeUUID transferId; + public final TimeUUID planId; + public final MutationId activationId; + public final NodeId coordinatorId; + public final Phase phase; + + public enum Phase + { + PREPARE(0), + COMMIT(1); + + private final int id; + private static final Phase[] ids; + static + { + ids = new Phase[values().length]; + for (Phase phase : values()) + ids[phase.id] = phase; + } + + Phase(int id) + { + this.id = id; + } + + static Phase from(int id) + { + Phase phase = ids[id]; + Preconditions.checkState(phase.id == id); + return phase; + } + } + + public TransferActivation(CoordinatedTransfer transfer, InetAddressAndPort peer, Phase phase) + { + this(transfer.transferId, transfer.streams.get(peer).planId(), transfer.activationId, ClusterMetadata.current().myNodeId(), phase); + } + + TransferActivation(TimeUUID transferId, TimeUUID planId, MutationId activationId, NodeId coordinatorId, Phase phase) + { + this.transferId = transferId; + Preconditions.checkArgument(!activationId.isNone()); + Preconditions.checkNotNull(planId); + Preconditions.checkNotNull(coordinatorId); + this.planId = planId; + this.activationId = activationId; + this.coordinatorId = coordinatorId; + this.phase = phase; + } + + MutationId id() + { + return activationId; + } + + public boolean isPropose() + { + return phase == Phase.PREPARE; + } + + public boolean isCommit() + { + return phase == Phase.COMMIT; + } + + public void apply() + { + MutationTrackingService.instance.activateLocal(this); + } + + public static final Serializer serializer = new Serializer(); + + public static class Serializer implements IVersionedSerializer + { + @Override + public void serialize(TransferActivation activate, DataOutputPlus out, int version) throws IOException + { + TimeUUID.Serializer.instance.serialize(activate.transferId, out, version); + TimeUUID.Serializer.instance.serialize(activate.planId, out, version); + MutationId.serializer.serialize(activate.activationId, out, version); + NodeId.messagingSerializer.serialize(activate.coordinatorId, out, version); + out.writeByte(activate.phase.id); + } + + @Override + public TransferActivation deserialize(DataInputPlus in, int version) throws IOException + { + TimeUUID transferId = TimeUUID.Serializer.instance.deserialize(in, version); + TimeUUID planId = TimeUUID.Serializer.instance.deserialize(in, version); + MutationId activationId = MutationId.serializer.deserialize(in, version); + NodeId coordinatorId = NodeId.messagingSerializer.deserialize(in, version); + Phase phase = Phase.from(in.readByte()); + return new TransferActivation(transferId, planId, activationId, coordinatorId, phase); + } + + @Override + public long serializedSize(TransferActivation activate, int version) + { + long size = 0; + size += TimeUUID.Serializer.instance.serializedSize(activate.transferId, version); + size += TimeUUID.Serializer.instance.serializedSize(activate.planId, version); + size += MutationId.serializer.serializedSize(activate.activationId, version); + size += NodeId.messagingSerializer.serializedSize(activate.coordinatorId, version); + size += TypeSizes.BYTE_SIZE; // Enum ordinal + return size; + } + } + + public static class VerbHandler implements IVerbHandler + { + @Override + public void doVerb(Message msg) throws IOException + { + msg.payload.apply(); + MessagingService.instance().respond(NoPayload.noPayload, msg); + } + } + + public static final VerbHandler verbHandler = new VerbHandler(); + + @Override + public String toString() + { + return "TransferActivation{" + + "transferId=" + transferId + + ", planId=" + planId + + ", activationId=" + activationId + + ", coordinatorId=" + coordinatorId + + ", phase=" + phase + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) return false; + TransferActivation that = (TransferActivation) o; + return Objects.equals(transferId, that.transferId) && Objects.equals(planId, that.planId) && Objects.equals(activationId, that.activationId) && Objects.equals(coordinatorId, that.coordinatorId) && phase == that.phase; + } + + @Override + public int hashCode() + { + return Objects.hash(transferId, planId, activationId, coordinatorId, phase); + } +} diff --git a/src/java/org/apache/cassandra/replication/TransferFailed.java b/src/java/org/apache/cassandra/replication/TransferFailed.java new file mode 100644 index 000000000000..73a9a4bf4452 --- /dev/null +++ b/src/java/org/apache/cassandra/replication/TransferFailed.java @@ -0,0 +1,66 @@ +/* + * 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 org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.utils.TimeUUID; + +public class TransferFailed +{ + final TimeUUID planId; + + public TransferFailed(TimeUUID planId) + { + this.planId = planId; + } + + public static final IVersionedSerializer serializer = new IVersionedSerializer() + { + @Override + public void serialize(TransferFailed t, DataOutputPlus out, int version) throws IOException + { + TimeUUID.Serializer.instance.serialize(t.planId, out, version); + } + + @Override + public TransferFailed deserialize(DataInputPlus in, int version) throws IOException + { + TimeUUID planId = TimeUUID.Serializer.instance.deserialize(in, version); + return new TransferFailed(planId); + } + + @Override + public long serializedSize(TransferFailed t, int version) + { + return TimeUUID.Serializer.instance.serializedSize(t.planId, version); + } + }; + + @Override + public String toString() + { + return "TransferFailed{" + + "planId=" + planId + + '}'; + } +} diff --git a/src/java/org/apache/cassandra/replication/UnreconciledMutations.java b/src/java/org/apache/cassandra/replication/UnreconciledMutations.java index 036f624b4295..87aed43bb493 100644 --- a/src/java/org/apache/cassandra/replication/UnreconciledMutations.java +++ b/src/java/org/apache/cassandra/replication/UnreconciledMutations.java @@ -27,11 +27,15 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.agrona.collections.Int2ObjectHashMap; import org.apache.cassandra.db.Mutation; import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.schema.TableId; /** @@ -40,9 +44,15 @@ */ public class UnreconciledMutations { + private static final Logger logger = LoggerFactory.getLogger(UnreconciledMutations.class); + + // Mutations (single-partition) private final Int2ObjectHashMap statesMap = new Int2ObjectHashMap<>(); private final SortedSet statesSet = new TreeSet<>(Entry.comparator); + // Transfers (partition-range) + private final ActivatedTransfers transfers = new ActivatedTransfers(); + enum Visibility { PENDING, // written to the journal, but not yet to LSM @@ -135,15 +145,23 @@ public void finishWriting(Mutation mutation) public void remove(int offset) { Entry state = statesMap.remove(offset); - if (state != null) + if (state == null) + transfers.removeOffset(offset); + else statesSet.remove(state); } + public void activatedTransfer(MutationId activationId, Collection sstables) + { + transfers.add(activationId, sstables); + } + public UnreconciledMutations copy() { UnreconciledMutations copy = new UnreconciledMutations(); copy.statesMap.putAll(statesMap); copy.statesSet.addAll(statesSet); + copy.transfers.addAll(transfers); return copy; } @@ -151,12 +169,14 @@ public boolean collect(AbstractBounds range, TableId tableId, { Entry start = Entry.start(range.left.getToken(), range.left.kind() != PartitionPosition.Kind.MAX_BOUND); Entry end = Entry.end(range.right.getToken(), range.right.kind() != PartitionPosition.Kind.MIN_BOUND); + transfers.forEachIntersecting(range, id -> into.add(id.offset())); return collect(start, end, tableId, includePending, into); } public boolean collect(Token token, TableId tableId, boolean includePending, Offsets.OffsetReciever into) { SortedSet subset = statesSet.subSet(Entry.start(token, true), Entry.end(token, true)); + transfers.forEachIntersecting(token, id -> into.add(id.offset())); return collect(subset, tableId, includePending, into); } @@ -210,7 +230,7 @@ else if (cmp > 0) @VisibleForTesting boolean equalsForTesting(UnreconciledMutations other) { - return this.statesMap.equals(other.statesMap) && this.statesSet.equals(other.statesSet); + return this.statesMap.equals(other.statesMap) && this.statesSet.equals(other.statesSet) && this.transfers.equals(other.transfers); } @VisibleForTesting @@ -243,10 +263,25 @@ static UnreconciledMutations loadFromJournal(Node2OffsetsMap witnessedOffsets, i for (int offset = iter.start(), end = iter.end(); offset <= end; offset++) { ShortMutationId id = new ShortMutationId(witnessed.logId, offset); - result.addDirectly(MutationJournal.instance.read(id)); + Mutation mutation = MutationJournal.instance.read(id); + if (mutation != null) + { + result.addDirectly(mutation); + continue; + } + CoordinatedTransfer transfer = LocalTransfers.instance().getActivatedTransfer(id); + if (transfer != null) + { + result.transfers.add(transfer.activationId, transfer.sstables); + continue; + } + + logger.error("Cannot load unknown mutation ID {}", id); } } + // Transfers are never present in the journal, since they're added as SSTables directly + return result; } } diff --git a/src/java/org/apache/cassandra/service/reads/tracked/ReadReconciliations.java b/src/java/org/apache/cassandra/service/reads/tracked/ReadReconciliations.java index 13e1e69d1074..33d3c68e2fec 100644 --- a/src/java/org/apache/cassandra/service/reads/tracked/ReadReconciliations.java +++ b/src/java/org/apache/cassandra/service/reads/tracked/ReadReconciliations.java @@ -23,6 +23,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.agrona.collections.IntArrayList; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.gms.FailureDetector; @@ -126,6 +129,7 @@ public void acceptSyncAck(InetAddressAndPort from, TrackedRead.Id id) public void acceptMutation(TrackedRead.Id id, ShortMutationId mutationId) { + logger.debug("Accepted mutation {} {}", id, mutationId); Coordinator reconcile = coordinators.get(id); if (reconcile != null && reconcile.acceptMutation(mutationId)) // could be already timed out / expired coordinators.remove(id); @@ -155,6 +159,8 @@ public int expire(long nanoTime) private static final class Coordinator { + private static final Logger logger = LoggerFactory.getLogger(Coordinator.class); + private static final AtomicLongFieldUpdater remainingUpdater = AtomicLongFieldUpdater.newUpdater(Coordinator.class, "remaining"); private volatile long remaining; // three values packed into one atomic long @@ -343,6 +349,7 @@ private long updateRemaining(int mutationsDelta, int summariesDelta, int syncAck int mutations = remainingMutations(prev) + mutationsDelta; int summaries = remainingSummaries(prev) + summariesDelta; int syncAcks = remainingSyncAcks(prev) + syncAcksDelta; + logger.trace("[Read {}] Still waiting for {} mutations, {} summaries, {} syncAcks", id, mutations, summaries, syncAcks); next = remaining(mutations, summaries, syncAcks); } while (!remainingUpdater.compareAndSet(this, prev, next)); return next; @@ -369,6 +376,8 @@ boolean isPurgeable(long nanoTime) } } + private static final Logger logger = LoggerFactory.getLogger(ReadReconciliations.class); + /** * @param node node id of the remote replica from which we got the summary * @param offsets offsets that we need to pull - from the coordinator, if alive, or from the @@ -389,6 +398,7 @@ private static void pull(int node, Offsets offsets, IncomingMutations.Callback c if (!toPull.isEmpty()) { PullMutationsRequest pull = new PullMutationsRequest(Offsets.Immutable.copy(toPull)); + logger.debug("Pulling {} from {}", pull, pullFrom); MessagingService.instance().send(Message.out(Verb.PULL_MUTATIONS_REQ, pull), pullFrom); } } diff --git a/src/java/org/apache/cassandra/service/reads/tracked/TrackedLocalReads.java b/src/java/org/apache/cassandra/service/reads/tracked/TrackedLocalReads.java index aa58dc331910..3223e9f25170 100644 --- a/src/java/org/apache/cassandra/service/reads/tracked/TrackedLocalReads.java +++ b/src/java/org/apache/cassandra/service/reads/tracked/TrackedLocalReads.java @@ -18,6 +18,7 @@ package org.apache.cassandra.service.reads.tracked; import java.util.ArrayList; +import java.util.Iterator; import java.util.Map; import com.google.common.annotations.VisibleForTesting; @@ -128,6 +129,9 @@ private void beginReadInternal(TrackedRead.Id readId, // any mutations that may have arrived during initial read execution. secondarySummary = command.createMutationSummary(true); processDelta(read, initialSummary, secondarySummary); + + // Include in summary any transfer IDs that were present for the read + secondarySummary = merge(controller.getActivationIds(), secondarySummary); } catch (Exception e) { @@ -150,6 +154,30 @@ private void beginReadInternal(TrackedRead.Id readId, ReadReconciliations.instance.acceptLocalSummary(readId, secondarySummary, summaryNodes); } + private static MutationSummary merge(Iterator activationIds, MutationSummary summary) + { + if (activationIds == null || !activationIds.hasNext()) + return summary; + + MutationSummary.Builder builder = new MutationSummary.Builder(summary.tableId()); + + // TODO: Make faster without a copy + for (int i = 0; i < summary.size(); i++) + { + MutationSummary.CoordinatorSummary coordinatorSummary = summary.get(i); + MutationSummary.CoordinatorSummary.Builder coordinatorSummaryBuilder = builder.builderForLog(coordinatorSummary.logId()); + coordinatorSummaryBuilder.unreconciled.addAll(coordinatorSummary.unreconciled); + coordinatorSummaryBuilder.reconciled.addAll(coordinatorSummary.reconciled); + } + + while (activationIds.hasNext()) + { + ShortMutationId id = activationIds.next(); + builder.builderForLog(id).unreconciled.add(id.offset()); + } + return builder.build(); + } + @VisibleForTesting public static void processDelta(PartialTrackedRead read, MutationSummary initialSummary, MutationSummary secondarySummary) { diff --git a/src/java/org/apache/cassandra/streaming/StreamOperation.java b/src/java/org/apache/cassandra/streaming/StreamOperation.java index b1c5908f7fe8..45dd7e9a2cb3 100644 --- a/src/java/org/apache/cassandra/streaming/StreamOperation.java +++ b/src/java/org/apache/cassandra/streaming/StreamOperation.java @@ -26,7 +26,8 @@ public enum StreamOperation BOOTSTRAP("Bootstrap", false, true, false), REBUILD("Rebuild", false, true, false), BULK_LOAD("Bulk Load", true, false, false), - REPAIR("Repair", true, false, true); + REPAIR("Repair", true, false, true), + TRACKED_TRANSFER("Tracked Transfer", false, false, false); private final String description; private final boolean requiresViewBuild; diff --git a/src/java/org/apache/cassandra/streaming/StreamPlan.java b/src/java/org/apache/cassandra/streaming/StreamPlan.java index 1f900032f0ec..4d249926264c 100644 --- a/src/java/org/apache/cassandra/streaming/StreamPlan.java +++ b/src/java/org/apache/cassandra/streaming/StreamPlan.java @@ -69,8 +69,7 @@ public StreamPlan(StreamOperation streamOperation, int connectionsPerHost, boolean connectSequentially, TimeUUID pendingRepair, PreviewKind previewKind) { this.streamOperation = streamOperation; - this.coordinator = new StreamCoordinator(streamOperation, connectionsPerHost, streamingFactory(), - false, connectSequentially, pendingRepair, previewKind); + this.coordinator = new StreamCoordinator(streamOperation, connectionsPerHost, streamingFactory(), false, connectSequentially, pendingRepair, previewKind); } /** diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-CompressionInfo.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-CompressionInfo.db index ba7ce3f01ae2..2591570214f7 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-CompressionInfo.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Data.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Data.db index ac900e5814d2..390767c7752f 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Data.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Data.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Digest.crc32 b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Digest.crc32 index 048ad49aafbc..c90437f307b0 100644 --- a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Digest.crc32 +++ b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Digest.crc32 @@ -1 +1 @@ -2530067741 \ No newline at end of file +2415910404 \ No newline at end of file diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Rows.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Rows.db index b1ffb7400f12..6d0a448d65ee 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Rows.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Rows.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Statistics.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Statistics.db index 8a8a4f1f4a04..c213386bcf22 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Statistics.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-Statistics.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-TOC.txt b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-TOC.txt index 298910cfdc58..5bfa06ac544e 100644 --- a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-TOC.txt +++ b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust/db-51-bti-TOC.txt @@ -1,8 +1,8 @@ +CompressionInfo.db Data.db -Statistics.db Digest.crc32 -TOC.txt -CompressionInfo.db Filter.db Partitions.db Rows.db +Statistics.db +TOC.txt diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-CompressionInfo.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-CompressionInfo.db index 4528bde73f26..389b20f1e1a5 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-CompressionInfo.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Data.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Data.db index 55c580d0b98f..28d8bda9fab6 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Data.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Data.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Digest.crc32 b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Digest.crc32 index 024f42fedf9d..bd884f1aec74 100644 --- a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Digest.crc32 +++ b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Digest.crc32 @@ -1 +1 @@ -1370392555 \ No newline at end of file +2922608909 \ No newline at end of file diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Rows.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Rows.db index b923a40bac2f..6d0a448d65ee 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Rows.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Rows.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Statistics.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Statistics.db index 4311795c8bcc..ec17d2798b54 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Statistics.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_be_index_summary/db-51-bti-Statistics.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-CompressionInfo.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-CompressionInfo.db index 867a774cb29e..f878d2fbd93c 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-CompressionInfo.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Data.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Data.db index 3f530d2cb652..554b8ce1df6c 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Data.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Data.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Digest.crc32 b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Digest.crc32 index ac6e4ec43839..89d8ecfd510b 100644 --- a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Digest.crc32 +++ b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Digest.crc32 @@ -1 +1 @@ -53972413 \ No newline at end of file +3585813939 \ No newline at end of file diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Rows.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Rows.db index 43d466285234..e6027e61f87e 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Rows.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Rows.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Statistics.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Statistics.db index 1c7f09c9d720..15b371e8ab39 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Statistics.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-Statistics.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-TOC.txt b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-TOC.txt index 298910cfdc58..5bfa06ac544e 100644 --- a/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-TOC.txt +++ b/test/data/legacy-sstables/db/legacy_tables/legacy_db_clust_counter/db-51-bti-TOC.txt @@ -1,8 +1,8 @@ +CompressionInfo.db Data.db -Statistics.db Digest.crc32 -TOC.txt -CompressionInfo.db Filter.db Partitions.db Rows.db +Statistics.db +TOC.txt diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-Data.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-Data.db index f082ef1be0d1..7ccb6bc5b2db 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-Data.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-Data.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-Digest.crc32 b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-Digest.crc32 index fd953e56ba6d..6979c2eeee45 100644 --- a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-Digest.crc32 +++ b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-Digest.crc32 @@ -1 +1 @@ -3597571582 \ No newline at end of file +851105930 \ No newline at end of file diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-Statistics.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-Statistics.db index 3bc02aa78a3f..b1e5e43221ef 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-Statistics.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-Statistics.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-TOC.txt b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-TOC.txt index 298910cfdc58..5bfa06ac544e 100644 --- a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-TOC.txt +++ b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple/db-51-bti-TOC.txt @@ -1,8 +1,8 @@ +CompressionInfo.db Data.db -Statistics.db Digest.crc32 -TOC.txt -CompressionInfo.db Filter.db Partitions.db Rows.db +Statistics.db +TOC.txt diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-Data.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-Data.db index 6bf3d6ff53ff..c743838b5dad 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-Data.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-Data.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-Digest.crc32 b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-Digest.crc32 index 687d787888bb..7fa37f31b393 100644 --- a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-Digest.crc32 +++ b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-Digest.crc32 @@ -1 +1 @@ -3514184066 \ No newline at end of file +4193692618 \ No newline at end of file diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-Statistics.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-Statistics.db index e8b44ee5991e..0eeb754e2f0d 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-Statistics.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-Statistics.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-TOC.txt b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-TOC.txt index 298910cfdc58..5bfa06ac544e 100644 --- a/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-TOC.txt +++ b/test/data/legacy-sstables/db/legacy_tables/legacy_db_simple_counter/db-51-bti-TOC.txt @@ -1,8 +1,8 @@ +CompressionInfo.db Data.db -Statistics.db Digest.crc32 -TOC.txt -CompressionInfo.db Filter.db Partitions.db Rows.db +Statistics.db +TOC.txt diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-Data.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-Data.db index aa261ff13b4f..3aa1ac183936 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-Data.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-Data.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-Digest.crc32 b/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-Digest.crc32 index 0c820c94b796..4d83fd345300 100644 --- a/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-Digest.crc32 +++ b/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-Digest.crc32 @@ -1 +1 @@ -1960712659 \ No newline at end of file +2761811502 \ No newline at end of file diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-Statistics.db b/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-Statistics.db index a3c99f13c96e..6c3cfb2141b1 100644 Binary files a/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-Statistics.db and b/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-Statistics.db differ diff --git a/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-TOC.txt b/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-TOC.txt index 298910cfdc58..5bfa06ac544e 100644 --- a/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-TOC.txt +++ b/test/data/legacy-sstables/db/legacy_tables/legacy_db_tuple/db-51-bti-TOC.txt @@ -1,8 +1,8 @@ +CompressionInfo.db Data.db -Statistics.db Digest.crc32 -TOC.txt -CompressionInfo.db Filter.db Partitions.db Rows.db +Statistics.db +TOC.txt diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-CompressionInfo.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-CompressionInfo.db index efba218e453b..6a4bc503e6df 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-CompressionInfo.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Data.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Data.db index fbc8ce33b263..40356af60266 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Data.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Data.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Digest.crc32 b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Digest.crc32 index f0bd564de9ea..e38e8e8b8429 100644 --- a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Digest.crc32 +++ b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Digest.crc32 @@ -1 +1 @@ -2672104528 \ No newline at end of file +733768223 \ No newline at end of file diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Index.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Index.db index de64643dd9e9..c87b8775e31f 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Index.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Index.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Statistics.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Statistics.db index ae9b9467a7bd..95f78c9d227b 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Statistics.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust/ob-51-big-Statistics.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-CompressionInfo.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-CompressionInfo.db index 962fd24580c5..2b305a35fad0 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-CompressionInfo.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Data.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Data.db index 75270dc694f0..58a6fae174c6 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Data.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Data.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Digest.crc32 b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Digest.crc32 index eb2c8d0807ce..a047e36a2b45 100644 --- a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Digest.crc32 +++ b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Digest.crc32 @@ -1 +1 @@ -2155890726 \ No newline at end of file +1457015109 \ No newline at end of file diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Index.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Index.db index bd25a71d86d1..c87b8775e31f 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Index.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Index.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Statistics.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Statistics.db index 1d4529efcccb..9840fc1f467a 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Statistics.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_be_index_summary/ob-51-big-Statistics.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-CompressionInfo.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-CompressionInfo.db index e661f5226721..13ceb02c9674 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-CompressionInfo.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-CompressionInfo.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Data.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Data.db index 1912950610bc..fadf92bb8bbc 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Data.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Data.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Digest.crc32 b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Digest.crc32 index a570cc66c831..807e80dc3740 100644 --- a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Digest.crc32 +++ b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Digest.crc32 @@ -1 +1 @@ -1361141780 \ No newline at end of file +880796784 \ No newline at end of file diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Index.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Index.db index 4d6412ce5929..e0f9d44d9b00 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Index.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Index.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Statistics.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Statistics.db index 7398ca55ce07..99f23ae3b9f8 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Statistics.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_clust_counter/ob-51-big-Statistics.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple/ob-51-big-Data.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple/ob-51-big-Data.db index 0e5836f0a918..30c195a10a93 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple/ob-51-big-Data.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple/ob-51-big-Data.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple/ob-51-big-Digest.crc32 b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple/ob-51-big-Digest.crc32 index a83276d89106..165812c255cd 100644 --- a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple/ob-51-big-Digest.crc32 +++ b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple/ob-51-big-Digest.crc32 @@ -1 +1 @@ -1375280580 \ No newline at end of file +2614540741 \ No newline at end of file diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple/ob-51-big-Statistics.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple/ob-51-big-Statistics.db index 1c84c471c1cb..3b1520a649f0 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple/ob-51-big-Statistics.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple/ob-51-big-Statistics.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple_counter/ob-51-big-Data.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple_counter/ob-51-big-Data.db index 060f1a3fa278..611e15824494 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple_counter/ob-51-big-Data.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple_counter/ob-51-big-Data.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple_counter/ob-51-big-Digest.crc32 b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple_counter/ob-51-big-Digest.crc32 index a880c21aeb22..0e892223f0a1 100644 --- a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple_counter/ob-51-big-Digest.crc32 +++ b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple_counter/ob-51-big-Digest.crc32 @@ -1 +1 @@ -575257059 \ No newline at end of file +19151750 \ No newline at end of file diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple_counter/ob-51-big-Statistics.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple_counter/ob-51-big-Statistics.db index 23503f1c02ab..3e58e0c3b292 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple_counter/ob-51-big-Statistics.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_simple_counter/ob-51-big-Statistics.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_tuple/ob-51-big-Data.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_tuple/ob-51-big-Data.db index 40a42c6bd351..d8e9eab6f2b5 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_tuple/ob-51-big-Data.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_tuple/ob-51-big-Data.db differ diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_tuple/ob-51-big-Digest.crc32 b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_tuple/ob-51-big-Digest.crc32 index a38308e51f8d..c52c5f658f63 100644 --- a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_tuple/ob-51-big-Digest.crc32 +++ b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_tuple/ob-51-big-Digest.crc32 @@ -1 +1 @@ -383358291 \ No newline at end of file +4267206168 \ No newline at end of file diff --git a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_tuple/ob-51-big-Statistics.db b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_tuple/ob-51-big-Statistics.db index 163fede2849d..c5de8b85657e 100644 Binary files a/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_tuple/ob-51-big-Statistics.db and b/test/data/legacy-sstables/ob/legacy_tables/legacy_ob_tuple/ob-51-big-Statistics.db differ diff --git a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java index a9fe78f6429f..07cd610afb9b 100644 --- a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java +++ b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java @@ -1744,4 +1744,14 @@ public static List getPrimaryRanges(IInvokableInstance instance, String k .collect(Collectors.toList()); }); } + + public static Integer instanceId(IInvokableInstance instance) + { + return instance.callOnInstance(() -> { + ClassLoader cl = Thread.currentThread().getContextClassLoader(); + if (cl instanceof InstanceClassLoader) + return ((InstanceClassLoader) cl).getInstanceId(); + return null; + }); + } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/tracking/BulkTransfersTest.java b/test/distributed/org/apache/cassandra/distributed/test/tracking/BulkTransfersTest.java new file mode 100644 index 000000000000..01056cecb516 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/tracking/BulkTransfersTest.java @@ -0,0 +1,812 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.tracking; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import org.junit.Ignore; +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.MethodDelegation; +import net.bytebuddy.implementation.bind.annotation.SuperCall; +import net.bytebuddy.implementation.bind.annotation.This; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.compaction.CompactionManager; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.streaming.CassandraStreamReceiver; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.ICoordinator; +import org.apache.cassandra.distributed.api.IInstanceInitializer; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.IIsolatedExecutor; +import org.apache.cassandra.distributed.shared.ClusterUtils; +import org.apache.cassandra.distributed.shared.Uninterruptibles; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.sstable.CQLSSTableWriter; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.replication.ActivatedTransfers; +import org.apache.cassandra.replication.ImmutableCoordinatorLogOffsets; +import org.apache.cassandra.replication.MutationSummary; +import org.apache.cassandra.replication.MutationTrackingService; +import org.apache.cassandra.replication.ShortMutationId; +import org.apache.cassandra.replication.TransferActivation; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ownership.DataPlacement; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.assertj.core.api.Assertions; + +import static net.bytebuddy.matcher.ElementMatchers.named; +import static net.bytebuddy.matcher.ElementMatchers.takesNoArguments; +import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows; +import static org.apache.cassandra.distributed.shared.AssertUtils.row; +import static org.apache.cassandra.replication.TransferActivation.Phase.COMMIT; +import static org.apache.cassandra.replication.TransferActivation.Phase.PREPARE; + +/** + * For now, tracked import with a replica down is not supported. The intention is to support this scenario by allowing + * users to provide a {@link ConsistencyLevel} for tracked import operations, where the import will complete if + * sufficient replicas acknowledge the transfer and activate it. + */ +public class BulkTransfersTest extends TestBaseImpl +{ + private static final Logger logger = LoggerFactory.getLogger(BulkTransfersTest.class); + + private static final String TABLE = "tbl"; + private static final String KEYSPACE_TABLE = String.format("%s.%s", KEYSPACE, TABLE); + private static final String TABLE_SCHEMA_CQL = String.format(withKeyspace("CREATE TABLE %s." + TABLE + " (k int primary key, v int);")); + + private static final int IMPORT_PK = 1; + private static final Token IMPORT_TOKEN = Murmur3Partitioner.instance.getToken(Int32Type.instance.decompose(IMPORT_PK)); + private static final int NODES = 3; + + private static final IIsolatedExecutor.SerializableConsumer TRANSFERS_EXIST = sstable -> { + Assertions.assertThat(sstable.getCoordinatorLogOffsets().transfers()) + .isNotEmpty(); + Assertions.assertThat(sstable.isRepaired()).isFalse(); + }; + private static final IIsolatedExecutor.SerializableConsumer TRANSFERS_EMPTY = sstable -> { + Assertions.assertThat(sstable.getCoordinatorLogOffsets().transfers()) + .isEmpty(); + Assertions.assertThat(sstable.isRepaired()).isTrue(); + }; + private static final IIsolatedExecutor.SerializableConsumer NOOP = sstable -> {}; + + @Test + public void importHappyPath() throws Throwable + { + try (Cluster cluster = cluster()) + { + createSchema(cluster); + doImport(cluster); + + // All pending/ dirs should be empty, should have no SSTables left if all the transfers completed + assertPendingDirs(cluster, (File pendingUuidDir) -> { + Assertions.assertThat(pendingUuidDir.listUnchecked(File::isFile)).isEmpty(); + }); + + // Verify transfer IDs exist before compaction, then compact, then verify they're removed + assertCompaction(cluster, cluster, TRANSFERS_EXIST, TRANSFERS_EMPTY); + + // Run after compaction, to enforce offset persistence + broadcast + assertSummary(cluster, summary -> { + Assertions.assertThat(summary).satisfies(s -> { + assert s.reconciledIds() == 1; + assert s.unreconciledIds() == 0; + }); + }); + + assertLocalSelect(cluster, rows -> assertRows(rows, row(1, 1))); + } + } + + @Test + @Ignore + public void importReplicaDown() throws Throwable + { + try (Cluster cluster = cluster()) + { + createSchema(cluster); + + Iterable down = Collections.singleton(cluster.get(3)); + Iterable up = cluster.stream().filter(instance -> instance != down).collect(Collectors.toList()); + for (IInvokableInstance instance : down) + instance.shutdown().get(); + + doImport(cluster); + + cluster.get(3).startup(); + + // Transfers did not complete, files should still exist on up replicas + assertPendingDirs(up, (File pendingUuidDir) -> { + Assertions.assertThat(pendingUuidDir.listUnchecked(File::isFile)).isNotEmpty(); + }); + assertPendingDirs(down, (File pendingUuidDir) -> { + Assertions.assertThat(pendingUuidDir.listUnchecked(File::isFile)).isEmpty(); + }); + + // Transfers did not complete, transfer IDs should not be removed + assertCompaction(cluster, cluster, TRANSFERS_EXIST, TRANSFERS_EXIST); + + assertLocalSelect(up, rows -> assertRows(rows, row(1, 1))); + } + } + + @Test + public void importMissedActivationPrepare() throws Throwable + { + importMissedActivation(PREPARE); + } + + @Test + public void importMissedActivationCommit() throws Throwable + { + importMissedActivation(COMMIT); + } + + public void importMissedActivation(TransferActivation.Phase phase) throws Throwable + { + int MISSED_ACTIVATION = 2; + try (Cluster cluster = cluster(ByteBuddyInjections.SkipActivation.install(MISSED_ACTIVATION))) + { + ByteBuddyInjections.SkipActivation.setup(cluster, phase); + createSchema(cluster); + + Set missed = Collections.singleton(cluster.get(MISSED_ACTIVATION)); + Iterable received = cluster.stream().filter(instance -> !missed.contains(instance)).collect(Collectors.toList()); + + Assertions.assertThatThrownBy(() -> doImport(cluster)) + .hasMessageContaining("Failed adding SSTables") + .cause() + .hasMessage("Tracked import failed during " + phase + " on " + cluster.get(MISSED_ACTIVATION).broadcastAddress() + " due to TIMEOUT") + .hasNoCause(); + + assertSummary(received, summary -> { + Assertions.assertThat(summary).satisfies(s -> { + assert s.reconciledIds() == 0; + assert s.unreconciledIds() == (phase == COMMIT ? 1 : 0); + }); + }); + assertSummary(missed, summary -> { + Assertions.assertThat(summary).satisfies(s -> { + assert s.reconciledIds() == 0; + assert s.unreconciledIds() == 0; + }); + }); + + switch (phase) + { + case PREPARE: + // Activation did not start, files should be cleaned up + Uninterruptibles.sleepUninterruptibly(5, TimeUnit.SECONDS); + assertPendingDirs(cluster, (File pendingUuidDir) -> { + Assertions.assertThat(pendingUuidDir.listUnchecked(File::isFile)).isEmpty(); + }); + break; + case COMMIT: + // Activation did not complete, files should still exist on all replicas + assertPendingDirs(cluster, (File pendingUuidDir) -> { + Assertions.assertThat(pendingUuidDir.listUnchecked(File::isFile)).isNotEmpty(); + }); + break; + } + + // If the activation is not everywhere, it shouldn't be purged on compaction + assertCompaction(cluster, received, TRANSFERS_EXIST, TRANSFERS_EXIST); + + if (phase == PREPARE) + return; + + // Permit activation of missed commits during read reconciliation + ByteBuddyInjections.SkipActivation.setup(cluster, null); + + // Use coordinated query rather to confirm read reconciliation triggers activation + IInvokableInstance coordinator = cluster.get(3); // not initial transfer coordinator, but received activation + assertCoordinatedRead(coordinator, rows -> { + assertRows(rows, row(1, 1)); + }); + + // Confirm others receive activation + assertLocalSelect(missed, rows -> { + assertRows(rows, row(1, 1)); + }); + + assertCompaction(cluster, cluster, TRANSFERS_EXIST, TRANSFERS_EMPTY); + + // Activation completed, files should be removed + assertPendingDirs(cluster, (File pendingUuidDir) -> { + Assertions.assertThat(pendingUuidDir.listUnchecked(File::isFile)).isEmpty(); + }); + } + } + + /* + * When an import fails, bounce must not move the pending SSTables into the live set. + */ + @Test + public void importBounceAfterPending() throws Throwable + { + IInstanceInitializer initializer = ByteBuddyInjections.SkipActivation.install(1, 2, 3); + try (Cluster cluster = cluster(initializer)) + { + ByteBuddyInjections.SkipActivation.setup(cluster, COMMIT); + createSchema(cluster); + + Assertions.assertThatThrownBy(() -> doImport(cluster)) + .hasMessageContaining("Failed adding SSTables") + .cause() + .hasMessageContaining("Tracked import failed during COMMIT"); + + Runnable assertEmpty = () -> { + // Activation did not complete, files should still exist on all replicas + assertPendingDirs(cluster, (File pendingUuidDir) -> { + Assertions.assertThat(pendingUuidDir.listUnchecked(File::isFile)).isNotEmpty(); + }); + + // No one has activated, so should not be present in any summary + assertSummary(cluster, summary -> { + Assertions.assertThat(summary).satisfies(s -> { + assert s.reconciledIds() == 0; + assert s.unreconciledIds() == 0; + }); + }); + + assertLocalSelect(cluster, rows -> assertRows(rows, EMPTY_ROWS)); + }; + + assertEmpty.run(); + + bounce(cluster); + + assertEmpty.run(); + } + } + + @Test + public void importOutOfRange() throws Throwable + { + try (Cluster cluster = cluster()) + { + createSchema(cluster, 1); + + Set inRange = new HashSet<>(); + Set outOfRange = new HashSet<>(); + cluster.forEach(instance -> { + boolean importReplica = instance.callOnInstance(() -> { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, TABLE); + DataPlacement placement = ClusterMetadata.current().placements.get(cfs.keyspace.getMetadata().params.replication); + return placement.writes.forToken(IMPORT_TOKEN).get().containsSelf(); + }); + (importReplica ? inRange : outOfRange).add(instance); + }); + logger.info("inRange: {}, outOfRange: {}", inRange, outOfRange); + + Assertions.assertThat(inRange).hasSize(1); + IInvokableInstance onlyInRange = inRange.iterator().next(); + + // Reject import out of range + for (IInvokableInstance instance : outOfRange) + { + long mark = instance.logs().mark(); + Consumer> onResult = failedDirs -> Assertions.assertThat(failedDirs).hasSize(1); + doImport(cluster, instance, onResult); + instance.logs().grep(mark, "java.lang.RuntimeException: Key DecoratedKey(-4069959284402364209, 00000001) is not contained in the given ranges"); + } + + doImport(cluster, onlyInRange); + + assertSummary(Collections.singleton(onlyInRange), summary -> { + Assertions.assertThat(summary).satisfies(s -> { + assert s.reconciledIds() == 1; + assert s.unreconciledIds() == 0; + }); + }); + + for (IInvokableInstance instance : outOfRange) + { + // Out of range shouldn't have any transfers + assertCompaction(cluster, Collections.singleton(instance), TRANSFERS_EMPTY, TRANSFERS_EMPTY); + + // Run after compaction, to enforce offset persistence + broadcast + assertSummary(Collections.singleton(instance), summary -> { + Assertions.assertThat(summary).satisfies(s -> { + assert s.reconciledIds() == 0; + assert s.unreconciledIds() == 0; + }); + }); + } + } + } + + /* + * Ensure that activation IDs attached to SSTables aren't spread across Token boundaries by compaction. + * + * For example: + * IMPORT_TOKEN is owned by replicas (A, B) + * OUTSIDE_IMPORT_TOKEN is owned by replicas (B, C) + * Execute import so (A, B) have IMPORT_TOKEN + * Execute plain write so (B, C) have OUTSIDE_IMPORT_TOKEN + * Do a major compaction on B so IMPORT_TOKEN and OUTSIDE_IMPORT_TOKEN are compacted together into the same SSTable + * Execute a data read for OUTSIDE_IMPORT_TOKEN against B, ensure it doesn't contain any activation IDs + */ + @Test + public void importActivationMergedByCompaction() throws Throwable + { + IInstanceInitializer initializer = (cl, tg, instance, gen) -> { + ByteBuddyInjections.SkipPurgeTransfers.install().initialise(cl, tg, instance, gen); + }; + try (Cluster cluster = cluster(initializer)) + { + createSchema(cluster, 2); + + Set inImportRange = new HashSet<>(); + cluster.forEach(instance -> { + logger.debug("Instance {} ring is {}", ClusterUtils.instanceId(instance), ClusterUtils.ring(instance)); + boolean isInRange = instance.callOnInstance(() -> { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, TABLE); + DataPlacement placement = ClusterMetadata.current().placements.get(cfs.keyspace.getMetadata().params.replication); + return placement.writes.forToken(IMPORT_TOKEN).get().containsSelf(); + }); + if (isInRange) + inImportRange.add(instance); + }); + Assertions.assertThat(inImportRange).hasSize(2); + + // Find a partition key that's not owned by the same replicas as the import + Murmur3Partitioner.LongToken NON_IMPORT_TOKEN = new Murmur3Partitioner.LongToken(IMPORT_TOKEN.getLongValue() * 3); + int NON_IMPORT_PK = Int32Type.instance.compose(Murmur3Partitioner.LongToken.keyForToken(NON_IMPORT_TOKEN)); + + Set inNonImportRange = new HashSet<>(); + cluster.forEach(instance -> { + boolean isInRange = instance.callOnInstance(() -> { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, TABLE); + DataPlacement placement = ClusterMetadata.current().placements.get(cfs.keyspace.getMetadata().params.replication); + return placement.writes.forToken(NON_IMPORT_TOKEN).get().containsSelf(); + }); + if (isInRange) + inNonImportRange.add(instance); + }); + Assertions.assertThat(inNonImportRange).hasSize(2); + Assertions.assertThat(inNonImportRange).isNotEqualTo(inImportRange); + + // Import: (A, B) + // Plain: (B, C) + IInvokableInstance A = null; + IInvokableInstance B = null; + IInvokableInstance C = null; + for (IInvokableInstance instance : cluster) + { + boolean isImport = inImportRange.contains(instance); + boolean isNonImport = inNonImportRange.contains(instance); + if (isImport && isNonImport) + B = instance; + else if (isImport) + A = instance; + else if (isNonImport) + C = instance; + }; + Assertions.assertThat(A).isNotNull(); + Assertions.assertThat(B).isNotNull(); + Assertions.assertThat(C).isNotNull(); + + doImport(cluster, A); + assertLocalSelect(List.of(A, B), (IIsolatedExecutor.SerializableConsumer) rows -> { + assertRows(rows, row(IMPORT_PK, IMPORT_PK)); + }); + + ShortMutationId importActivationId = callSerialized(A, () -> ShortMutationId.serializer, () -> { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, TABLE); + for (SSTableReader sstable : cfs.getLiveSSTables()) + { + ActivatedTransfers transfers = sstable.getCoordinatorLogOffsets().transfers(); + if (!transfers.isEmpty()) + return transfers.iterator().next(); + } + return null; + }); + Assertions.assertThat(importActivationId).isNotNull(); + C.coordinator().execute(withKeyspace("INSERT INTO %s." + TABLE + "(k, v) VALUES (?, ?)"), ConsistencyLevel.ALL, NON_IMPORT_PK, NON_IMPORT_PK); + assertCompaction(cluster, Collections.singleton(B), NOOP, NOOP); + + // Reading from B for a range that doesn't include the import shouldn't include any transfer IDs, even though they've been compacted together + long mark = B.logs().mark(); + Object[][] rows = B.coordinator().execute(withKeyspace("SELECT * FROM %s." + TABLE + " WHERE k = ?"), ConsistencyLevel.ALL, NON_IMPORT_PK); + assertRows(rows, row(NON_IMPORT_PK, NON_IMPORT_PK)); + Assertions.assertThat(B.logs().grep(mark, "Adding overlapping activation ID ").getResult()).isEmpty(); + + // But if the read range does include a transfer ID, it should have been added + mark = B.logs().mark(); + rows = B.coordinator().execute(withKeyspace("SELECT * FROM %s." + TABLE + " WHERE k = ?"), ConsistencyLevel.ALL, IMPORT_PK); + assertRows(rows, row(IMPORT_PK, IMPORT_PK)); + Assertions.assertThat(B.logs().grep(mark, "Adding overlapping activation ID ").getResult()).isNotEmpty(); + } + } + + @Test + public void importFailedStreamCleanup() throws Throwable + { + int FAILED_STREAM = 3; + try (Cluster cluster = cluster(ByteBuddyInjections.FailIncomingStream.install(FAILED_STREAM))) + { + createSchema(cluster); + + IInvokableInstance importer = cluster.get(1); + IInvokableInstance missed = cluster.get(FAILED_STREAM); + + long mark = importer.logs().mark(); + Assertions.assertThatThrownBy(() -> doImport(cluster, importer)) + .isInstanceOf(RuntimeException.class) + .cause() + .isInstanceOf(RuntimeException.class) + .cause() + .hasMessageContaining("Remote peer " + missed.broadcastAddress() + " failed stream session"); + List logs = importer.logs().watchFor(mark, "Remote peer " + missed.broadcastAddress().toString() + " failed stream session").getResult(); + Assertions.assertThat(logs).isNotEmpty(); + + // Await cleanup of failed stream + Uninterruptibles.sleepUninterruptibly(5, TimeUnit.SECONDS); + + assertPendingDirs(cluster, (File pendingUuidDir) -> { + Assertions.assertThat(pendingUuidDir.listUnchecked(File::isFile)).isEmpty(); + }); + + assertLocalSelect(cluster, rows -> { + assertRows(rows); // empty + }); + } + } + + public static class ByteBuddyInjections + { + // Only skips direct transfer activation, not activation as part of read reconciliation + public static class SkipActivation + { + // null to not skip + public static volatile TransferActivation.Phase phase; + + public static IInstanceInitializer install(int...nodes) + { + return (ClassLoader cl, ThreadGroup tg, int num, int generation) -> { + for (int node : nodes) + if (node == num) + new ByteBuddy().rebase(TransferActivation.VerbHandler.class) + .method(named("doVerb")) + .intercept(MethodDelegation.to(ByteBuddyInjections.SkipActivation.class)) + .make() + .load(cl, ClassLoadingStrategy.Default.INJECTION); + }; + } + + // 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)); + } + + @SuppressWarnings("unused") + public static void doVerb(Message msg, @SuperCall Callable zuper) + { + if (phase != null && msg.payload.phase == SkipActivation.phase) + { + logger.info("Skipping activation for test: {}", msg.payload); + return; + } + + logger.info("Test running activation as usual: {}", msg.payload); + + try + { + zuper.call(); + } + catch (Exception e) + { + throw new RuntimeException(e); + } + } + } + + // ImmutableCoordinatorLogOffsets.Builder.purgeTransfers() + public static class SkipPurgeTransfers + { + public static IInstanceInitializer install() + { + return (ClassLoader cl, ThreadGroup tg, int num, int generation) -> { + new ByteBuddy().rebase(ImmutableCoordinatorLogOffsets.Builder.class) + .method(named("purgeTransfers").and(takesNoArguments())) + .intercept(MethodDelegation.to(SkipPurgeTransfers.class)) + .make() + .load(cl, ClassLoadingStrategy.Default.INJECTION); + }; + } + + @SuppressWarnings("unused") + public static void purgeTransfers() + { + logger.debug("Skipping purgeTransfers for test"); + } + } + + // CassandraStreamReceiver.finished + public static class FailIncomingStream + { + private static volatile boolean enabled = true; + + public static IInstanceInitializer install(int... nodes) + { + return (ClassLoader cl, ThreadGroup tg, int num, int generation) -> { + for (int node : nodes) + if (node == num) + new ByteBuddy().rebase(CassandraStreamReceiver.class) + .method(named("finished").and(takesNoArguments())) + .intercept(MethodDelegation.to(FailIncomingStream.class)) + .make() + .load(cl, ClassLoadingStrategy.Default.INJECTION); + }; + } + + @SuppressWarnings("unused") + public static void finished(@This CassandraStreamReceiver self) + { + throw new RuntimeException("Failing incoming stream for test"); + } + + public static void toggle(Cluster cluster, boolean enable) + { + enabled = enable; + cluster.forEach(instance -> instance.runOnInstance(() -> FailIncomingStream.enabled = enable)); + } + } + } + + private static Cluster cluster() throws IOException + { + return cluster((cl, tg, instance, generation) -> {}); + } + + private static Cluster cluster(IInstanceInitializer initializer) throws IOException + { + return Cluster.build(NODES) + .withConfig(cfg -> cfg.with(Feature.NETWORK) + .with(Feature.GOSSIP) + .set("mutation_tracking_enabled", "true") + .set("write_request_timeout", "1000ms") + .set("autocompaction_on_startup_enabled", false) + .set("repair_request_timeout", "2s") + .set("stream_transfer_task_timeout", "10s")) + .withInstanceInitializer(initializer) + .start(); + } + + private static void createSchema(Cluster cluster) + { + createSchema(cluster, NODES); + } + + private static void createSchema(Cluster cluster, int rf) + { + cluster.schemaChange(String.format(withKeyspace("CREATE KEYSPACE %s WITH replication = " + + "{'class': 'SimpleStrategy', 'replication_factor': " + rf + "} " + + "AND replication_type='tracked';"))); + cluster.schemaChange(TABLE_SCHEMA_CQL); + } + + private static void doImport(Cluster cluster) throws IOException + { + doImport(cluster, cluster.get(1)); + } + + private static void doImport(Cluster cluster, IInvokableInstance target) throws IOException + { + doImport(cluster, target, failedDirs -> Assertions.assertThat(failedDirs).isEmpty()); + } + + private static void doImport(Cluster cluster, IInvokableInstance target, Consumer> onFailedDirs) throws IOException + { + String file = Files.createTempDirectory(MutationTrackingTest.class.getSimpleName()).toString(); + + // Needs to run outside of instance executor because creates schema + try (CQLSSTableWriter writer = CQLSSTableWriter.builder() + .forTable(TABLE_SCHEMA_CQL) + .inDirectory(file) + .using("INSERT INTO " + KEYSPACE_TABLE + " (k, v) " + "VALUES (?, ?)") + .build()) + { + writer.addRow(IMPORT_PK, 1); + } + + assertLocalSelect(cluster, rows -> { + assertRows(rows); // empty + }); + + List failed = target.callOnInstance(() -> { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, TABLE); + Set paths = Set.of(file); + logger.info("Importing SSTables {}", paths); + return cfs.importNewSSTables(paths, true, true, true, true, true, true, true); + }); + + // Sleep for a while to make sure import completes + Uninterruptibles.sleepUninterruptibly(3, TimeUnit.SECONDS); + onFailedDirs.accept(failed); + } + + private static void assertCoordinatedRead(IInvokableInstance instance, IIsolatedExecutor.SerializableConsumer onRows) + { + ICoordinator coordinator = instance.coordinator(); + String cql = "SELECT * FROM %s." + TABLE + " WHERE k = 1"; + Object[][] rows = coordinator.execute(withKeyspace(cql), ConsistencyLevel.ALL); + onRows.accept(rows); + } + + private static void assertPendingDirs(Iterable validate, IIsolatedExecutor.SerializableConsumer forPendingUuidDir) + { + for (IInvokableInstance instance : validate) + { + instance.runOnInstance(() -> { + Set allPendingDirs = ColumnFamilyStore.getIfExists(KEYSPACE, TABLE).getDirectories().getPendingLocations(); + for (File pendingDir : allPendingDirs) + { + File[] pendingUuidDirs = pendingDir.listUnchecked(File::isDirectory); + for (File pendingUuidDir : pendingUuidDirs) + { + forPendingUuidDir.accept(pendingUuidDir); + } + } + }); + } + } + + private static void assertSummary(Iterable validate, IIsolatedExecutor.SerializableConsumer onSummary) + { + for (IInvokableInstance instance : validate) + { + instance.runOnInstance(() -> { + DecoratedKey key = DatabaseDescriptor.getPartitioner().decorateKey(ByteBufferUtil.bytes(1)); + TableId tableId = ColumnFamilyStore.getIfExists(KEYSPACE, TABLE).metadata().id; + MutationSummary summary = MutationTrackingService.instance.createSummaryForKey(key, tableId, false); + logger.debug("Validating summary {}", summary); + onSummary.accept(summary); + }); + } + } + + private static void assertCompaction(Cluster cluster, Iterable validate, + IIsolatedExecutor.SerializableConsumer before, + IIsolatedExecutor.SerializableConsumer after) + { + for (IInvokableInstance instance : validate) + { + instance.runOnInstance(() -> { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, TABLE); + for (SSTableReader sstable : cfs.getLiveSSTables()) + { + logger.info("SSTable {} before compaction: {}", sstable.getFilename(), sstable.getCoordinatorLogOffsets()); + before.accept(sstable); + } + }); + } + + // Activation ID must be persisted and broadcast across all peers in the cluster for any to mark as persisted + reconciled + cluster.forEach(i -> { + i.runOnInstance(() -> { + MutationTrackingService.instance.persistLogStateForTesting(); + MutationTrackingService.instance.broadcastOffsetsForTesting(); + }); + }); + + // Broadcast is async, wait until completion + Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); + + for (IInvokableInstance instance : validate) + { + instance.runOnInstance(() -> { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, TABLE); + logger.info("Triggering compaction on instance {}", cfs.metadata.keyspace); + CompactionManager.instance.performMaximal(cfs); + + for (SSTableReader sstable : cfs.getLiveSSTables()) + { + logger.info("SSTable {} after compaction: {}", sstable.getFilename(), sstable.getCoordinatorLogOffsets()); + after.accept(sstable); + } + }); + } + } + + private static void assertLocalSelect(Iterable validate, IIsolatedExecutor.SerializableConsumer onRows) + { + for (IInvokableInstance instance : validate) + { + { + Object[][] rows = instance.executeInternal(withKeyspace("SELECT * FROM %s." + TABLE + " WHERE k = 1")); + onRows.accept(rows); + } + { + Object[][] rows = instance.executeInternal(withKeyspace("SELECT * FROM %s." + TABLE)); + onRows.accept(rows); + } + } + } + + private static void bounce(Cluster cluster) + { + cluster.forEach(instance -> { + try + { + instance.shutdown().get(); + } + catch (InterruptedException | ExecutionException e) + { + throw new RuntimeException(e); + } + instance.startup(); + }); + } + + private static T callSerialized(IInvokableInstance instance, IIsolatedExecutor.SerializableSupplier> serializer, IIsolatedExecutor.SerializableCallable callable) + { + ByteBuffer serialized = instance.callOnInstance(() -> { + T deserialized = callable.call(); + IVersionedSerializer serialize = serializer.get(); + try + { + return serialize.serialize(deserialized, MessagingService.current_version); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }); + try + { + return serializer.get().deserialize(serialized, MessagingService.current_version); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + } +} diff --git a/test/unit/org/apache/cassandra/ServerTestUtils.java b/test/unit/org/apache/cassandra/ServerTestUtils.java index 702e2b687100..649b4cd101bc 100644 --- a/test/unit/org/apache/cassandra/ServerTestUtils.java +++ b/test/unit/org/apache/cassandra/ServerTestUtils.java @@ -49,6 +49,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.BaseProximity; +import org.apache.cassandra.replication.MutationJournal; import org.apache.cassandra.security.ThreadAwareSecurityManager; import org.apache.cassandra.service.DiskErrorsHandlerService; import org.apache.cassandra.service.EmbeddedCassandraService; @@ -153,6 +154,9 @@ public static void prepareServerNoRegister() { daemonInitialization(); + // Need to happen after daemonInitialization for config to be set, but before CFS initialization + MutationJournal.instance.start(); + if (isServerPrepared) return; diff --git a/test/unit/org/apache/cassandra/db/CoordinatorLogOffsetsLifecycleTest.java b/test/unit/org/apache/cassandra/db/CoordinatorLogOffsetsLifecycleTest.java index 20d098fcb10b..da9a1c2a6b57 100644 --- a/test/unit/org/apache/cassandra/db/CoordinatorLogOffsetsLifecycleTest.java +++ b/test/unit/org/apache/cassandra/db/CoordinatorLogOffsetsLifecycleTest.java @@ -180,8 +180,8 @@ public void mutationIdLifecycleTest() Memtable memtable = view.getCurrentMemtable(); ImmutableCoordinatorLogOffsets logOffsets = memtable.getFlushSet(null, null).coordinatorLogOffsets(); - Assertions.assertThat(logOffsets.size()).isEqualTo(1); - Assertions.assertThat(logOffsets.offsets(id2.logId()).contains(id2.offset())).isTrue(); + Assertions.assertThat(logOffsets.mutations().size()).isEqualTo(1); + Assertions.assertThat(logOffsets.mutations().offsets(id2.logId()).contains(id2.offset())).isTrue(); } // flush 1 @@ -195,8 +195,8 @@ public void mutationIdLifecycleTest() SSTableReader sstable = Iterables.getOnlyElement(view.liveSSTables()); ImmutableCoordinatorLogOffsets logOffsets = sstable.getCoordinatorLogOffsets(); - Assertions.assertThat(logOffsets.size()).isEqualTo(1); - Assertions.assertThat(logOffsets.offsets(id2.logId()).contains(id2.offset())).isTrue(); + Assertions.assertThat(logOffsets.mutations().size()).isEqualTo(1); + Assertions.assertThat(logOffsets.mutations().offsets(id2.logId()).contains(id2.offset())).isTrue(); // Single-participant, so mutations are immediately reconciled once applied Assertions.assertThat(sstable.isRepaired()).isTrue(); } @@ -214,8 +214,8 @@ public void mutationIdLifecycleTest() Memtable memtable = view.getCurrentMemtable(); ImmutableCoordinatorLogOffsets logOffsets = memtable.getFlushSet(null, null).coordinatorLogOffsets(); - Assertions.assertThat(logOffsets.size()).isEqualTo(1); - Assertions.assertThat(logOffsets.offsets(id4.logId()).contains(id4.offset())).isTrue(); + Assertions.assertThat(logOffsets.mutations().size()).isEqualTo(1); + Assertions.assertThat(logOffsets.mutations().offsets(id4.logId()).contains(id4.offset())).isTrue(); } // flush 2 @@ -231,14 +231,14 @@ public void mutationIdLifecycleTest() sstables.sort(Comparator.comparing(sst -> sst.descriptor.id.asBytes())); { ImmutableCoordinatorLogOffsets logOffsets = sstables.get(0).getCoordinatorLogOffsets(); - Assertions.assertThat(logOffsets.size()).isEqualTo(1); - Assertions.assertThat(logOffsets.offsets(id2.logId()).contains(id2.offset())).isTrue(); + Assertions.assertThat(logOffsets.mutations().size()).isEqualTo(1); + Assertions.assertThat(logOffsets.mutations().offsets(id2.logId()).contains(id2.offset())).isTrue(); } { ImmutableCoordinatorLogOffsets logOffsets = sstables.get(1).getCoordinatorLogOffsets(); - Assertions.assertThat(logOffsets.size()).isEqualTo(1); - Assertions.assertThat(logOffsets.offsets(id4.logId()).contains(id4.offset())).isTrue(); + Assertions.assertThat(logOffsets.mutations().size()).isEqualTo(1); + Assertions.assertThat(logOffsets.mutations().offsets(id4.logId()).contains(id4.offset())).isTrue(); } for (SSTableReader sstable : sstables) Assertions.assertThat(sstable.isRepaired()).isTrue(); @@ -254,8 +254,8 @@ public void mutationIdLifecycleTest() SSTableReader sstable = Iterables.getOnlyElement(view.liveSSTables()); ImmutableCoordinatorLogOffsets logOffsets = sstable.getCoordinatorLogOffsets(); - Assertions.assertThat(logOffsets.size()).isEqualTo(1); - Assertions.assertThat(logOffsets.offsets(id4.logId()).contains(id4.offset())).isTrue(); + Assertions.assertThat(logOffsets.mutations().size()).isEqualTo(1); + Assertions.assertThat(logOffsets.mutations().offsets(id4.logId()).contains(id4.offset())).isTrue(); Assertions.assertThat(sstable.isRepaired()).isTrue(); } } diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java index a6bcc3a2b281..20d94f98c54d 100644 --- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java +++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java @@ -62,7 +62,7 @@ public class TrackerTest { - private static final int COMPONENT_STATS_SIZE_BYTES = 4805; + private static final int COMPONENT_STATS_SIZE_BYTES = 4806; private static final class MockListener implements INotificationConsumer { diff --git a/test/unit/org/apache/cassandra/replication/ActivatedTransfersTest.java b/test/unit/org/apache/cassandra/replication/ActivatedTransfersTest.java new file mode 100644 index 000000000000..b309e92618ce --- /dev/null +++ b/test/unit/org/apache/cassandra/replication/ActivatedTransfersTest.java @@ -0,0 +1,280 @@ +/* + * 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.HashSet; +import java.util.List; +import java.util.Set; + +import org.junit.BeforeClass; +import org.junit.Test; + +import accord.utils.Gen; +import accord.utils.Gens; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.BufferDecoratedKey; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Bounds; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.utils.AccordGenerators; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.CassandraGenerators; +import org.assertj.core.api.Assertions; + +import static accord.utils.Property.qt; + +public class ActivatedTransfersTest +{ + @BeforeClass + public static void setUp() + { + DatabaseDescriptor.daemonInitialization(); + DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); + } + + private static Token tk(long token) + { + return new Murmur3Partitioner.LongToken(token); + } + + private static PartitionPosition pos(long token) + { + Token t = tk(token); + return new BufferDecoratedKey(t, ByteBufferUtil.bytes(token)); + } + + private static Bounds bounds(long left, long right) + { + return (Bounds) bounds(left, true, right, true); + } + + private static Range range(long left, long right) + { + return (Range) bounds(left, false, right, true); + } + + private static AbstractBounds bounds(long left, boolean leftInclusive, long right, boolean rightInclusive) + { + return AbstractBounds.bounds(pos(left), leftInclusive, pos(right), rightInclusive); + } + + private static ShortMutationId id(int logId, int offset) + { + return new ShortMutationId(logId, offset); + } + + private static Gen tokenGen() + { + return AccordGenerators.fromQT(CassandraGenerators.murmurToken()); + } + + private static Gen logIdGen() + { + return rs -> new CoordinatorLogId(rs.nextInt(), rs.nextInt()); + } + + private static Gen idGen() + { + return rs -> { + int offset = (short) rs.nextInt(Short.MIN_VALUE, Short.MAX_VALUE); + return new ShortMutationId(logIdGen().next(rs).asLong(), offset); + }; + } + + private static Gen activatedTransfersGen() + { + return rs -> { + List entries = Gens.lists(activatedTransferGen()).ofSizeBetween(0, 10).next(rs); + ActivatedTransfers transfers = new ActivatedTransfers(); + entries.forEach(entry -> transfers.add(entry.id, entry.bounds)); + return transfers; + }; + } + + private static Gen activatedTransferGen() + { + return rs -> { + ShortMutationId id = idGen().next(rs); + while (true) + { + Token left = tokenGen().next(rs); + Token right = tokenGen().next(rs); + + if (!AbstractBounds.strictlyWrapsAround(left, right)) + return new ActivatedTransfers.ActivatedTransfer(id, new Bounds<>(left, right)); + } + }; + } + + @Test + public void testSerdeRoundtrip() + { + qt() + .forAll(activatedTransfersGen()) + .check(transfers -> { + int version = MessagingService.current_version; + ActivatedTransfers deserialized; + try (DataOutputBuffer out = new DataOutputBuffer()) + { + ActivatedTransfers.serializer.serialize(transfers, out, version); + + try (DataInputBuffer in = new DataInputBuffer(out.buffer(), true)) + { + deserialized = ActivatedTransfers.serializer.deserialize(in, version); + } + } + + Assertions.assertThat(deserialized).isEqualTo(transfers); + }); + } + + @Test + public void testIntersectsSingle() + { + ActivatedTransfers transfers = new ActivatedTransfers(); + ShortMutationId id1 = id(1, 0); + transfers.add(id1, new Bounds<>(tk(400), tk(500))); // [400, 500] + + final long min = Murmur3Partitioner.instance.getMinimumToken().token; + + // Token + assertNoIntersection(transfers, tk(0)); + assertIntersects(transfers, id1, tk(400)); + assertIntersects(transfers, id1, tk(450)); + assertIntersects(transfers, id1, tk(500)); + assertNoIntersection(transfers, tk(550)); + + // Bounds [] + assertNoIntersection(transfers, bounds(100, 300)); // [100, 300] + assertIntersects(transfers, id1, bounds(100, 400L)); // [100, 400] - overlap at boundary + assertIntersects(transfers, id1, bounds(100, 450L)); // [100, 450] + assertIntersects(transfers, id1, bounds(400, 500L)); // [400, 500] + assertIntersects(transfers, id1, bounds(500, 600)); // [500, 600] - overlap at boundary + assertNoIntersection(transfers, bounds(600, 700)); // [600, 700] + assertIntersects(transfers, id1, bounds(0, 1000)); // [0, 1000] + assertIntersects(transfers, id1, bounds(400, 400)); // [400, 400] + assertIntersects(transfers, id1, new Bounds<>(tk(500).minKeyBound(), tk(min).maxKeyBound())); // [400, 400] + + // Range (] + assertNoIntersection(transfers, range(100, 300)); // (100, 300] + assertIntersects(transfers, id1, range(100, 400L)); // (100, 400] - overlap at boundary + assertIntersects(transfers, id1, range(100, 450L)); // (100, 450] + assertIntersects(transfers, id1, range(400, 500L)); // (400, 500] + assertNoIntersection(transfers, range(500, 600)); // (500, 600] + assertNoIntersection(transfers, range(600, 700)); // (600, 700] + assertIntersects(transfers, id1, range(0, 1000)); // (0, 1000] + assertIntersects(transfers, id1, range(0, min)); // (0, MIN] + assertIntersects(transfers, id1, range(450, min)); // (450, MIN] + assertIntersects(transfers, id1, range(0, 0)); // (0, 0] + assertNoIntersection(transfers, range(600, 300)); // (600, 300] + + // [) + assertNoIntersection(transfers, bounds(100, true, 300, false)); // [100, 300) + assertNoIntersection(transfers, bounds(100, true, 400, false)); // [100, 400) + assertIntersects(transfers, id1, bounds(100, true, 450, false)); // [100, 450) + assertIntersects(transfers, id1, bounds(400, true, 500, false)); // [400, 500) + assertIntersects(transfers, id1, bounds(500, true, 600, false)); // [500, 600) + assertNoIntersection(transfers, bounds(600, true, 700, false)); // [600, 700) + assertIntersects(transfers, id1, bounds(0, true, 1000, false)); // [0, 1000) + + // () + assertNoIntersection(transfers, bounds(100, false, 300, false)); // (100, 300) + assertNoIntersection(transfers, bounds(100, false, 400, false)); // (100, 400) + assertIntersects(transfers, id1, bounds(100, false, 450, false)); // (100, 450) + assertIntersects(transfers, id1, bounds(400, false, 500, false)); // (400, 500) + assertNoIntersection(transfers, bounds(500, false, 600, false)); // (500, 600) + assertNoIntersection(transfers, bounds(600, false, 700, false)); // (600, 700) + assertIntersects(transfers, id1, bounds(0, false, 1000, false)); // (0, 1000) + } + + @Test + public void testIntersectsMultiple() + { + ActivatedTransfers transfers = new ActivatedTransfers(); + ShortMutationId id1 = id(100, 1); + ShortMutationId id2 = id(100, 2); + ShortMutationId id3 = id(100, 3); + + transfers.add(id1, new Bounds<>(tk(100), tk(200))); + transfers.add(id2, new Bounds<>(tk(300), tk(400))); + transfers.add(id3, new Bounds<>(tk(500), tk(600))); + + Set ids1 = new HashSet<>(); + transfers.forEachIntersecting(new Bounds<>(pos(50), pos(150)), ids1::add); + Assertions.assertThat(ids1).containsExactly(id1); + + Set ids2 = new HashSet<>(); + transfers.forEachIntersecting(new Bounds<>(pos(50), pos(350)), ids2::add); + Assertions.assertThat(ids2).containsExactly(id1, id2); + + Set ids3 = new HashSet<>(); + transfers.forEachIntersecting(new Bounds<>(pos(0), pos(700)), ids3::add); + Assertions.assertThat(ids3).containsExactly(id1, id2, id3); + } + + @Test + public void testRemove() + { + ActivatedTransfers transfers = new ActivatedTransfers(); + ShortMutationId id1 = id(100, 1); + ShortMutationId id2 = id(100, 2); + ShortMutationId id3 = id(100, 3); + + transfers.add(id1, new Bounds<>(tk(100), tk(200))); + transfers.add(id2, new Bounds<>(tk(300), tk(400))); + transfers.add(id3, new Bounds<>(tk(500), tk(600))); + + transfers.removeOffset(1); + + assertNoIntersection(transfers, bounds(50, 100)); + } + + private void assertIntersects(ActivatedTransfers transfers, ShortMutationId expectedId, Token token) + { + Set ids = new HashSet<>(); + transfers.forEachIntersecting(token, ids::add); + Assertions.assertThat(ids).contains(expectedId); + } + + private void assertIntersects(ActivatedTransfers transfers, ShortMutationId expectedId, AbstractBounds range) + { + Set ids = new HashSet<>(); + transfers.forEachIntersecting(range, ids::add); + Assertions.assertThat(ids).contains(expectedId); + } + + private void assertNoIntersection(ActivatedTransfers transfers, Token token) + { + Set ids = new HashSet<>(); + transfers.forEachIntersecting(token, ids::add); + Assertions.assertThat(ids).isEmpty(); + } + + private void assertNoIntersection(ActivatedTransfers transfers, AbstractBounds range) + { + Set ids = new HashSet<>(); + transfers.forEachIntersecting(range, ids::add); + Assertions.assertThat(ids).isEmpty(); + } +} diff --git a/test/unit/org/apache/cassandra/replication/CoordinatorLogOffsetsTest.java b/test/unit/org/apache/cassandra/replication/CoordinatorLogOffsetsTest.java index 253d81526b34..f2d6387bfb12 100644 --- a/test/unit/org/apache/cassandra/replication/CoordinatorLogOffsetsTest.java +++ b/test/unit/org/apache/cassandra/replication/CoordinatorLogOffsetsTest.java @@ -22,6 +22,7 @@ import org.apache.cassandra.db.Mutation; import org.apache.cassandra.db.commitlog.CommitLog; import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.dht.Bounds; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; @@ -82,10 +83,13 @@ public class CoordinatorLogOffsetsTest private static final Gen MUTATION_ID_GEN = rs -> new MutationId(LOG_ID_GEN.next(rs), SEQUENCE_ID_GEN.next(rs)); private static final Gen COORDINATOR_LOG_OFFSETS_GEN = rs -> { - int numIds = rs.nextBiasedInt(0, 10, 1000); - ImmutableCoordinatorLogOffsets.Builder builder = new ImmutableCoordinatorLogOffsets.Builder(numIds); - for (int i = 0; i < numIds; i++) + int numMutations = rs.nextBiasedInt(0, 10, 1000); + ImmutableCoordinatorLogOffsets.Builder builder = new ImmutableCoordinatorLogOffsets.Builder(numMutations); + for (int i = 0; i < numMutations; i++) builder.add(MUTATION_ID_GEN.next(rs)); + int numTransfers = rs.nextBiasedInt(0, 1, 10); + for (int i = 0; i < numTransfers; i++) + builder.addTransfer(MUTATION_ID_GEN.next(rs), new Bounds<>(new Murmur3Partitioner.LongToken(1), new Murmur3Partitioner.LongToken(2))); return builder.build(); }; @@ -100,18 +104,18 @@ public void roundtripSerde() { qt() .forAll(COORDINATOR_LOG_OFFSETS_GEN) - .check(offsets -> { + .check(originals -> { try (DataOutputBuffer outputBuffer = DataOutputBuffer.scratchBuffer.get()) { - ImmutableCoordinatorLogOffsets.serializer.serialize(offsets, outputBuffer, MessagingService.current_version); - byte[] bytes = outputBuffer.toByteArray(); - try (DataInputBuffer inputBuffer = new DataInputBuffer(bytes)) + ImmutableCoordinatorLogOffsets.serializer.serialize(originals, outputBuffer, MessagingService.current_version); + try (DataInputBuffer inputBuffer = new DataInputBuffer(outputBuffer.buffer(), true)) { ImmutableCoordinatorLogOffsets deserialized = ImmutableCoordinatorLogOffsets.serializer.deserialize(inputBuffer, MessagingService.current_version); - Assertions.assertThat(Sets.newHashSet(deserialized.iterator())).isEqualTo(Sets.newHashSet(offsets.iterator())); - for (long logId : offsets) - Assertions.assertThat(deserialized.offsets(logId)).isEqualTo(offsets.offsets(logId)); - Assertions.assertThat(bytes.length).isEqualTo(ImmutableCoordinatorLogOffsets.serializer.serializedSize(offsets, MessagingService.current_version)); + CoordinatorLogOffsets.Mutations mutations = deserialized.mutations(); + Assertions.assertThat(Sets.newHashSet(mutations)).isEqualTo(Sets.newHashSet(originals.mutations().iterator())); + for (long logId : originals.mutations()) + Assertions.assertThat(mutations.offsets(logId)).isEqualTo(originals.mutations().offsets(logId)); + Assertions.assertThat(outputBuffer.getLength()).isEqualTo(ImmutableCoordinatorLogOffsets.serializer.serializedSize(originals, MessagingService.current_version)); } } }); @@ -132,10 +136,10 @@ public void monotonicAdd(Supplier ctor) MutableCoordinatorLogOffsets logOffsets = ctor.get(); for (MutationId id : ids) { - Offsets originalOffsets = logOffsets.offsets(id.logId()); + Offsets originalOffsets = logOffsets.mutations().offsets(id.logId()); boolean existed = originalOffsets.contains(id.offset()); logOffsets.add(id); - Offsets updatedOffsets = logOffsets.offsets(id.logId()); + Offsets updatedOffsets = logOffsets.mutations().offsets(id.logId()); if (existed) Assertions.assertThat(updatedOffsets).hasSameSizeAs(originalOffsets); Assertions.assertThat(updatedOffsets.contains(id.offset())).isTrue(); @@ -156,13 +160,13 @@ public void monotonicMerge(Supplier ctor) .forAll(COORDINATOR_LOG_OFFSETS_GEN, COORDINATOR_LOG_OFFSETS_GEN) .check((left, right) -> { MutableCoordinatorLogOffsets merged = ctor.get(); - merged.addAll(left); - merged.addAll(right); - for (Long logId : merged) + merged.addAll(left.mutations()); + merged.addAll(right.mutations()); + for (Long logId : merged.mutations()) { - Offsets leftOffsets = left.offsets(logId); - Offsets rightOffsets = right.offsets(logId); - Offsets mergedOffsets = Offsets.Immutable.copy(merged.offsets(logId)); + Offsets leftOffsets = left.mutations().offsets(logId); + Offsets rightOffsets = right.mutations().offsets(logId); + Offsets mergedOffsets = Offsets.Immutable.copy(merged.mutations().offsets(logId)); Assertions.assertThat(mergedOffsets).isEqualTo(Offsets.Immutable.union(leftOffsets, rightOffsets)); } }); @@ -189,9 +193,9 @@ public void builderEquivalentToMutable(Supplier ct } ImmutableCoordinatorLogOffsets fromBuilder = builder.build(); - Assertions.assertThat(fromBuilder).hasSize(logOffsets.size()); - for (Long logId : logOffsets) - Assertions.assertThat(fromBuilder.offsets(logId)).isEqualTo(Offsets.Immutable.copy(logOffsets.offsets(logId))); + Assertions.assertThat(fromBuilder.mutations()).hasSize(logOffsets.mutations().size()); + for (Long logId : logOffsets.mutations()) + Assertions.assertThat(fromBuilder.mutations().offsets(logId)).isEqualTo(Offsets.Immutable.copy(logOffsets.mutations().offsets(logId))); }); } @@ -232,9 +236,9 @@ public Args(List ids, int contentions) for (Future task : concurrentUpdates) task.get(); - Assertions.assertThatIterable(exclusive).hasSameSizeAs(concurrent); - for (Long logId : exclusive) - Assertions.assertThat(exclusive.offsets(logId)).isEqualTo(concurrent.offsets(logId)); + Assertions.assertThatIterable(exclusive.mutations()).hasSameSizeAs(concurrent.mutations()); + for (Long logId : exclusive.mutations()) + Assertions.assertThat(exclusive.mutations().offsets(logId)).isEqualTo(concurrent.mutations().offsets(logId)); }); } @@ -289,7 +293,7 @@ public void reconciledBounds() throws InterruptedException, ExecutionException { .add(mutation.id()) .build(); Range range = getShardRange(mutation); - List offsets = Collections.singletonList(logOffsets.offsets(mutation.id().logId())); + List offsets = Collections.singletonList(logOffsets.mutations().offsets(mutation.id().logId())); MutationTrackingService.instance.updateReplicatedOffsets(ks, range, offsets, true, addr2); MutationTrackingService.instance.updateReplicatedOffsets(ks, range, offsets, true, addr3); @@ -323,7 +327,7 @@ public void reconciledBounds() throws InterruptedException, ExecutionException { .build(); Range range = getShardRange(mutation); - List offsets = Collections.singletonList(logOffsets.offsets(mutation.id().logId())); + List offsets = Collections.singletonList(logOffsets.mutations().offsets(mutation.id().logId())); MutationTrackingService.instance.updateReplicatedOffsets(ks, range, offsets, true, addr2); MutationTrackingService.instance.updateReplicatedOffsets(ks, range, offsets, true, addr3); diff --git a/test/unit/org/apache/cassandra/replication/LocalTransfersTest.java b/test/unit/org/apache/cassandra/replication/LocalTransfersTest.java new file mode 100644 index 000000000000..3f8d79f27894 --- /dev/null +++ b/test/unit/org/apache/cassandra/replication/LocalTransfersTest.java @@ -0,0 +1,358 @@ +/* + * 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.Collections; + +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.dht.Murmur3Partitioner; +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.InetAddressAndPort; +import org.apache.cassandra.utils.TimeUUID; +import org.assertj.core.api.Assertions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.mock; + +public class LocalTransfersTest +{ + private LocalTransfers localTransfers; + + @BeforeClass + public static void setUpClass() + { + DatabaseDescriptor.daemonInitialization(); + DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); + } + + @Before + public void setUp() throws Exception + { + localTransfers = new LocalTransfers(); + } + + private CoordinatedTransfer coordinatedTransfer(TimeUUID transferId, ShortMutationId activationId) + { + return coordinatedTransfer(transferId, activationId, new Range<>(tk(0), tk(1000))); + } + + private CoordinatedTransfer coordinatedTransfer(TimeUUID transferId, ShortMutationId activationId, Range range) + { + MutationId mutationId = activationId != null + ? new MutationId(activationId.logId(), activationId.offset(), (int) System.currentTimeMillis()) + : null; + return new CoordinatedTransfer(transferId, range, mutationId); + } + + private PendingLocalTransfer pendingTransfer(TimeUUID planId) + { + SSTableReader mockSSTable = mock(SSTableReader.class); + Collection sstables = Collections.singletonList(mockSSTable); + + return new PendingLocalTransfer(planId, sstables); + } + + private static Token tk(long token) + { + return new Murmur3Partitioner.LongToken(token); + } + + private static ShortMutationId createActivationId(int logId, int offset) + { + return new ShortMutationId(logId, offset); + } + + @Test + public void testSaveCoordinatedTransfer() + { + TimeUUID transferId = TimeUUID.Generator.nextTimeUUID(); + ShortMutationId activationId = createActivationId(1, 100); + CoordinatedTransfer transfer = coordinatedTransfer(transferId, activationId); + + localTransfers.save(transfer); + localTransfers.activating(transfer); + + CoordinatedTransfer loaded = localTransfers.getActivatedTransfer(activationId); + Assertions.assertThat(loaded).isEqualTo(transfer); + + assertThatThrownBy(() -> localTransfers.save(transfer)) + .isInstanceOf(IllegalStateException.class); + } + + @Test + public void testActivatingTransfer() + { + TimeUUID transferId = TimeUUID.Generator.nextTimeUUID(); + ShortMutationId activationId = createActivationId(1, 100); + CoordinatedTransfer transfer = coordinatedTransfer(transferId, activationId); + + localTransfers.save(transfer); + localTransfers.activating(transfer); + + CoordinatedTransfer retrieved = localTransfers.getActivatedTransfer(activationId); + assertThat(retrieved).isEqualTo(transfer); + } + + @Test + public void testReceivedTransfer() + { + TimeUUID planId = TimeUUID.Generator.nextTimeUUID(); + PendingLocalTransfer transfer = pendingTransfer(planId); + localTransfers.received(transfer); + PendingLocalTransfer retrieved = localTransfers.getPendingTransfer(planId); + assertThat(retrieved).isEqualTo(transfer); + } + + @Test + public void testReceivedEmptyTransferThrows() + { + TimeUUID planId = TimeUUID.Generator.nextTimeUUID(); + assertThatThrownBy(() -> new PendingLocalTransfer(planId, Collections.emptyList())) + .isInstanceOf(IllegalStateException.class); + } + + @Test + public void testGetPendingTransferNotFound() + { + TimeUUID planId = TimeUUID.Generator.nextTimeUUID(); + PendingLocalTransfer retrieved = localTransfers.getPendingTransfer(planId); + assertThat(retrieved).isNull(); + } + + @Test + public void testGetActivatedTransferNotFound() + { + ShortMutationId activationId = createActivationId(1, 100); + CoordinatedTransfer retrieved = localTransfers.getActivatedTransfer(activationId); + assertThat(retrieved).isNull(); + } + + @Test + public void testPurgingTransferNotStarted() + { + TimeUUID transferId = TimeUUID.Generator.nextTimeUUID(); + ShortMutationId activationId = createActivationId(1, 100); + CoordinatedTransfer transfer = coordinatedTransfer(transferId, activationId); + + // All streams in INIT state - should NOT be purgeable (stream hasn't started yet) + CoordinatedTransfer.SingleTransferResult result = CoordinatedTransfer.SingleTransferResult.Init(); + result.state = CoordinatedTransfer.SingleTransferResult.State.INIT; + transfer.streams.put(mock(InetAddressAndPort.class), result); + + Assertions.assertThat(localTransfers.purger.test(transfer)).isFalse(); + } + + @Test + public void testPurgingTransferAllStreamsComplete() + { + TimeUUID transferId = TimeUUID.Generator.nextTimeUUID(); + ShortMutationId activationId = createActivationId(1, 100); + CoordinatedTransfer transfer = coordinatedTransfer(transferId, activationId); + + // All streams in STREAM_COMPLETE state - should NOT be purgeable (no failures) + CoordinatedTransfer.SingleTransferResult result1 = CoordinatedTransfer.SingleTransferResult.Init(); + result1.state = CoordinatedTransfer.SingleTransferResult.State.STREAM_COMPLETE; + CoordinatedTransfer.SingleTransferResult result2 = CoordinatedTransfer.SingleTransferResult.Init(); + result2.state = CoordinatedTransfer.SingleTransferResult.State.STREAM_COMPLETE; + + transfer.streams.put(mock(InetAddressAndPort.class), result1); + transfer.streams.put(mock(InetAddressAndPort.class), result2); + + Assertions.assertThat(localTransfers.purger.test(transfer)).isFalse(); + } + + @Test + public void testPurgingTransferAllActivationComplete() + { + TimeUUID transferId = TimeUUID.Generator.nextTimeUUID(); + ShortMutationId activationId = createActivationId(1, 100); + CoordinatedTransfer transfer = coordinatedTransfer(transferId, activationId); + + // All streams in ACTIVATE_COMPLETE state - should be purgeable (allComplete = true) + CoordinatedTransfer.SingleTransferResult result1 = CoordinatedTransfer.SingleTransferResult.Init(); + result1.completedActivation(); + CoordinatedTransfer.SingleTransferResult result2 = CoordinatedTransfer.SingleTransferResult.Init(); + result2.completedActivation(); + + transfer.streams.put(mock(InetAddressAndPort.class), result1); + transfer.streams.put(mock(InetAddressAndPort.class), result2); + + Assertions.assertThat(localTransfers.purger.test(transfer)).isTrue(); + } + + @Test + public void testPurgingTransferMixedCompleteAndNoop() + { + TimeUUID transferId = TimeUUID.Generator.nextTimeUUID(); + ShortMutationId activationId = createActivationId(1, 100); + CoordinatedTransfer transfer = coordinatedTransfer(transferId, activationId); + + // Mix of ACTIVATE_COMPLETE and STREAM_NOOP - should be purgeable (allComplete = true) + CoordinatedTransfer.SingleTransferResult result1 = CoordinatedTransfer.SingleTransferResult.Init(); + result1.completedActivation(); + CoordinatedTransfer.SingleTransferResult result2 = CoordinatedTransfer.SingleTransferResult.Init(); + result2.state = CoordinatedTransfer.SingleTransferResult.State.STREAM_NOOP; + + transfer.streams.put(mock(InetAddressAndPort.class), result1); + transfer.streams.put(mock(InetAddressAndPort.class), result2); + + Assertions.assertThat(localTransfers.purger.test(transfer)).isTrue(); + } + + @Test + public void testPurgingTransferActivationInProgress() + { + TimeUUID transferId = TimeUUID.Generator.nextTimeUUID(); + ShortMutationId activationId = createActivationId(1, 100); + CoordinatedTransfer transfer = coordinatedTransfer(transferId, activationId); + + // One stream in ACTIVATE_START - should NOT be purgeable + CoordinatedTransfer.SingleTransferResult result1 = CoordinatedTransfer.SingleTransferResult.Init(); + result1.sentActivation(); + CoordinatedTransfer.SingleTransferResult result2 = CoordinatedTransfer.SingleTransferResult.Init(); + result2.completedActivation(); + + transfer.streams.put(mock(InetAddressAndPort.class), result1); + transfer.streams.put(mock(InetAddressAndPort.class), result2); + + Assertions.assertThat(localTransfers.purger.test(transfer)).isFalse(); + } + + @Test + public void testPurgingTransferAllStreamsFailed() + { + TimeUUID transferId = TimeUUID.Generator.nextTimeUUID(); + ShortMutationId activationId = createActivationId(1, 100); + CoordinatedTransfer transfer = coordinatedTransfer(transferId, activationId); + + // All streams in STREAM_FAILED state - should be purgeable (noneActivated = true) + CoordinatedTransfer.SingleTransferResult result1 = CoordinatedTransfer.SingleTransferResult.Init(); + result1.streamFailed(TimeUUID.Generator.nextTimeUUID()); + CoordinatedTransfer.SingleTransferResult result2 = CoordinatedTransfer.SingleTransferResult.Init(); + result2.streamFailed(TimeUUID.Generator.nextTimeUUID()); + + transfer.streams.put(mock(InetAddressAndPort.class), result1); + transfer.streams.put(mock(InetAddressAndPort.class), result2); + + Assertions.assertThat(localTransfers.purger.test(transfer)).isTrue(); + } + + @Test + public void testPurgingTransferMixedInitAndFailed() + { + TimeUUID transferId = TimeUUID.Generator.nextTimeUUID(); + ShortMutationId activationId = createActivationId(1, 100); + CoordinatedTransfer transfer = coordinatedTransfer(transferId, activationId); + + // Mix of INIT and STREAM_FAILED - should be purgeable (has failure, none activated) + CoordinatedTransfer.SingleTransferResult result1 = CoordinatedTransfer.SingleTransferResult.Init(); + result1.state = CoordinatedTransfer.SingleTransferResult.State.INIT; + CoordinatedTransfer.SingleTransferResult result2 = CoordinatedTransfer.SingleTransferResult.Init(); + result2.streamFailed(TimeUUID.Generator.nextTimeUUID()); + + transfer.streams.put(mock(InetAddressAndPort.class), result1); + transfer.streams.put(mock(InetAddressAndPort.class), result2); + + Assertions.assertThat(localTransfers.purger.test(transfer)).isTrue(); + } + + @Test + public void testPurgingTransferMixedCompleteAndFailed() + { + TimeUUID transferId = TimeUUID.Generator.nextTimeUUID(); + ShortMutationId activationId = createActivationId(1, 100); + CoordinatedTransfer transfer = coordinatedTransfer(transferId, activationId); + + // Mix of STREAM_COMPLETE and STREAM_FAILED - should be purgeable (has failure, none activated) + CoordinatedTransfer.SingleTransferResult result1 = CoordinatedTransfer.SingleTransferResult.Init(); + result1.state = CoordinatedTransfer.SingleTransferResult.State.STREAM_COMPLETE; + CoordinatedTransfer.SingleTransferResult result2 = CoordinatedTransfer.SingleTransferResult.Init(); + result2.streamFailed(TimeUUID.Generator.nextTimeUUID()); + + transfer.streams.put(mock(InetAddressAndPort.class), result1); + transfer.streams.put(mock(InetAddressAndPort.class), result2); + + Assertions.assertThat(localTransfers.purger.test(transfer)).isTrue(); + } + + @Test + public void testPurgingTransferMixedCompleteAndActivationStart() + { + TimeUUID transferId = TimeUUID.Generator.nextTimeUUID(); + ShortMutationId activationId = createActivationId(1, 100); + CoordinatedTransfer transfer = coordinatedTransfer(transferId, activationId); + + // Mix of STREAM_COMPLETE and ACTIVATE_START - should NOT be purgeable + // (noneActivated = false because of ACTIVATE_START, allComplete = false) + CoordinatedTransfer.SingleTransferResult result1 = CoordinatedTransfer.SingleTransferResult.Init(); + result1.state = CoordinatedTransfer.SingleTransferResult.State.STREAM_COMPLETE; + CoordinatedTransfer.SingleTransferResult result2 = CoordinatedTransfer.SingleTransferResult.Init(); + result2.sentActivation(); + + transfer.streams.put(mock(InetAddressAndPort.class), result1); + transfer.streams.put(mock(InetAddressAndPort.class), result2); + + Assertions.assertThat(localTransfers.purger.test(transfer)).isFalse(); + } + + @Test + public void testPurgingTransferWithNullActivationId() + { + TimeUUID transferId = TimeUUID.Generator.nextTimeUUID(); + CoordinatedTransfer transfer = coordinatedTransfer(transferId, null); + + // All streams complete but activationId is null - should NOT be purgeable + CoordinatedTransfer.SingleTransferResult result1 = CoordinatedTransfer.SingleTransferResult.Init(); + result1.completedActivation(); + CoordinatedTransfer.SingleTransferResult result2 = CoordinatedTransfer.SingleTransferResult.Init(); + result2.completedActivation(); + + transfer.streams.put(mock(InetAddressAndPort.class), result1); + transfer.streams.put(mock(InetAddressAndPort.class), result2); + + // allComplete = true, but activationId is null, so should not purge + Assertions.assertThat(localTransfers.purger.test(transfer)).isFalse(); + } + + @Test + public void testPurgingTransferNoopOnly() + { + TimeUUID transferId = TimeUUID.Generator.nextTimeUUID(); + ShortMutationId activationId = createActivationId(1, 100); + CoordinatedTransfer transfer = coordinatedTransfer(transferId, activationId); + + // All streams in STREAM_NOOP - should be purgeable (both noneActivated and allComplete are true) + CoordinatedTransfer.SingleTransferResult result1 = CoordinatedTransfer.SingleTransferResult.Init(); + result1.state = CoordinatedTransfer.SingleTransferResult.State.STREAM_NOOP; + CoordinatedTransfer.SingleTransferResult result2 = CoordinatedTransfer.SingleTransferResult.Init(); + result2.state = CoordinatedTransfer.SingleTransferResult.State.STREAM_NOOP; + + transfer.streams.put(mock(InetAddressAndPort.class), result1); + transfer.streams.put(mock(InetAddressAndPort.class), result2); + + Assertions.assertThat(localTransfers.purger.test(transfer)).isTrue(); + } +} diff --git a/test/unit/org/apache/cassandra/replication/TransferActivationSerializationTest.java b/test/unit/org/apache/cassandra/replication/TransferActivationSerializationTest.java new file mode 100644 index 000000000000..00c2e07d748a --- /dev/null +++ b/test/unit/org/apache/cassandra/replication/TransferActivationSerializationTest.java @@ -0,0 +1,77 @@ +/* + * 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 org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.Serializers; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.utils.TimeUUID; + +import static org.apache.cassandra.utils.TimeUUID.Generator.nextTimeUUID; + +public class TransferActivationSerializationTest +{ + private static final int VERSION = MessagingService.current_version; + + @BeforeClass + public static void setUpClass() + { + DatabaseDescriptor.daemonInitialization(); + } + + @Test + public void testRoundtripPreparePhase() throws IOException + { + TimeUUID transferId = nextTimeUUID(); + TimeUUID planId = nextTimeUUID(); + MutationId activationId = new MutationId(1L, 100, 200); + NodeId coordinatorId = new NodeId(1); + TransferActivation.Phase phase = TransferActivation.Phase.PREPARE; + + TransferActivation activation = new TransferActivation(transferId, planId, activationId, coordinatorId, phase); + + try (DataOutputBuffer output = new DataOutputBuffer()) + { + Serializers.testSerde(output, TransferActivation.serializer, activation, VERSION); + } + } + + @Test + public void testRoundtripCommitPhase() throws IOException + { + TimeUUID transferId = nextTimeUUID(); + TimeUUID planId = nextTimeUUID(); + MutationId activationId = new MutationId(2L, 300, 400); + NodeId coordinatorId = new NodeId(2); + TransferActivation.Phase phase = TransferActivation.Phase.COMMIT; + + TransferActivation activation = new TransferActivation(transferId, planId, activationId, coordinatorId, phase); + + try (DataOutputBuffer output = new DataOutputBuffer()) + { + Serializers.testSerde(output, TransferActivation.serializer, activation, VERSION); + } + } +}