diff --git a/src/java/org/apache/cassandra/db/Mutation.java b/src/java/org/apache/cassandra/db/Mutation.java index f8388bfe89c0..f1662cd1605a 100644 --- a/src/java/org/apache/cassandra/db/Mutation.java +++ b/src/java/org/apache/cassandra/db/Mutation.java @@ -44,6 +44,7 @@ import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.rows.DeserializationHelper; import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.io.util.DataOutputPlus; @@ -56,6 +57,7 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.AbstractWriteResponseHandler; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.concurrent.Future; import static com.google.common.base.Preconditions.checkState; @@ -656,6 +658,24 @@ public Mutation deserialize(DataInputPlus in, int version, DeserializationHelper } } + /** + * Partially deserializes mutation data to get the key and table metadata. This does not skip the remainder of + * the mutation, hence the in argument being DataInputBuffer not DataInputPlus + */ + public Pair deserializeKeyAndTableMetadata(DataInputBuffer in, int version, DeserializationHelper.Flag flag) throws IOException + { + if (version >= VERSION_51) + in.skipBytes(1); // potentialTxnConflicts + + if (version >= VERSION_52) + MutationId.serializer.skip(in, version); + + int size = in.readUnsignedVInt32(); + assert size > 0; + + return PartitionUpdate.serializer.deserializeMetadataAndKey(in, version, flag); + } + public Mutation deserialize(DataInputPlus in, int version) throws IOException { return deserialize(in, version, DeserializationHelper.Flag.FROM_REMOTE); diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java index 3da60b3eb5da..3b19b82bf875 100644 --- a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java +++ b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java @@ -81,6 +81,7 @@ import org.apache.cassandra.service.accord.serializers.TableMetadatas; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.btree.BTree; import org.apache.cassandra.utils.btree.UpdateFunction; import org.apache.cassandra.utils.vint.VIntCoding; @@ -788,16 +789,11 @@ public void serializeWithoutKey(PartitionUpdate update, TableMetadatas tables, D } } - public PartitionUpdate deserialize(DataInputPlus in, int version, DeserializationHelper.Flag flag) throws IOException + private TableMetadata getTableMetadata(TableId tableId, Epoch remoteVersion) throws IOException { - TableId tableId = TableId.deserialize(in); - Epoch remoteVersion = null; - if (version >= MessagingService.VERSION_51) - remoteVersion = Epoch.serializer.deserialize(in); - TableMetadata tableMetadata; try { - tableMetadata = Schema.instance.getExistingTableMetadata(tableId); + return Schema.instance.getExistingTableMetadata(tableId); } catch (UnknownTableException e) { @@ -810,10 +806,34 @@ public PartitionUpdate deserialize(DataInputPlus in, int version, Deserializatio } throw e; } + } + + public PartitionUpdate deserialize(DataInputPlus in, int version, DeserializationHelper.Flag flag) throws IOException + { + TableId tableId = TableId.deserialize(in); + Epoch remoteVersion = null; + if (version >= MessagingService.VERSION_51) + remoteVersion = Epoch.serializer.deserialize(in); + TableMetadata tableMetadata = getTableMetadata(tableId, remoteVersion); UnfilteredRowIteratorSerializer.Header header = UnfilteredRowIteratorSerializer.serializer.deserializeHeader(tableMetadata, null, in, version, flag); return deserialize(header, remoteVersion, tableMetadata, in, version, flag); } + /** + * Partially deserializes a serialized partition update to get the key and table metadata. This does not skip the remainder of + * the mutation, hence the in argument being DataInputBuffer not DataInputPlus + */ + public Pair deserializeMetadataAndKey(DataInputBuffer in, int version, DeserializationHelper.Flag flag) throws IOException + { + TableId tableId = TableId.deserialize(in); + Epoch remoteVersion = null; + if (version >= MessagingService.VERSION_51) + remoteVersion = Epoch.serializer.deserialize(in); + TableMetadata tableMetadata = getTableMetadata(tableId, remoteVersion); + UnfilteredRowIteratorSerializer.Header header = UnfilteredRowIteratorSerializer.serializer.deserializeHeader(tableMetadata, null, in, version, flag); + return Pair.create(header.key, tableMetadata); + } + public PartitionUpdate deserialize(PartitionKey key, TableMetadatas tables, DataInputPlus in, int version, DeserializationHelper.Flag flag) throws IOException { TableMetadata tableMetadata = tables.deserialize(in); diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java index 505bd6b9287e..2fc802b03614 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java @@ -21,8 +21,10 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Set; +import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; @@ -42,6 +44,9 @@ import org.apache.cassandra.io.sstable.metadata.StatsMetadata; import org.apache.cassandra.locator.RangesAtEndpoint; import org.apache.cassandra.locator.Replica; +import org.apache.cassandra.replication.ImmutableCoordinatorLogOffsets; +import org.apache.cassandra.replication.Offsets; +import org.apache.cassandra.replication.ReconciledKeyspaceOffsets; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.streaming.IncomingStream; import org.apache.cassandra.streaming.OutgoingStream; @@ -86,8 +91,34 @@ public StreamReceiver createStreamReceiver(StreamSession session, List getSSTablePredicateForKeyspaceRanges(ReconciledKeyspaceOffsets reconciledKeyspaceOffsets) + { + if (reconciledKeyspaceOffsets == null) + return sstable -> true; + + return sstable -> { + if (sstable.isRepaired()) + return false; + + ImmutableCoordinatorLogOffsets sstableOffsets = sstable.getSSTableMetadata().coordinatorLogOffsets; + + // if it's not repaired and there are no offsets, it was probably written before the table was using + // mutation tracking and therefore should be considered unreconciled + if (sstableOffsets.isEmpty()) + return true; + + for (Map.Entry entry : sstableOffsets.entries()) + { + if (!reconciledKeyspaceOffsets.isFullyReconciled(entry.getKey(), entry.getValue())) + return true; + } + + return false; + }; + } + @Override - public Collection createOutgoingStreams(StreamSession session, RangesAtEndpoint replicas, TimeUUID pendingRepair, PreviewKind previewKind) + public Collection createOutgoingStreams(StreamSession session, RangesAtEndpoint replicas, TimeUUID pendingRepair, PreviewKind previewKind, ReconciledKeyspaceOffsets reconciledKeyspaceOffsets) { Refs refs = new Refs<>(); try @@ -99,7 +130,14 @@ public Collection createOutgoingStreams(StreamSession session, R Set sstables = Sets.newHashSet(); SSTableIntervalTree intervalTree = buildSSTableIntervalTree(ImmutableList.copyOf(view.select(SSTableSet.CANONICAL))); Predicate predicate; - if (previewKind.isPreview()) + if (reconciledKeyspaceOffsets != null) + { + // TODO: relax these restrictions as repair support is add + Preconditions.checkArgument(previewKind == PreviewKind.NONE); + Preconditions.checkArgument(pendingRepair == ActiveRepairService.NO_PENDING_REPAIR); + predicate = getSSTablePredicateForKeyspaceRanges(reconciledKeyspaceOffsets); + } + else if (previewKind.isPreview()) { predicate = previewKind.predicate(); } diff --git a/src/java/org/apache/cassandra/journal/ActiveSegment.java b/src/java/org/apache/cassandra/journal/ActiveSegment.java index 6000e2c9d65f..8a8f1d1bb357 100644 --- a/src/java/org/apache/cassandra/journal/ActiveSegment.java +++ b/src/java/org/apache/cassandra/journal/ActiveSegment.java @@ -153,6 +153,23 @@ boolean read(int offset, int size, EntrySerializer.EntryHolder into) return true; } + @Override + public void readAll(RecordConsumer consumer) + { + // Ensure all in-progress writes are completed and flushed to the buffer + updateWrittenTo(); + + // Use the forEach method to iterate through all key-offset pairs + index.forEach((key, offset, size) -> { + // Read the record at this offset + EntrySerializer.EntryHolder holder = new EntrySerializer.EntryHolder<>(); + if (read(offset, size, holder)) + { + consumer.accept(descriptor.timestamp, offset, key, holder.value, descriptor.userVersion); + } + }); + } + /** * Stop writing to this file, flush and close it. Does nothing if the file is already closed. */ diff --git a/src/java/org/apache/cassandra/journal/InMemoryIndex.java b/src/java/org/apache/cassandra/journal/InMemoryIndex.java index 49fe4d136714..c288e434289a 100644 --- a/src/java/org/apache/cassandra/journal/InMemoryIndex.java +++ b/src/java/org/apache/cassandra/journal/InMemoryIndex.java @@ -155,6 +155,26 @@ static InMemoryIndex rebuild(Descriptor descriptor, KeySupport keySupp return index; } + public interface EntryConsumer + { + void accept(K key, int offset, int size); + } + + /** + * Iterate through all key-offset pairs in the index. + * For each key, iterates through all its offsets and calls the consumer + * with the key and individual offset/size values. + */ + public void forEach(EntryConsumer consumer) + { + index.forEach((key, offsetsAndSizes) -> { + for (long offsetAndSize : offsetsAndSizes) + { + consumer.accept(key, Index.readOffset(offsetAndSize), Index.readSize(offsetAndSize)); + } + }); + } + @Override public void close() { diff --git a/src/java/org/apache/cassandra/journal/Journal.java b/src/java/org/apache/cassandra/journal/Journal.java index eaab157ea9c5..6f4e74260d1c 100644 --- a/src/java/org/apache/cassandra/journal/Journal.java +++ b/src/java/org/apache/cassandra/journal/Journal.java @@ -33,6 +33,7 @@ import java.util.function.*; import java.util.zip.CRC32; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import com.google.common.annotations.VisibleForTesting; @@ -174,6 +175,36 @@ public void run() } } + public static class Snapshot implements AutoCloseable + { + private final Segments.ReferencedSegments segments; + + private Snapshot(@Nonnull ReferencedSegments segments) + { + this.segments = segments; + } + + /** + * Read all records from all segments in the journal. + * This method provides an atomic snapshot of all segments and iterates through each one. + * + * @param consumer function to consume each record found + */ + public void readAll(RecordConsumer consumer) + { + for (Segment segment : segments.allSorted(false)) + { + segment.readAll(consumer); + } + } + + @Override + public void close() + { + segments.close(); + } + } + public Journal(String name, File directory, Params params, @@ -370,6 +401,28 @@ public List readAll(K id) return res; } + /** + * Read all records from all segments in the journal. + * This method provides an atomic snapshot of all segments and iterates through each one. + * + * @param consumer function to consume each record found + */ + public void readAll(RecordConsumer consumer) + { + try (OpOrder.Group group = readOrder.start()) + { + for (Segment segment : segments.get().allSorted(false)) + { + segment.readAll(consumer); + } + } + } + + public Snapshot snapshot(Predicate> predicate) + { + return new Snapshot<>(selectAndReference(predicate)); + } + /** * Looks up a record by the provided id, if the value satisfies the provided condition. *

@@ -566,6 +619,12 @@ private ActiveSegment.Allocation allocate(int entrySize) return alloc; } + @VisibleForTesting + public void advanceSegment() + { + advanceSegment(currentSegment); + } + /* * Segment allocation logic. */ @@ -742,7 +801,7 @@ private void closeAllSegments() } @SuppressWarnings("unused") - ReferencedSegments selectAndReference(Predicate> selector) + public ReferencedSegments selectAndReference(Predicate> selector) { while (true) { diff --git a/src/java/org/apache/cassandra/journal/Segment.java b/src/java/org/apache/cassandra/journal/Segment.java index a80bbb4f4cd7..8e054e6314ba 100644 --- a/src/java/org/apache/cassandra/journal/Segment.java +++ b/src/java/org/apache/cassandra/journal/Segment.java @@ -133,6 +133,12 @@ void readAll(K id, EntrySerializer.EntryHolder into, RecordConsumer onEntr } } + /** + * Read all records from this segment using the RecordConsumer interface. + * This provides access to deserialized keys and raw value ByteBuffers for streaming scenarios. + */ + public abstract void readAll(RecordConsumer consumer); + @Override public int compareTo(Segment that) { diff --git a/src/java/org/apache/cassandra/journal/StaticSegment.java b/src/java/org/apache/cassandra/journal/StaticSegment.java index 35c987c8a480..355f6c230dbf 100644 --- a/src/java/org/apache/cassandra/journal/StaticSegment.java +++ b/src/java/org/apache/cassandra/journal/StaticSegment.java @@ -302,6 +302,18 @@ public void forEachRecord(RecordConsumer consumer) } } + @Override + public void readAll(RecordConsumer consumer) + { + try (SequentialReader reader = sequentialReader(descriptor, keySupport, fsyncLimit)) + { + while (reader.advance()) + { + consumer.accept(descriptor.timestamp, reader.offset(), reader.key(), reader.record(), descriptor.userVersion); + } + } + } + /* * Sequential and in-key order reading (replay and components rebuild) */ diff --git a/src/java/org/apache/cassandra/replication/BroadcastLogOffsets.java b/src/java/org/apache/cassandra/replication/BroadcastLogOffsets.java index ce8fe512c9d6..ce1d4b672fa2 100644 --- a/src/java/org/apache/cassandra/replication/BroadcastLogOffsets.java +++ b/src/java/org/apache/cassandra/replication/BroadcastLogOffsets.java @@ -59,7 +59,16 @@ boolean isEmpty() @Override public String toString() { - return "ShardReplicatedOffsets{" + keyspace + ", " + range + ", " + replicatedOffsets + ", " + durable + '}'; + StringBuilder sb = new StringBuilder('['); + boolean isFirst = true; + for (Offsets.Immutable logOffsets : replicatedOffsets) + { + if (!isFirst) sb.append(", "); + sb.append('(').append(logOffsets.logId()).append(" -> ").append(logOffsets).append(')'); + isFirst = false; + } + sb.append(']'); + return "ShardReplicatedOffsets{" + keyspace + ", " + range + ", " + sb + ", " + durable + '}'; } public static final IVerbHandler verbHandler = message -> { diff --git a/src/java/org/apache/cassandra/replication/CoordinatorLog.java b/src/java/org/apache/cassandra/replication/CoordinatorLog.java index 312c35523795..cebfcb3ee23e 100644 --- a/src/java/org/apache/cassandra/replication/CoordinatorLog.java +++ b/src/java/org/apache/cassandra/replication/CoordinatorLog.java @@ -72,7 +72,7 @@ public abstract class CoordinatorLog protected final Offsets.Mutable reconciledOffsets; protected final Offsets.Mutable reconciledPersistedOffsets; - protected final ReadWriteLock lock; + protected final ReadWriteLock lock = new ReentrantReadWriteLock(); abstract void receivedWriteResponse(ShortMutationId mutationId, int fromNodeId); @@ -95,7 +95,6 @@ public abstract class CoordinatorLog this.reconciledOffsets = witnessedOffsets.intersection(); this.persistedOffsets = persistedOffsets; this.reconciledPersistedOffsets = persistedOffsets.intersection(); - this.lock = new ReentrantReadWriteLock(); } CoordinatorLog(String keyspace, Range range, int localNodeId, CoordinatorLogId logId, Participants participants) @@ -117,11 +116,85 @@ static CoordinatorLog recreate( : new CoordinatorLogReplica(keyspace, range, localNodeId, id, participants, witnessedOffsets, persistedOffsets, unreconciledMutations); } + abstract CoordinatorLog withUpdatedParticipants(Participants newParticipants, Node2OffsetsMap witnessedOffsets, Node2OffsetsMap persistedOffsets, UnreconciledMutations unreconciledMutations); + + CoordinatorLog withParticipants(Participants newParticipants) + { + if (participants.equals(newParticipants)) + return this; + + lock.readLock().lock(); + try + { + Node2OffsetsMap newWitnessedOffsets = new Node2OffsetsMap(); + Node2OffsetsMap newPersistedOffsets = new Node2OffsetsMap(); + Offsets passivelyReconciled = null; + for (int newIndex = 0; newIndex < newParticipants.size(); newIndex++) + { + int participantId = newParticipants.get(newIndex); + + Offsets.Mutable offsets; + if (participants.contains(participantId)) + { + offsets = witnessedOffsets.get(participantId); + } + else + { + offsets = new Offsets.Mutable(logId); + + // the new node doesn't actually have these reconciled offsets yet, but they will receive them + // as part of the topology change. We preemptively mark them as reconciled here to prevent so + // we don't stream journal entries that the new node will receive in sstables and to prevent + // 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; + 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)); + } + else + { + newUnreconciled = unreconciledMutations; + } + + if (logger.isTraceEnabled()) + logger.trace("Updating coordinator log {} participants: {} -> {}. Passively reconciled: {}", + logId, participants, newParticipants, passivelyReconciled); + + return withUpdatedParticipants(newParticipants, newWitnessedOffsets, newPersistedOffsets, newUnreconciled); + } + finally + { + lock.readLock().unlock(); + } + } + void updateReplicatedOffsets(Offsets offsets, boolean persisted, int onNodeId) { lock.writeLock().lock(); try { + // there may have been a topology change we're not yet aware of + if (!participants.contains(onNodeId)) + return; + if (persisted) updatePersistedReplicatedOffsets(offsets, onNodeId); else @@ -155,6 +228,20 @@ private void updatePersistedReplicatedOffsets(Offsets offsets, int onNodeId) reconciledPersistedOffsets.addAll(persistedOffsets.intersection()); } + public void recordFullyReconciledOffsets(Offsets.Immutable reconciled) + { + lock.writeLock().lock(); + try { + for (int i = 0; i < participants.size(); ++i) + { + int participant = participants.get(i); + updateWitnessedReplicatedOffsets(reconciled, participant); + } + } finally { + lock.writeLock().unlock(); + } + } + @Nullable Offsets.Immutable collectReplicatedOffsets(boolean persisted) { @@ -172,6 +259,19 @@ Offsets.Immutable collectReplicatedOffsets(boolean persisted) } } + Offsets.Immutable collectReconciledOffsets() + { + lock.readLock().lock(); + try + { + return Offsets.Immutable.copy(reconciledOffsets); + } + finally + { + lock.readLock().unlock(); + } + } + boolean startWriting(Mutation mutation) { lock.writeLock().lock(); @@ -342,6 +442,16 @@ static class CoordinatorLogPrimary extends CoordinatorLog super(keyspace, range, localNodeId, logId, participants); } + + + @Override + CoordinatorLog withUpdatedParticipants(Participants newParticipants, Node2OffsetsMap witnessedOffsets, Node2OffsetsMap persistedOffsets, UnreconciledMutations unreconciledMutations) + { + CoordinatorLogPrimary next = new CoordinatorLogPrimary(keyspace, range, localNodeId, logId, newParticipants, witnessedOffsets, persistedOffsets, unreconciledMutations); + next.sequenceId.set(sequenceId.get()); + return next; + } + @Override void receivedWriteResponse(ShortMutationId mutationId, int fromNodeId) { @@ -416,6 +526,12 @@ static class CoordinatorLogReplica extends CoordinatorLog super(keyspace, range, localNodeId, logId, participants); } + @Override + CoordinatorLog withUpdatedParticipants(Participants newParticipants, Node2OffsetsMap witnessedOffsets, Node2OffsetsMap persistedOffsets, UnreconciledMutations unreconciledMutations) + { + return new CoordinatorLogReplica(keyspace, range, localNodeId, logId, newParticipants, witnessedOffsets, persistedOffsets, unreconciledMutations); + } + @Override void receivedWriteResponse(ShortMutationId mutationId, int fromNodeId) { diff --git a/src/java/org/apache/cassandra/replication/CoordinatorLogId.java b/src/java/org/apache/cassandra/replication/CoordinatorLogId.java index 1beac3fb794d..101d95db9e4e 100644 --- a/src/java/org/apache/cassandra/replication/CoordinatorLogId.java +++ b/src/java/org/apache/cassandra/replication/CoordinatorLogId.java @@ -89,6 +89,11 @@ public static CoordinatorLogId none() return NONE; } + public static CoordinatorLogId fromLong(long logId) + { + return new CoordinatorLogId(logId); + } + static boolean isNone(int hostId, int hostLogId) { return hostId == NONE.hostId && hostLogId == NONE.hostLogId; @@ -121,7 +126,7 @@ public int hashCode() public static final Comparator comparator = (l, r) -> Long.compareUnsigned(l.asLong(), r.asLong()); - static final class Serializer implements IVersionedSerializer + public static final class Serializer implements IVersionedSerializer { @Override public void serialize(CoordinatorLogId logId, DataOutputPlus out, int version) throws IOException @@ -158,5 +163,5 @@ public long serializedSize(long logId, int version) } } - static final Serializer serializer = new Serializer(); + public static final Serializer serializer = new Serializer(); } diff --git a/src/java/org/apache/cassandra/replication/ImmutableCoordinatorLogOffsets.java b/src/java/org/apache/cassandra/replication/ImmutableCoordinatorLogOffsets.java index 865abbda088c..7d946ec9f737 100644 --- a/src/java/org/apache/cassandra/replication/ImmutableCoordinatorLogOffsets.java +++ b/src/java/org/apache/cassandra/replication/ImmutableCoordinatorLogOffsets.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.Map; import java.util.Objects; +import java.util.function.BiConsumer; import javax.annotation.concurrent.NotThreadSafe; import com.google.common.collect.Iterators; @@ -52,12 +53,23 @@ public int size() return ids.size(); } + public boolean isEmpty() + { + return size() == 0; + } + @Override public Iterator iterator() { return Iterators.unmodifiableIterator(ids.keySet().iterator()); } + public Iterable> entries() + { + return ids.entrySet(); + } + + @Override public boolean equals(Object o) { @@ -82,6 +94,11 @@ public ImmutableCoordinatorLogOffsets(Builder builder) ids.put(entry.getKey(), entry.getValue().build()); } + public void forEach(BiConsumer consumer) + { + ids.forEach((logId, offsets) -> consumer.accept(new CoordinatorLogId(logId), offsets)); + } + @NotThreadSafe public static class Builder { diff --git a/src/java/org/apache/cassandra/replication/MutationId.java b/src/java/org/apache/cassandra/replication/MutationId.java index 5faf4db7d5cd..d497df9b0316 100644 --- a/src/java/org/apache/cassandra/replication/MutationId.java +++ b/src/java/org/apache/cassandra/replication/MutationId.java @@ -107,7 +107,7 @@ public String toString() */ public static final Comparator comparator = ShortMutationId.comparator::compare; - public static final IVersionedSerializer serializer = new IVersionedSerializer<>() + public static class Serializer implements IVersionedSerializer { @Override public void serialize(MutationId id, DataOutputPlus out, int version) throws IOException @@ -131,5 +131,13 @@ public long serializedSize(MutationId id, int version) { return TypeSizes.sizeof(id.logId()) + TypeSizes.sizeof(id.sequenceId()); } + + public void skip(DataInputPlus in, int version) throws IOException + { + in.readLong(); + in.readLong(); + } }; + + public static Serializer serializer = new Serializer(); } diff --git a/src/java/org/apache/cassandra/replication/MutationJournal.java b/src/java/org/apache/cassandra/replication/MutationJournal.java index aeccc3f052a7..b3cc38d0a586 100644 --- a/src/java/org/apache/cassandra/replication/MutationJournal.java +++ b/src/java/org/apache/cassandra/replication/MutationJournal.java @@ -50,6 +50,27 @@ public class MutationJournal private final Journal journal; + public static class Snapshot implements AutoCloseable + { + private final Journal.Snapshot wrapped; + + public Snapshot(Journal.Snapshot wrapped) + { + this.wrapped = wrapped; + } + + public void readAll(RecordConsumer consumer) + { + wrapped.readAll(consumer); + } + + @Override + public void close() + { + wrapped.close(); + } + } + private MutationJournal() { this(new File(DatabaseDescriptor.getCommitLogLocation()), new JournalParams()); @@ -87,6 +108,17 @@ boolean read(ShortMutationId id, RecordConsumer consumer) return journal.readLast(id, consumer); } + public MutationJournal.Snapshot snapshot() + { + return new MutationJournal.Snapshot(journal.snapshot(s -> true)); + } + + @VisibleForTesting + public void advanceSegment() + { + journal.advanceSegment(); + } + /** * @return record pointer of the last mutation with the provided id, or null if not found */ @@ -116,6 +148,11 @@ public void readAll(Iterable ids, Collection into) } } + public void readAll(RecordConsumer consumer) + { + journal.readAll(consumer); + } + static class JournalParams implements Params { @Override diff --git a/src/java/org/apache/cassandra/replication/MutationTrackingService.java b/src/java/org/apache/cassandra/replication/MutationTrackingService.java index f780a35bfa9a..79f6273c1b54 100644 --- a/src/java/org/apache/cassandra/replication/MutationTrackingService.java +++ b/src/java/org/apache/cassandra/replication/MutationTrackingService.java @@ -20,6 +20,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -27,6 +28,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.LongSupplier; @@ -63,6 +65,8 @@ import org.apache.cassandra.service.reads.tracked.TrackedLocalReads; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.listeners.ChangeListener; import org.apache.cassandra.tcm.ownership.ReplicaGroups; import org.apache.cassandra.tcm.ownership.VersionedEndpoints; import org.apache.cassandra.utils.FBUtilities; @@ -92,8 +96,24 @@ public class MutationTrackingService public static final MutationTrackingService instance = new MutationTrackingService(); private final TrackedLocalReads localReads = new TrackedLocalReads(); - private final ConcurrentHashMap keyspaceShards = new ConcurrentHashMap<>(); - private final ConcurrentHashMap log2ShardMap = new ConcurrentHashMap<>(); + private ConcurrentHashMap keyspaceShards = new ConcurrentHashMap<>(); + private ConcurrentHashMap log2ShardMap = new ConcurrentHashMap<>(); + private final ChangeListener tcmListener; + + // prevents a race between topology changes (shard recreation) and coordinator log creation. + // + // coordinator log creation can race with topology updates and be lost if shard recreation discards the old + // KeyspaceShards containing newly created logs. + // + // the following usage patterns will guard against state corruption during topology changes + // - Read lock: All normal operations (log creation, mutations, reads) + // - Write lock: Topology changes only (shard recreation during cluster membership changes) + // + // Topology changes are rare vs shard recreation speed, so brief blocking during cluster changes seems acceptable + // for correctness vs complex protocols topology updates. You could make the case that mutable state would be + // a better tradeoff for node replacement, but it seems likely that handling token movements will be simpler + // if we use a copy on write pattern for topology changes. + private final ReentrantReadWriteLock shardLock = new ReentrantReadWriteLock(); private final ReplicatedOffsetsBroadcaster offsetsBroadcaster = new ReplicatedOffsetsBroadcaster(); private final LogStatePersister offsetsPersister = new LogStatePersister(); @@ -104,9 +124,18 @@ public class MutationTrackingService private volatile boolean started = false; - private MutationTrackingService() {} + private MutationTrackingService() + { + this.tcmListener = new ChangeListener() + { + @Override + public void notifyPostCommit(ClusterMetadata prev, ClusterMetadata next, boolean fromSnapshot) + { + onNewClusterMetadata(prev, next); + } + }; + } - // TODO (expected): implement a TCM ChangeListener public synchronized void start(ClusterMetadata metadata) { if (started) @@ -120,6 +149,8 @@ public synchronized void start(ClusterMetadata metadata) for (KeyspaceShards ks : KeyspaceShards.loadFromSystemTables(metadata, this::nextLogId, this::onNewLog)) keyspaceShards.put(ks.keyspace, ks); + onNewClusterMetadata(null, metadata); + offsetsBroadcaster.start(); offsetsPersister.start(); @@ -128,6 +159,38 @@ public synchronized void start(ClusterMetadata metadata) started = true; } + public void pauseOffsetBroadcast(boolean pause) + { + offsetsBroadcaster.pauseOffsetBroadcast(pause); + } + + /** + * Creates a ShardReconciledOffsets containing reconciled offsets and ranges for multiple keyspaces. + */ + public ReconciledLogSnapshot snapshotReconciledLogs() + { + ReconciledLogSnapshot.Builder builder = ReconciledLogSnapshot.builder(); + + shardLock.readLock().lock(); + try + { + keyspaceShards.forEach((keyspace, ksShards) -> { + ksShards.collectShardReconciledOffsetsToBuilder(builder); + }); + } + finally + { + shardLock.readLock().unlock(); + } + + return builder.build(); + } + + public void registerTCMListener() + { + ClusterMetadataService.instance().log().addListener(tcmListener); + } + public synchronized boolean isStarted() { return started; @@ -135,7 +198,7 @@ public synchronized boolean isStarted() public void shutdownBlocking() throws InterruptedException { - // TODO: FIXME + ClusterMetadataService.instance().log().removeListener(tcmListener); activeReconciler.shutdownBlocking(); executor.shutdown(); executor.awaitTermination(1, TimeUnit.MINUTES); @@ -149,9 +212,17 @@ public TrackedLocalReads localReads() public MutationId nextMutationId(String keyspace, Token token) { - MutationId id = getOrCreateShards(keyspace).nextMutationId(token); - logger.trace("Created new mutation id {}", id); - return id; + shardLock.readLock().lock(); + try + { + MutationId id = getOrCreateShards(keyspace).nextMutationId(token); + logger.trace("Created new mutation id {}", id); + return id; + } + finally + { + shardLock.readLock().unlock(); + } } public void sentWriteRequest(Mutation mutation, IntHashSet toHostIds) @@ -162,12 +233,20 @@ public void sentWriteRequest(Mutation mutation, IntHashSet toHostIds) public void receivedWriteResponse(ShortMutationId mutationId, InetAddressAndPort fromHost) { - Preconditions.checkArgument(!mutationId.isNone()); - Shard shard = getShardNullable(mutationId); - // A response to the coordinator (for a forwarded write) won't have the coordinator log matching it - if (shard != null) - shard.receivedWriteResponse(mutationId, fromHost); + shardLock.readLock().lock(); + try + { + Preconditions.checkArgument(!mutationId.isNone()); + Shard shard = getShardNullable(mutationId); + // A response to the coordinator (for a forwarded write) won't have the coordinator log matching it + if (shard != null) + shard.receivedWriteResponse(mutationId, fromHost); // outgoingMutations.receivedWriteResponse(mutationId, ClusterMetadata.current().directory.peerId(fromHost).id()); + } + finally + { + shardLock.readLock().unlock(); + } } public void retryFailedWrite(ShortMutationId mutationId, InetAddressAndPort onHost, RequestFailure reason) @@ -179,20 +258,61 @@ public void retryFailedWrite(ShortMutationId mutationId, InetAddressAndPort onHo public void updateReplicatedOffsets(String keyspace, Range range, List offsets, boolean durable, InetAddressAndPort onHost) { - getOrCreateShards(keyspace).updateReplicatedOffsets(range, offsets, durable, onHost); + shardLock.readLock().lock(); + try + { + getOrCreateShards(keyspace).updateReplicatedOffsets(range, offsets, durable, onHost); + } + finally + { + shardLock.readLock().unlock(); + } + } + + public void recordFullyReconciledOffsets(ReconciledLogSnapshot reconciledSnapshot) + { + shardLock.readLock().lock(); + try + { + reconciledSnapshot.forEach((keyspace, keyspaceOffsets) -> { + KeyspaceShards ksShards = getOrCreateShards(keyspace); + if (ksShards != null) + ksShards.recordFullyReconciledOffsets(keyspaceOffsets); + }); + } + finally + { + shardLock.readLock().unlock(); + } } public boolean startWriting(Mutation mutation) { - Preconditions.checkArgument(!mutation.id().isNone()); - return getOrCreateShards(mutation.getKeyspaceName()).startWriting(mutation); + shardLock.readLock().lock(); + try + { + Preconditions.checkArgument(!mutation.id().isNone()); + return getOrCreateShards(mutation.getKeyspaceName()).startWriting(mutation); + } + finally + { + shardLock.readLock().unlock(); + } } public void finishWriting(Mutation mutation) { - Preconditions.checkArgument(!mutation.id().isNone()); - getOrCreateShards(mutation.getKeyspaceName()).finishWriting(mutation); - incomingMutations.invokeListeners(mutation.id()); + shardLock.readLock().lock(); + try + { + Preconditions.checkArgument(!mutation.id().isNone()); + getOrCreateShards(mutation.getKeyspaceName()).finishWriting(mutation); + incomingMutations.invokeListeners(mutation.id()); + } + finally + { + shardLock.readLock().unlock(); + } } /** @@ -206,12 +326,28 @@ public boolean registerMutationCallback(ShortMutationId mutationId, IncomingMuta public MutationSummary createSummaryForKey(DecoratedKey key, TableId tableId, boolean includePending) { - return getOrCreateShards(tableId).createSummaryForKey(key, tableId, includePending); + shardLock.readLock().lock(); + try + { + return getOrCreateShards(tableId).createSummaryForKey(key, tableId, includePending); + } + finally + { + shardLock.readLock().unlock(); + } } public MutationSummary createSummaryForRange(AbstractBounds range, TableId tableId, boolean includePending) { - return getOrCreateShards(tableId).createSummaryForRange(range, tableId, includePending); + shardLock.readLock().lock(); + try + { + return getOrCreateShards(tableId).createSummaryForRange(range, tableId, includePending); + } + finally + { + shardLock.readLock().unlock(); + } } public MutationSummary createSummaryForRange(Range range, TableId tableId, boolean includePending) @@ -221,28 +357,52 @@ public MutationSummary createSummaryForRange(Range range, TableId tableId void forEachKeyspace(Consumer consumer) { - for (KeyspaceShards keyspaceShards : keyspaceShards.values()) - consumer.accept(keyspaceShards); + shardLock.readLock().lock(); + try + { + for (KeyspaceShards keyspaceShards : keyspaceShards.values()) + consumer.accept(keyspaceShards); + } + finally + { + shardLock.readLock().unlock(); + } } public void collectLocallyMissingMutations(MutationSummary remoteSummary, Log2OffsetsMap.Mutable into) { - Iterator iterator = remoteSummary.onlyUnreconciled(); - while (iterator.hasNext()) + shardLock.readLock().lock(); + try { - Offsets offsets = iterator.next(); - Shard shard = getShardNullable(offsets.logId); - if (shard == null) - into.add(offsets); // if the log/shard are unknown, then all the offsets are also unkown/missing - else - shard.collectLocallyMissingMutations(offsets, into); + Iterator iterator = remoteSummary.onlyUnreconciled(); + while (iterator.hasNext()) + { + Offsets offsets = iterator.next(); + Shard shard = getShardNullable(offsets.logId); + if (shard == null) + into.add(offsets); // if the log/shard are unknown, then all the offsets are also unkown/missing + else + shard.collectLocallyMissingMutations(offsets, into); + } + } + finally + { + shardLock.readLock().unlock(); } } public void collectRemotelyMissingMutations(Offsets localOffsets, IntArrayList remoteNodeIds, Node2OffsetsMap into) { - Shard shard = getShard(localOffsets.logId()); - shard.collectRemotelyMissingMutations(localOffsets, remoteNodeIds, into); + shardLock.readLock().lock(); + try + { + Shard shard = getShard(localOffsets.logId()); + shard.collectRemotelyMissingMutations(localOffsets, remoteNodeIds, into); + } + finally + { + shardLock.readLock().unlock(); + } } public void requestMissingMutations(Offsets offsets, InetAddressAndPort forHost) @@ -300,41 +460,238 @@ private int nextHostLogId() public boolean isDurablyReconciled(ImmutableCoordinatorLogOffsets logOffsets) { - // 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) + shardLock.readLock().lock(); + try { - Shard shard = getShardNullable(new CoordinatorLogId(logId)); - if (shard == null) - throw new IllegalStateException("Could not find shard for logId " + logId); + // 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) + { + Shard shard = getShardNullable(new CoordinatorLogId(logId)); + if (shard == null) + throw new IllegalStateException("Could not find shard for logId " + logId); + + if (!shard.isDurablyReconciled(logId, logOffsets)) + return false; + } + return true; + } + finally + { + shardLock.readLock().unlock(); + } + } + + private void onNewClusterMetadata(@Nullable ClusterMetadata prev, ClusterMetadata next) + { + if (logger.isTraceEnabled()) + logger.trace("Processing cluster metadata change - epoch {} -> {}", + prev != null ? prev.epoch : "none", next.epoch); + + shardLock.readLock().lock(); + try + { + if (!shardUpdateNeeded(keyspaceShards, prev, next)) + return; + } + finally + { + shardLock.readLock().unlock(); + } + + shardLock.writeLock().lock(); + ConcurrentHashMap originalLog2ShardMap = log2ShardMap; + ConcurrentHashMap originalKeyspaceShards = keyspaceShards; + try + { + if (!shardUpdateNeeded(keyspaceShards, prev, next)) + return; + + // recalculating the shards will repopulate this via the existing callbacks + log2ShardMap = new ConcurrentHashMap<>(); + keyspaceShards = applyUpdatedMetadata(keyspaceShards, prev, next, this::nextLogId, this::onNewLog); + } + catch (Throwable t) + { + log2ShardMap = originalLog2ShardMap; + keyspaceShards = originalKeyspaceShards; + throw t; + } + finally + { + shardLock.writeLock().unlock(); + } + + } + + private static boolean shardUpdateNeeded(Map current, @Nullable ClusterMetadata prev, ClusterMetadata next) + { + Preconditions.checkNotNull(next); + + current = new HashMap<>(current); + + Set allKeyspaces = new HashSet<>(); + allKeyspaces.addAll(current.keySet()); + allKeyspaces.addAll(next.schema.getKeyspaces().names()); + if (prev != null) + allKeyspaces.addAll(prev.schema.getKeyspaces().names()); + + for (String keyspace : allKeyspaces) + { + KeyspaceShards.UpdateDecision decision = KeyspaceShards.UpdateDecision.decisionForTopologyChange(keyspace, prev, next, current.containsKey(keyspace)); + if (decision != KeyspaceShards.UpdateDecision.NONE) + return true; + } + + return false; + } + + + private static ConcurrentHashMap applyUpdatedMetadata(Map keyspaceShardsMap, @Nullable ClusterMetadata prev, ClusterMetadata next, LongSupplier logIdProvider, BiConsumer onNewLog) + { + Preconditions.checkNotNull(next); - if (!shard.isDurablyReconciled(logId, logOffsets)) - return false; + Map currentShards = new HashMap<>(keyspaceShardsMap); + ConcurrentHashMap updated = new ConcurrentHashMap<>(); + + Set allKeyspaces = new HashSet<>(); + allKeyspaces.addAll(currentShards.keySet()); + allKeyspaces.addAll(next.schema.getKeyspaces().names()); + if (prev != null) + allKeyspaces.addAll(prev.schema.getKeyspaces().names()); + + + for (String keyspace : allKeyspaces) + { + KeyspaceShards current = currentShards.remove(keyspace); + KeyspaceShards.UpdateDecision decision = KeyspaceShards.UpdateDecision.decisionForTopologyChange(keyspace, prev, next, current != null); + switch (decision) + { + case NONE: + if (current != null) + updated.put(keyspace, current); + break; + case DROP: + // Don't carry forward the state for the dropped keyspace + break; + case REPLICA_GROUP: + // if there's an existing keyspace shards instance, update it, otherwise call through to CREATE + if (current != null) + { + KeyspaceShards ksShards = current.withUpdatedMetadata(next.schema.getKeyspaceMetadata(keyspace), next, logIdProvider, onNewLog); + updated.put(keyspace, ksShards); + break; + } + case CREATE: + Preconditions.checkState(current == null, + "Attempted to create a new keyspace shard for keyspace %s, but it already exists", keyspace); + KeyspaceShards ksShards = KeyspaceShards.make(next.schema.getKeyspaceMetadata(keyspace), + next, + logIdProvider, + onNewLog); + updated.put(keyspace, ksShards); + break; + case MIGRATE_TO: + case MIGRATE_FROM: + default: + throw new IllegalStateException("Unsupported keyspace shard update: " + decision); + } } - return true; + + if (!currentShards.isEmpty()) + throw new IllegalStateException("At least one keyspace shards instance wasn't migrated: " + currentShards); + + return updated; } // TODO (expected): when topology and state truncation is implemented, implement cleanup of this map as well private void onNewLog(Shard shard, CoordinatorLog log) { - log2ShardMap.put(log.logId, shard); + shardLock.readLock().lock(); + try + { + log2ShardMap.put(log.logId, shard); + } + finally + { + shardLock.readLock().unlock(); + } } - private static class KeyspaceShards + public static class KeyspaceShards { + private enum UpdateDecision + { + NONE, + CREATE, + DROP, + REPLICA_GROUP, + MIGRATE_TO, + MIGRATE_FROM; + + static UpdateDecision decisionForTopologyChange(String keyspace, ClusterMetadata prev, ClusterMetadata next, boolean hasExisting) + { + KeyspaceMetadata prevKsm = prev != null ? prev.schema.getKeyspaces().get(keyspace).orElse(null) : null; + KeyspaceMetadata nextKsm = next.schema.getKeyspaces().get(keyspace).orElse(null); + + if (prevKsm == null && nextKsm == null) + { + if (hasExisting) + throw new IllegalStateException(String.format("Mutation tracking exists for unknown keyspace %s", keyspace)); + return NONE; + } + + if (prevKsm == null) + return nextKsm.useMutationTracking() ? UpdateDecision.CREATE : UpdateDecision.NONE; + + if (nextKsm == null) + return prevKsm.useMutationTracking() ? UpdateDecision.DROP : UpdateDecision.NONE; + + if (!prevKsm.useMutationTracking() && !nextKsm.useMutationTracking()) + { + Preconditions.checkState(!hasExisting, "Existing shards found for keyspace, but prev & current ksm has mutation tracking disabled"); + return UpdateDecision.NONE; + } + + if (prevKsm.useMutationTracking() && !nextKsm.useMutationTracking()) + { + return UpdateDecision.MIGRATE_FROM; + } + + if (!prevKsm.useMutationTracking() && nextKsm.useMutationTracking()) + { + Preconditions.checkState(!hasExisting, "Existing shard found for keyspace, but prev ksn has mutation tracking disabled"); + return UpdateDecision.MIGRATE_TO; + } + + if (!calculateParticipantsForRange(nextKsm, next).equals(calculateParticipantsForRange(prevKsm, prev))) + return UpdateDecision.REPLICA_GROUP; + + return UpdateDecision.NONE; + } + } + private final String keyspace; private final Map, Shard> shards; private final ReplicaGroups groups; private transient final Map, Shard> ppShards; - static KeyspaceShards make(KeyspaceMetadata keyspace, ClusterMetadata cluster, LongSupplier logIdProvider, BiConsumer onNewLog) + private static class ParticipantForRange { - Preconditions.checkArgument(keyspace.params.replicationType.isTracked()); + final Participants participants; + final VersionedEndpoints.ForRange forRange; - Map, Shard> shards = new HashMap<>(); - Map, VersionedEndpoints.ForRange> groups = new HashMap<>(); + public ParticipantForRange(Participants participants, VersionedEndpoints.ForRange forRange) + { + this.participants = participants; + this.forRange = forRange; + } + } + private static Map, ParticipantForRange> calculateParticipantsForRange(KeyspaceMetadata keyspace, ClusterMetadata cluster) + { + Map, ParticipantForRange> result = new HashMap<>(); cluster.placements.get(keyspace.params.replication).writes.forEach((fullTokenRange, forRange) -> { if (!forRange.endpoints().contains(FBUtilities.getBroadcastAddressAndPort())) return; @@ -344,10 +701,31 @@ static KeyspaceShards make(KeyspaceMetadata keyspace, ClusterMetadata cluster, L participantList.add(cluster.directory.peerId(endpoint).id()); Participants participants = new Participants(participantList); - Optional splitter = fullTokenRange.left.getPartitioner().splitter(); - Set> ranges = splitter.isPresent() && SHARD_MULTIPLIER > 1 - ? splitter.get().split(fullTokenRange, SHARD_MULTIPLIER) - : Collections.singleton(fullTokenRange); + result.put(fullTokenRange, new ParticipantForRange(participants, forRange)); + }); + return result; + } + + private static Set> splitRange(Range range) + { + Optional splitter = range.left.getPartitioner().splitter(); + return splitter.isPresent() && SHARD_MULTIPLIER > 1 + ? splitter.get().split(range, SHARD_MULTIPLIER) + : Collections.singleton(range); + } + + static KeyspaceShards make(KeyspaceMetadata keyspace, ClusterMetadata cluster, LongSupplier logIdProvider, BiConsumer onNewLog) + { + Preconditions.checkArgument(keyspace.params.replicationType.isTracked()); + + Map, Shard> shards = new HashMap<>(); + Map, VersionedEndpoints.ForRange> groups = new HashMap<>(); + + calculateParticipantsForRange(keyspace, cluster).forEach((fullTokenRange, participantForRange) -> { + Participants participants = participantForRange.participants; + VersionedEndpoints.ForRange forRange = participantForRange.forRange; + + Set> ranges = splitRange(fullTokenRange); for (Range tokenRange : ranges) { @@ -371,6 +749,39 @@ static KeyspaceShards make(KeyspaceMetadata keyspace, ClusterMetadata cluster, L this.ppShards = ppShards; } + KeyspaceShards withUpdatedMetadata(KeyspaceMetadata keyspace, ClusterMetadata cluster, LongSupplier logIdProvider, BiConsumer onNewLog) + { + Map, Shard> currentShards = new HashMap<>(shards); + Map, Shard> newShards = new HashMap<>(); + Map, VersionedEndpoints.ForRange> newGroups = new HashMap<>(); + + calculateParticipantsForRange(keyspace, cluster).forEach((fullTokenRange, participantForRange) -> { + Participants participants = participantForRange.participants; + VersionedEndpoints.ForRange forRange = participantForRange.forRange; + + Set> ranges = splitRange(fullTokenRange); + + for (Range tokenRange : ranges) + { + Shard currentShard = currentShards.remove(tokenRange); + if (currentShard != null) + { + newShards.put(tokenRange, currentShard.withParticipants(participants)); + newGroups.put(tokenRange, forRange.map(original -> original.withRange(tokenRange))); + } + else + { + newShards.put(tokenRange, new Shard(cluster.myNodeId().id(), keyspace.name, tokenRange, participants, logIdProvider, onNewLog)); + newGroups.put(tokenRange, forRange.map(original -> original.withRange(tokenRange))); + } + } + }); + + newShards.values().forEach(Shard::reportAllLogsToCallback); + + return new KeyspaceShards(keyspace.name, newShards, new ReplicaGroups(newGroups)); + } + MutationId nextMutationId(Token token) { return lookUp(token).nextId(); @@ -378,7 +789,10 @@ MutationId nextMutationId(Token token) void updateReplicatedOffsets(Range range, List offsets, boolean durable, InetAddressAndPort onHost) { - shards.get(range).updateReplicatedOffsets(offsets, durable, onHost); + Shard shard = shards.get(range); + if (shard == null) + return; + shard.updateReplicatedOffsets(offsets, durable, onHost); } boolean startWriting(Mutation mutation) @@ -416,6 +830,22 @@ private void forEachIntersectingShard(AbstractBounds bounds, }); } + void collectShardReconciledOffsetsToBuilder(ReconciledLogSnapshot.Builder builder) + { + ReconciledKeyspaceOffsets.Builder keyspaceBuilder = builder.getKeyspaceBuilder(keyspace); + ppShards.values().forEach(shard -> shard.collectShardReconciledOffsetsToBuilder(keyspaceBuilder)); + } + + void recordFullyReconciledOffsets(ReconciledKeyspaceOffsets keyspaceOffsets) + { + keyspaceOffsets.forEach((logId, entry) -> { + // Find the shard that should contain this log based on the range + Shard shard = shards.get(entry.range); + if (shard != null) + shard.recordFullyReconciledOffsets(logId, entry.offsets); + }); + } + void forEachShard(Consumer consumer) { for (Shard shard : shards.values()) @@ -492,6 +922,8 @@ private static class ReplicatedOffsetsBroadcaster private static final long TRANSIENT_BROADCAST_INTERVAL_MILLIS = 200; private static final long DURABLE_BROADCAST_INTERVAL_MILLIS = 60_000; + private volatile boolean isPaused = false; + void start() { executor.scheduleWithFixedDelay(() -> run(false), @@ -504,6 +936,11 @@ void start() TimeUnit.MILLISECONDS); } + public void pauseOffsetBroadcast(boolean pause) + { + isPaused = pause; + } + public void run(boolean durable) { MutationTrackingService.instance.forEachKeyspace(ks -> run(ks, durable)); @@ -511,7 +948,8 @@ public void run(boolean durable) private void run(KeyspaceShards shards, boolean durable) { - shards.forEachShard(sh -> run(sh, durable)); + if (!isPaused) + shards.forEachShard(sh -> run(sh, durable)); } private void run(Shard shard, boolean durable) @@ -573,4 +1011,18 @@ public void resumeActiveReconciler() { activeReconciler.resumeForTesting(); } + + @VisibleForTesting + public static class TestAccess + { + public static MutationTrackingService create() + { + return new MutationTrackingService(); + } + + public static KeyspaceShards getKeyspaceShards(MutationTrackingService service, String keyspace) + { + return service.keyspaceShards.get(keyspace); + } + } } diff --git a/src/java/org/apache/cassandra/replication/Participants.java b/src/java/org/apache/cassandra/replication/Participants.java index 89494f240c5e..5fd0f5bb8d93 100644 --- a/src/java/org/apache/cassandra/replication/Participants.java +++ b/src/java/org/apache/cassandra/replication/Participants.java @@ -51,7 +51,7 @@ int indexOf(int hostId) boolean contains(int hostId) { - return indexOf(hostId) >= 0; + return Arrays.binarySearch(hosts, hostId) >= 0; } int get(int idx) diff --git a/src/java/org/apache/cassandra/replication/ReconciledKeyspaceOffsets.java b/src/java/org/apache/cassandra/replication/ReconciledKeyspaceOffsets.java new file mode 100644 index 000000000000..e28a69821cee --- /dev/null +++ b/src/java/org/apache/cassandra/replication/ReconciledKeyspaceOffsets.java @@ -0,0 +1,269 @@ +/* + * 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.Objects; +import java.util.function.BiConsumer; + +import org.agrona.collections.Long2ObjectHashMap; +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.IPartitioner; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; + +public class ReconciledKeyspaceOffsets +{ + public static final IVersionedSerializer serializer = new Serializer(); + + /** + * Simple data holder for offsets and their associated range + */ + static class Entry + { + public final Offsets.Immutable offsets; + public final Range range; + + public Entry(Offsets.Immutable offsets, Range range) + { + this.offsets = offsets; + this.range = range; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Entry entry = (Entry) o; + return Objects.equals(offsets, entry.offsets) && Objects.equals(range, entry.range); + } + + @Override + public int hashCode() + { + return Objects.hash(offsets, range); + } + + @Override + public String toString() + { + return "LogEntry{offsets=" + offsets + ", range=" + range + '}'; + } + } + + private final Long2ObjectHashMap logEntries; + + private ReconciledKeyspaceOffsets(Long2ObjectHashMap logEntries) + { + this.logEntries = logEntries; + } + + public boolean isFullyReconciled(ShortMutationId mutationId) + { + Entry entry = logEntries.get(mutationId.logId()); + return entry != null && entry.offsets.contains(mutationId.offset()); + } + + public boolean isFullyReconciled(long logId, Offsets.Immutable offsets) + { + Entry entry = logEntries.get(logId); + if (entry == null) + return false; + + Offsets.RangeIterator diff = Offsets.difference(offsets.rangeIterator(), entry.offsets.rangeIterator()); + return !diff.tryAdvance(); + } + + public Offsets.Immutable get(CoordinatorLogId logId) + { + Entry entry = logEntries.get(logId.asLong()); + return entry != null ? entry.offsets : null; + } + + public Range getRange(CoordinatorLogId logId) + { + Entry entry = logEntries.get(logId.asLong()); + return entry != null ? entry.range : null; + } + + public Entry getLogEntry(CoordinatorLogId logId) + { + return logEntries.get(logId.asLong()); + } + + public Long2ObjectHashMap getAllOffsets() + { + Long2ObjectHashMap result = new Long2ObjectHashMap<>(); + logEntries.forEachLong((logId, entry) -> result.put(logId, entry.offsets)); + return result; + } + + public Long2ObjectHashMap> getAllRanges() + { + Long2ObjectHashMap> result = new Long2ObjectHashMap<>(); + logEntries.forEachLong((logId, entry) -> result.put(logId, entry.range)); + return result; + } + + void forEach(BiConsumer consumer) + { + logEntries.forEachLong((logId, entry) -> consumer.accept(new CoordinatorLogId(logId), entry)); + } + + public boolean isEmpty() + { + return logEntries.isEmpty(); + } + + public int size() + { + return logEntries.size(); + } + + public boolean contains(CoordinatorLogId logId) + { + return logEntries.containsKey(logId.asLong()); + } + + /** + * Selects log entries whose ranges intersect with any of the target ranges + * and adds them to the provided builder. + * + * @param targetRanges ranges to intersect with + * @param builder builder to add intersecting entries to + */ + void selectIntersecting(Collection> targetRanges, Builder builder) + { + logEntries.forEachLong((logId, entry) -> { + Range logRange = entry.range; + for (Range targetRange : targetRanges) + { + if (logRange.intersects(targetRange)) + { + builder.put(new CoordinatorLogId(logId), entry.offsets, entry.range); + break; + } + } + }); + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ReconciledKeyspaceOffsets that = (ReconciledKeyspaceOffsets) o; + return Objects.equals(logEntries, that.logEntries); + } + + @Override + public int hashCode() + { + return Objects.hash(logEntries); + } + + @Override + public String toString() + { + return "ReconciledKeyspaceOffsets{" + + "logEntries=" + logEntries + + '}'; + } + + public static class Builder + { + private final Long2ObjectHashMap logEntries = new Long2ObjectHashMap<>(); + + public Builder put(CoordinatorLogId logId, Offsets.Immutable reconciled, Range range) + { + logEntries.put(logId.asLong(), new Entry(reconciled, range)); + return this; + } + + public ReconciledKeyspaceOffsets build() + { + return new ReconciledKeyspaceOffsets(logEntries); + } + } + + public static Builder builder() + { + return new Builder(); + } + + public static class Serializer implements IVersionedSerializer + { + @Override + public void serialize(ReconciledKeyspaceOffsets keyspaceOffsets, DataOutputPlus out, int version) throws IOException + { + out.writeInt(keyspaceOffsets.logEntries.size()); + + keyspaceOffsets.logEntries.forEachLong((logId, entry) -> { + try + { + out.writeLong(logId); + Offsets.serializer.serialize(entry.offsets, out, version); + AbstractBounds.tokenSerializer.serialize(entry.range, out, version); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }); + } + + @Override + public ReconciledKeyspaceOffsets deserialize(DataInputPlus in, int version) throws IOException + { + int logCount = in.readInt(); + Long2ObjectHashMap logEntries = new Long2ObjectHashMap<>(); + + for (int j = 0; j < logCount; j++) + { + long logId = in.readLong(); + Offsets.Immutable offsets = Offsets.serializer.deserialize(in, version); + Range range = (Range) AbstractBounds.tokenSerializer.deserialize(in, IPartitioner.global(), version); + logEntries.put(logId, new Entry(offsets, range)); + } + + return new ReconciledKeyspaceOffsets(logEntries); + } + + @Override + public long serializedSize(ReconciledKeyspaceOffsets keyspaceOffsets, int version) + { + long size = TypeSizes.sizeof(keyspaceOffsets.logEntries.size()); + + final long[] totalSize = { size }; + keyspaceOffsets.logEntries.forEachLong((logId, entry) -> { + totalSize[0] += TypeSizes.sizeof(logId); + totalSize[0] += Offsets.serializer.serializedSize(entry.offsets, version); + totalSize[0] += AbstractBounds.tokenSerializer.serializedSize(entry.range, version); + }); + + return totalSize[0]; + } + } +} diff --git a/src/java/org/apache/cassandra/replication/ReconciledLogSnapshot.java b/src/java/org/apache/cassandra/replication/ReconciledLogSnapshot.java new file mode 100644 index 000000000000..08ebf104bcdb --- /dev/null +++ b/src/java/org/apache/cassandra/replication/ReconciledLogSnapshot.java @@ -0,0 +1,224 @@ +/* + * 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.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.function.BiConsumer; + +import com.google.common.collect.ImmutableMap; + +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; + +/** + * Container for reconciled offsets organized by keyspace and shard, including range information for each log. + * This is similar to LogReconciledOffsets but adds range tracking for coordinator logs. + */ +public class ReconciledLogSnapshot +{ + public static final IVersionedSerializer serializer = new Serializer(); + + private final ImmutableMap reconciled; + + private ReconciledLogSnapshot(ImmutableMap reconciled) + { + this.reconciled = reconciled; + } + + public boolean isFullyReconciled(String keyspace, ShortMutationId mutationId) + { + ReconciledKeyspaceOffsets keyspaceOffsets = reconciled.get(keyspace); + if (keyspaceOffsets == null) + return true; + return keyspaceOffsets.isFullyReconciled(mutationId); + } + + public void forEach(BiConsumer consumer) + { + reconciled.forEach(consumer); + } + + public Offsets.Immutable get(String keyspace, CoordinatorLogId logId) + { + ReconciledKeyspaceOffsets keyspaceOffsets = reconciled.get(keyspace); + return keyspaceOffsets != null ? keyspaceOffsets.get(logId) : null; + } + + public Range getRange(String keyspace, CoordinatorLogId logId) + { + ReconciledKeyspaceOffsets keyspaceOffsets = reconciled.get(keyspace); + return keyspaceOffsets != null ? keyspaceOffsets.getRange(logId) : null; + } + + public ReconciledKeyspaceOffsets getKeyspace(String keyspace) + { + return reconciled.get(keyspace); + } + + public ImmutableMap getAll() + { + return reconciled; + } + + public boolean isEmpty() + { + return size() == 0; + } + + public int size() + { + return reconciled.values().stream().mapToInt(ReconciledKeyspaceOffsets::size).sum(); + } + + /** + * Creates a filtered subset of this snapshot containing only log entries whose ranges + * intersect with the specified keyspace ranges. + * + * @param keyspaceRanges map of keyspace name to ranges to filter by + * @return new ReconciledLogSnapshot containing only intersecting entries + */ + public ReconciledLogSnapshot select(Map>> keyspaceRanges) + { + ReconciledLogSnapshot.Builder builder = ReconciledLogSnapshot.builder(); + + for (Map.Entry>> entry : keyspaceRanges.entrySet()) + { + String keyspace = entry.getKey(); + Collection> targetRanges = entry.getValue(); + + ReconciledKeyspaceOffsets keyspaceOffsets = reconciled.get(keyspace); + if (keyspaceOffsets == null) + continue; + + ReconciledKeyspaceOffsets.Builder keyspaceBuilder = builder.getKeyspaceBuilder(keyspace); + keyspaceOffsets.selectIntersecting(targetRanges, keyspaceBuilder); + } + + return builder.build(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ReconciledLogSnapshot that = (ReconciledLogSnapshot) o; + return Objects.equals(reconciled, that.reconciled); + } + + @Override + public int hashCode() + { + return Objects.hash(reconciled); + } + + @Override + public String toString() + { + return "ShardReconciledOffsets{" + + "reconciled=" + reconciled + + '}'; + } + + public static class Builder + { + private final Map keyspaceBuilders = new HashMap<>(); + + public Builder put(String keyspace, CoordinatorLogId logId, Offsets.Immutable offsets, Range range) + { + keyspaceBuilders.computeIfAbsent(keyspace, k -> ReconciledKeyspaceOffsets.builder()) + .put(logId, offsets, range); + return this; + } + + ReconciledKeyspaceOffsets.Builder getKeyspaceBuilder(String keyspace) + { + return keyspaceBuilders.computeIfAbsent(keyspace, k -> ReconciledKeyspaceOffsets.builder()); + } + + public ReconciledLogSnapshot build() + { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (Map.Entry entry : keyspaceBuilders.entrySet()) + { + ReconciledKeyspaceOffsets ks = entry.getValue().build(); + if (!ks.isEmpty()) + builder.put(entry.getKey(), ks); + } + return new ReconciledLogSnapshot(builder.build()); + } + } + + public static Builder builder() + { + return new Builder(); + } + + public static class Serializer implements IVersionedSerializer + { + @Override + public void serialize(ReconciledLogSnapshot offsets, DataOutputPlus out, int version) throws IOException + { + out.writeInt(offsets.reconciled.size()); + + for (Map.Entry keyspaceEntry : offsets.reconciled.entrySet()) + { + out.writeUTF(keyspaceEntry.getKey()); + ReconciledKeyspaceOffsets.serializer.serialize(keyspaceEntry.getValue(), out, version); + } + } + + @Override + public ReconciledLogSnapshot deserialize(DataInputPlus in, int version) throws IOException + { + int keyspaceCount = in.readInt(); + ImmutableMap.Builder builder = ImmutableMap.builder(); + + for (int i = 0; i < keyspaceCount; i++) + { + String keyspace = in.readUTF(); + ReconciledKeyspaceOffsets keyspaceOffsets = ReconciledKeyspaceOffsets.serializer.deserialize(in, version); + builder.put(keyspace, keyspaceOffsets); + } + + return new ReconciledLogSnapshot(builder.build()); + } + + @Override + public long serializedSize(ReconciledLogSnapshot offsets, int version) + { + long size = TypeSizes.sizeof(offsets.reconciled.size()); + + for (Map.Entry keyspaceEntry : offsets.reconciled.entrySet()) + { + size += TypeSizes.sizeof(keyspaceEntry.getKey()); + size += ReconciledKeyspaceOffsets.serializer.serializedSize(keyspaceEntry.getValue(), version); + } + + return size; + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/replication/Shard.java b/src/java/org/apache/cassandra/replication/Shard.java index 6f198e686d59..0717c228178e 100644 --- a/src/java/org/apache/cassandra/replication/Shard.java +++ b/src/java/org/apache/cassandra/replication/Shard.java @@ -58,7 +58,6 @@ public class Shard final String keyspace; final Range range; final Participants participants; - private final LongSupplier logIdProvider; private final BiConsumer onNewLog; private final NonBlockingHashMapLong logs; @@ -94,6 +93,63 @@ public class Shard this(localNodeId, keyspace, range, participants, Collections.emptyList(), logIdProvider, onNewLog); } + Shard(int localNodeId, + String keyspace, + Range range, + Participants participants, + NonBlockingHashMapLong logs, + CoordinatorLog.CoordinatorLogPrimary currentLocalLog, + LongSupplier logIdProvider, + BiConsumer onNewLog) + { + this.localNodeId = localNodeId; + this.keyspace = keyspace; + this.range = range; + this.participants = participants; + this.logIdProvider = logIdProvider; + this.logs = logs; + this.onNewLog = onNewLog; + this.currentLocalLog = currentLocalLog; + } + + /** + * For rebuilding the MTS log->shard index after a topology change + */ + void reportAllLogsToCallback() + { + logs.values().forEach(log -> { + onNewLog.accept(this, log); + }); + } + + Shard withParticipants(Participants newParticipants) + { + if (participants.equals(newParticipants)) + return this; + + if (logger.isTraceEnabled()) + logger.trace("Reconfiguring shard {} participants: {} -> {}", + range, participants, newParticipants); + + NonBlockingHashMapLong newLogs = new NonBlockingHashMapLong<>(); + CoordinatorLog.CoordinatorLogPrimary newCurrentLocalLog = null; + + // FIXME: confirm all new logs are added to the relevant views + for (CoordinatorLog log : logs.values()) + { + CoordinatorLog newLog = log.withParticipants(newParticipants); + newLogs.put(newLog.logId.asLong(), newLog); + + if (log == currentLocalLog) + newCurrentLocalLog = (CoordinatorLog.CoordinatorLogPrimary) newLog; + } + + Shard shard = new Shard(localNodeId, keyspace, range, newParticipants, + newLogs, newCurrentLocalLog, logIdProvider, onNewLog); + newLogs.values().forEach(log -> onNewLog.accept(shard, log)); + return shard; + } + MutationId nextId() { MutationId nextId = currentLocalLog.nextId(); @@ -127,6 +183,17 @@ void updateReplicatedOffsets(List offsets, boolean durable, I getOrCreate(logOffsets.logId()).updateReplicatedOffsets(logOffsets, durable, onHostId); } + public void recordFullyReconciledOffsets(CoordinatorLogId logId, Offsets.Immutable reconciled) + { + CoordinatorLog log = logs.get(logId.asLong()); + + // Create the coordinator log if it doesn't exist + if (log == null) + log = getOrCreate(logId); + + log.recordFullyReconciledOffsets(reconciled); + } + boolean startWriting(Mutation mutation) { return getOrCreate(mutation).startWriting(mutation); @@ -172,7 +239,18 @@ List remoteReplicas() { int hostId = participants.get(i); if (hostId != localNodeId) - replicas.add(ClusterMetadata.current().directory.endpoint(new NodeId(hostId))); + { + InetAddressAndPort ep = ClusterMetadata.current().directory.endpoint(new NodeId(hostId)); + if (ep == null) + { + // offset broadcasting can race with topology changes + // TODO (expected): consider adding a more sophisticated check so we don't die during normal topology operations, but still detect bugs + logger.warn("No endpoint found for hostId {}", hostId); + continue; + } + + replicas.add(ep); + } } return replicas; } @@ -282,4 +360,14 @@ static ArrayList loadFromSystemTables(int localNodeId, LongSupplier logId } return shards; } + + public Range tokenRange() + { + return range; + } + + void collectShardReconciledOffsetsToBuilder(ReconciledKeyspaceOffsets.Builder keyspaceBuilder) + { + logs.values().forEach(log -> keyspaceBuilder.put(log.logId, log.collectReconciledOffsets(), range)); + } } diff --git a/src/java/org/apache/cassandra/replication/TrackedWriteRequest.java b/src/java/org/apache/cassandra/replication/TrackedWriteRequest.java index e9ee3ce7cf04..95dda0143165 100644 --- a/src/java/org/apache/cassandra/replication/TrackedWriteRequest.java +++ b/src/java/org/apache/cassandra/replication/TrackedWriteRequest.java @@ -103,6 +103,11 @@ public static AbstractWriteResponseHandler perform( writeMetrics.localRequests.mark(); MutationId id = MutationTrackingService.instance.nextMutationId(keyspaceName, token); mutation = mutation.withMutationId(id); + + if (logger.isTraceEnabled()) + logger.trace("Write replication plan for mutation {}: live={}, pending={}, all={}", + id, plan.live(), plan.pending(), plan.contacts()); + TrackedWriteResponseHandler handler = TrackedWriteResponseHandler.wrap(rs.getWriteResponseHandler(plan, null, WriteType.SIMPLE, null, requestTime), id); applyLocallyAndSendToReplicas(mutation, plan, handler); @@ -137,6 +142,8 @@ public static void applyLocallyAndSendToReplicas(Mutation mutation, ReplicaPlan. { if (!plan.isAlive(destination)) { + if (logger.isTraceEnabled()) + logger.trace("Skipping dead replica {} for mutation {}", destination, mutation.id()); handler.expired(); // immediately mark the response as expired since the request will not be sent continue; } @@ -186,6 +193,8 @@ public static void applyLocallyAndSendToReplicas(Mutation mutation, ReplicaPlan. { for (Replica replica : localDCReplicas) { + if (logger.isTraceEnabled()) + logger.trace("Sending mutation {} to local replica {}", mutation.id(), replica); MessagingService.instance().sendWriteWithCallback(message, replica, handler); remoteReplicas.add(ClusterMetadata.current().directory.peerId(replica.endpoint()).id()); } @@ -196,6 +205,8 @@ public static void applyLocallyAndSendToReplicas(Mutation mutation, ReplicaPlan. // for each datacenter, send the message to one node to relay the write to other replicas for (List dcReplicas : remoteDCReplicas.values()) { + if (logger.isTraceEnabled()) + logger.trace("Sending mutation {} to remote dc replicas {}", mutation.id(), dcReplicas); sendMessagesToRemoteDC(message, EndpointsForToken.copyOf(mutation.key().getToken(), dcReplicas), handler, null); for (Replica replica : dcReplicas) remoteReplicas.add(ClusterMetadata.current().directory.peerId(replica.endpoint()).id()); @@ -203,7 +214,11 @@ public static void applyLocallyAndSendToReplicas(Mutation mutation, ReplicaPlan. } if (remoteReplicas != null) + { + if (logger.isTraceEnabled()) + logger.trace("Sending mutation {} to remote replicas {}", mutation.id(), remoteReplicas); MutationTrackingService.instance.sentWriteRequest(mutation, remoteReplicas); + } } static void applyMutationLocally(Mutation mutation, RequestCallback handler) diff --git a/src/java/org/apache/cassandra/replication/UnreconciledMutations.java b/src/java/org/apache/cassandra/replication/UnreconciledMutations.java index 63dff77349f4..036f624b4295 100644 --- a/src/java/org/apache/cassandra/replication/UnreconciledMutations.java +++ b/src/java/org/apache/cassandra/replication/UnreconciledMutations.java @@ -135,8 +135,16 @@ public void finishWriting(Mutation mutation) public void remove(int offset) { Entry state = statesMap.remove(offset); - Preconditions.checkNotNull(state); - statesSet.remove(state); + if (state != null) + statesSet.remove(state); + } + + public UnreconciledMutations copy() + { + UnreconciledMutations copy = new UnreconciledMutations(); + copy.statesMap.putAll(statesMap); + copy.statesSet.addAll(statesSet); + return copy; } public boolean collect(AbstractBounds range, TableId tableId, boolean includePending, Offsets.OffsetReciever into) @@ -214,6 +222,12 @@ void addDirectly(Mutation mutation) statesSet.add(entry); } + @VisibleForTesting + boolean isEmpty() + { + return statesMap.isEmpty(); + } + static UnreconciledMutations loadFromJournal(Node2OffsetsMap witnessedOffsets, int localNodeId) { UnreconciledMutations result = new UnreconciledMutations(); diff --git a/src/java/org/apache/cassandra/service/TrackedWriteResponseHandler.java b/src/java/org/apache/cassandra/service/TrackedWriteResponseHandler.java index 213cc90976c9..38e4480c9d87 100644 --- a/src/java/org/apache/cassandra/service/TrackedWriteResponseHandler.java +++ b/src/java/org/apache/cassandra/service/TrackedWriteResponseHandler.java @@ -17,6 +17,8 @@ */ package org.apache.cassandra.service; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.cassandra.exceptions.RequestFailure; import org.apache.cassandra.exceptions.WriteFailureException; import org.apache.cassandra.exceptions.WriteTimeoutException; @@ -28,6 +30,8 @@ public class TrackedWriteResponseHandler extends AbstractWriteResponseHandler { + private static final Logger logger = LoggerFactory.getLogger(TrackedWriteResponseHandler.class); + private final AbstractWriteResponseHandler wrapped; private final MutationId mutationId; @@ -49,13 +53,19 @@ public void onResponse(Message msg) { // Local mutations are witnessed from Keyspace.applyInternalTracked if (msg != null) + { + if (logger.isTraceEnabled()) + logger.trace("Received write response for mutation {} from {}", mutationId, msg.from()); MutationTrackingService.instance.receivedWriteResponse(mutationId, msg.from()); + } wrapped.onResponse(msg); } @Override public void onFailure(InetAddressAndPort from, RequestFailure failure) { + if (logger.isTraceEnabled()) + logger.trace("Write failed for mutation {} from {}: {}", mutationId, from, failure); MutationTrackingService.instance.retryFailedWrite(mutationId, from, failure); wrapped.onFailure(from, failure); } diff --git a/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedRead.java b/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedRead.java index c4db7ce259d9..32409e59d033 100644 --- a/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedRead.java +++ b/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedRead.java @@ -98,7 +98,7 @@ default void augment(Log2OffsetsMap augmentingOffsets) default void augment(ShortMutationId mutationId) { Mutation mutation = MutationJournal.instance.read(mutationId); - Preconditions.checkNotNull(mutation); + Preconditions.checkNotNull(mutation, "Missing mutation %s", mutationId); if (!command().selectsKey(mutation.key())) { logger.trace("Skipping mutation {} - {} not in read range", mutationId, mutation.key()); diff --git a/src/java/org/apache/cassandra/streaming/LogReceiveTask.java b/src/java/org/apache/cassandra/streaming/LogReceiveTask.java new file mode 100644 index 000000000000..ba83051b1921 --- /dev/null +++ b/src/java/org/apache/cassandra/streaming/LogReceiveTask.java @@ -0,0 +1,52 @@ +/* + * 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.streaming; + +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.streaming.messages.IncomingMutationLogStreamMessage; +import org.apache.cassandra.streaming.messages.MutationLogReceivedMessage; + +/** + * Task for tracking reception of mutation log streams. + */ +public class LogReceiveTask extends LogStreamTask +{ + + public LogReceiveTask(StreamSession session, InetAddressAndPort peer) + { + super(session, peer); + } + + public synchronized void received(IncomingMutationLogStreamMessage message) + { + // TODO: validate message header with expected ranges + if (markCompleted()) + { + session.taskCompleted(this); + // Send acknowledgment on successful completion + session.sendControlMessage(new MutationLogReceivedMessage()).syncUninterruptibly(); + + } + } + + @Override + public void abort() + { + // cleanup if needed + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/streaming/LogStreamHeader.java b/src/java/org/apache/cassandra/streaming/LogStreamHeader.java new file mode 100644 index 000000000000..c011c8c0e52a --- /dev/null +++ b/src/java/org/apache/cassandra/streaming/LogStreamHeader.java @@ -0,0 +1,122 @@ +/* + * 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.streaming; + +import java.io.IOException; +import java.util.Objects; + +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.replication.ReconciledLogSnapshot; +import org.apache.cassandra.utils.TimeUUID; + +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; + +/** + * Header for mutation log stream messages containing both manifest and session metadata. + */ +public class LogStreamHeader +{ + public final LogStreamManifest manifest; + public final ReconciledLogSnapshot reconciled; + public final InetAddressAndPort sender; + public final TimeUUID planId; + public final int sessionIndex; + public final boolean sendByFollower; + + public LogStreamHeader(LogStreamManifest manifest, + ReconciledLogSnapshot reconciled, + InetAddressAndPort sender, + TimeUUID planId, + int sessionIndex, + boolean sendByFollower) + { + this.manifest = manifest; + this.reconciled = reconciled; + this.sender = sender; + this.planId = planId; + this.sessionIndex = sessionIndex; + this.sendByFollower = sendByFollower; + } + + public static class Serializer implements IVersionedSerializer + { + public void serialize(LogStreamHeader header, DataOutputPlus out, int version) throws IOException + { + LogStreamManifest.serializer.serialize(header.manifest, out, version); + ReconciledLogSnapshot.serializer.serialize(header.reconciled, out, version); + inetAddressAndPortSerializer.serialize(header.sender, out, version); + header.planId.serialize(out); + out.writeInt(header.sessionIndex); + out.writeBoolean(header.sendByFollower); + } + + public LogStreamHeader deserialize(DataInputPlus in, int version) throws IOException + { + LogStreamManifest manifest = LogStreamManifest.serializer.deserialize(in, version); + ReconciledLogSnapshot reconciled = ReconciledLogSnapshot.serializer.deserialize(in, version); + InetAddressAndPort sender = inetAddressAndPortSerializer.deserialize(in, version); + TimeUUID planId = TimeUUID.deserialize(in); + int sessionIndex = in.readInt(); + boolean sendByFollower = in.readBoolean(); + return new LogStreamHeader(manifest, reconciled, sender, planId, sessionIndex, sendByFollower); + } + + public long serializedSize(LogStreamHeader header, int version) + { + return LogStreamManifest.serializer.serializedSize(header.manifest, version) + + ReconciledLogSnapshot.serializer.serializedSize(header.reconciled, version) + + inetAddressAndPortSerializer.serializedSize(header.sender, version) + + TimeUUID.sizeInBytes() + + TypeSizes.sizeof(header.sessionIndex) + + TypeSizes.sizeof(header.sendByFollower); + } + } + + public static final Serializer serializer = new Serializer(); + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) return false; + LogStreamHeader that = (LogStreamHeader) o; + return sessionIndex == that.sessionIndex && + sendByFollower == that.sendByFollower && + Objects.equals(manifest, that.manifest) && + Objects.equals(sender, that.sender) && + Objects.equals(planId, that.planId) && + Objects.equals(reconciled, that.reconciled); + } + + @Override + public int hashCode() + { + return Objects.hash(manifest, sender, planId, sessionIndex, sendByFollower, reconciled); + } + + @Override + public String toString() + { + return String.format("LogStreamHeader{manifest=%s, sender=%s, planId=%s, sessionIndex=%d, sendByFollower=%s, reconciledOffsets=%s}", + manifest, sender, planId, sessionIndex, sendByFollower, reconciled); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/streaming/LogStreamManifest.java b/src/java/org/apache/cassandra/streaming/LogStreamManifest.java new file mode 100644 index 000000000000..3c1e53e912f7 --- /dev/null +++ b/src/java/org/apache/cassandra/streaming/LogStreamManifest.java @@ -0,0 +1,168 @@ +/* + * 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.streaming; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; + +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; + +import static org.apache.cassandra.utils.CollectionSerializers.deserializeCollectionToConsumer; +import static org.apache.cassandra.utils.CollectionSerializers.deserializeMapToConsumer; +import static org.apache.cassandra.utils.CollectionSerializers.serializeCollection; +import static org.apache.cassandra.utils.CollectionSerializers.serializeMap; +import static org.apache.cassandra.utils.CollectionSerializers.serializedCollectionSize; +import static org.apache.cassandra.utils.CollectionSerializers.serializedMapSize; + +public class LogStreamManifest +{ + public final ImmutableMap>> keyspaceRanges; + + public LogStreamManifest(ImmutableMap>> keyspaceRanges) + { + this.keyspaceRanges = keyspaceRanges; + } + + public static LogStreamManifest create(Map>> keyspaceRanges) + { + ImmutableMap.Builder>> builder = ImmutableMap.builder(); + keyspaceRanges.forEach((keyspace, ranges) -> builder.put(keyspace, ImmutableSet.copyOf(ranges))); + return new LogStreamManifest(builder.build()); + } + + public static class Serializer + { + private static final IVersionedSerializer strSerializer = new IVersionedSerializer<>() + { + @Override + public void serialize(String str, DataOutputPlus out, int version) throws IOException + { + out.writeUTF(str); + } + + @Override + public String deserialize(DataInputPlus in, int version) throws IOException + { + return in.readUTF(); + } + + @Override + public long serializedSize(String str, int version) + { + return TypeSizes.sizeof(str); + } + }; + + private static final IVersionedSerializer> rangeSerializer = new IVersionedSerializer>() + { + @Override + public void serialize(Range range, DataOutputPlus out, int version) throws IOException + { + Token.serializer.serialize(range.left, out, version); + Token.serializer.serialize(range.right, out, version); + } + + @Override + public Range deserialize(DataInputPlus in, int version) throws IOException + { + return new Range<>(Token.serializer.deserialize(in, version), Token.serializer.deserialize(in, version)); + } + + @Override + public long serializedSize(Range range, int version) + { + return Token.serializer.serializedSize(range.left, version) + + Token.serializer.serializedSize(range.right, version); + } + }; + + private static final IVersionedSerializer>> rangeSetSerializer = new IVersionedSerializer<>() + { + @Override + public void serialize(ImmutableSet> t, DataOutputPlus out, int version) throws IOException + { + serializeCollection(t, out, version, rangeSerializer); + } + + @Override + public ImmutableSet> deserialize(DataInputPlus in, int version) throws IOException + { + ImmutableSet.Builder> builder = ImmutableSet.builder(); + deserializeCollectionToConsumer(in, version, rangeSerializer, builder::add); + return builder.build(); + } + + @Override + public long serializedSize(ImmutableSet> t, int version) + { + return serializedCollectionSize(t, version, rangeSerializer); + } + }; + + + public void serialize(LogStreamManifest header, DataOutputPlus out, int version) throws IOException + { + serializeMap(header.keyspaceRanges, out, version, strSerializer, rangeSetSerializer); + } + + public LogStreamManifest deserialize(DataInputPlus in, int version) throws IOException + { + ImmutableMap.Builder>> builder = ImmutableMap.builder(); + deserializeMapToConsumer(in, version, strSerializer, rangeSetSerializer, builder::put); + return new LogStreamManifest(builder.build()); + } + + public long serializedSize(LogStreamManifest header, int version) + { + return serializedMapSize(header.keyspaceRanges, version, strSerializer, rangeSetSerializer); + } + } + + public static final Serializer serializer = new Serializer(); + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) return false; + LogStreamManifest that = (LogStreamManifest) o; + return Objects.equals(keyspaceRanges, that.keyspaceRanges); + } + + @Override + public int hashCode() + { + return Objects.hashCode(keyspaceRanges); + } + + @Override + public String toString() + { + return String.format("MutationLogStreamHeader{keyspaceRanges=%s}", keyspaceRanges); + } +} diff --git a/src/java/org/apache/cassandra/streaming/LogStreamTask.java b/src/java/org/apache/cassandra/streaming/LogStreamTask.java new file mode 100644 index 000000000000..342d3dd334d5 --- /dev/null +++ b/src/java/org/apache/cassandra/streaming/LogStreamTask.java @@ -0,0 +1,82 @@ +/* + * 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.streaming; + +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.RangesAtEndpoint; + +/** + * Base class for log streaming tasks that track mutation log transfers and receives. + */ +public abstract class LogStreamTask +{ + protected final StreamSession session; + protected final InetAddressAndPort peer; + protected boolean completed = false; + + private final Map>> keyspaceRanges = new HashMap<>(); + + public LogStreamTask(StreamSession session, InetAddressAndPort peer) + { + this.session = session; + this.peer = peer; + } + + public synchronized void addKeyspaceRanges(String keyspace, RangesAtEndpoint ranges) + { + addKeyspaceRanges(keyspace, ranges.ranges()); + } + + public synchronized void addKeyspaceRanges(String keyspace, Collection> ranges) + { + keyspaceRanges.computeIfAbsent(keyspace, k -> new HashSet<>()).addAll(ranges); + } + + public abstract void abort(); + + public boolean isCompleted() + { + return completed; + } + + public LogStreamManifest getManifest() + { + return LogStreamManifest.create(keyspaceRanges); + } + + protected boolean markCompleted() + { + if (completed) + return false; + completed = true; + return true; + } + + public InetAddressAndPort getPeer() + { + return peer; + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/streaming/LogTransferTask.java b/src/java/org/apache/cassandra/streaming/LogTransferTask.java new file mode 100644 index 000000000000..4075b9b1a0ec --- /dev/null +++ b/src/java/org/apache/cassandra/streaming/LogTransferTask.java @@ -0,0 +1,113 @@ +/* + * 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.streaming; + +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.concurrent.ScheduledExecutors; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.replication.MutationJournal; +import org.apache.cassandra.replication.ReconciledKeyspaceOffsets; +import org.apache.cassandra.replication.ReconciledLogSnapshot; +import org.apache.cassandra.streaming.messages.OutgoingMutationLogStreamMessage; +import org.apache.cassandra.utils.FBUtilities; + +/** + * Task for tracking sending of mutation log streams. + */ +public class LogTransferTask extends LogStreamTask +{ + private static final Logger logger = LoggerFactory.getLogger(LogTransferTask.class); + + private volatile ScheduledFuture timeoutFuture; + private final ReconciledLogSnapshot reconciled; + private final MutationJournal.Snapshot snapshot; + + public LogTransferTask(StreamSession session, InetAddressAndPort peer, ReconciledLogSnapshot reconciled, MutationJournal.Snapshot snapshot) + { + super(session, peer); + this.reconciled = reconciled; + this.snapshot = snapshot; + } + + public ReconciledKeyspaceOffsets reconciled(String keyspace, Collection> ranges) + { + ReconciledLogSnapshot subset = reconciled.select(Collections.singletonMap(keyspace, ranges)); + return subset.getKeyspace(keyspace); + } + + public OutgoingMutationLogStreamMessage getMessage(StreamSession session) + { + LogStreamManifest manifest = getManifest(); + + ReconciledLogSnapshot subset = reconciled.select(manifest.keyspaceRanges); + + LogStreamHeader header = new LogStreamHeader(manifest, + subset, + FBUtilities.getBroadcastAddressAndPort(), + session.planId(), + 0, + session.isFollower()); + logger.trace("[Stream #{}] Creating outgoing mutation log stream message for peer {}", session.planId(), peer); + return new OutgoingMutationLogStreamMessage(header, snapshot); + } + + private synchronized void cancelTimeout() + { + if (timeoutFuture != null) + { + timeoutFuture.cancel(false); + timeoutFuture = null; + } + } + + public synchronized void complete() + { + // Cancel timeout on successful completion + cancelTimeout(); + // TODO: validate message header with expected ranges + logger.trace("[Stream #{}] Log transfer task completed for peer {}", session.planId(), peer); + if (markCompleted()) + session.taskCompleted(this); + } + + public void scheduleTimeout() + { + timeoutFuture = ScheduledExecutors.nonPeriodicTasks.schedule(session::sessionTimeout, DatabaseDescriptor.getStreamTransferTaskTimeout().toMilliseconds(), TimeUnit.MILLISECONDS); + } + + public void timeout() + { + session.sessionTimeout(); + } + + @Override + public void abort() + { + cancelTimeout(); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/streaming/StreamPlan.java b/src/java/org/apache/cassandra/streaming/StreamPlan.java index 93b864d79b07..1f900032f0ec 100644 --- a/src/java/org/apache/cassandra/streaming/StreamPlan.java +++ b/src/java/org/apache/cassandra/streaming/StreamPlan.java @@ -24,8 +24,10 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.RangesAtEndpoint; import org.apache.cassandra.locator.Replica; +import org.apache.cassandra.replication.ReconciledKeyspaceOffsets; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.TimeUUID; import static com.google.common.collect.Iterables.all; @@ -112,7 +114,14 @@ public StreamPlan requestRanges(InetAddressAndPort from, String keyspace, Ranges assert all(transientRanges, Replica::isSelf) || RangesAtEndpoint.isDummyList(transientRanges) : transientRanges.toString(); StreamSession session = coordinator.getOrCreateOutboundSession(from); + // TODO: add flag for fully reconciled data only if this is for a tracked keyspace session.addStreamRequest(keyspace, fullRanges, transientRanges, Arrays.asList(columnFamilies)); + + // Automatically include mutation logs for tracked keyspaces + if (isTrackedReplicationEnabled(keyspace)) { + session.addMutationLogRequest(keyspace, fullRanges, transientRanges); + } + return this; } @@ -128,7 +137,14 @@ public StreamPlan requestRanges(InetAddressAndPort from, String keyspace, Ranges public StreamPlan transferRanges(InetAddressAndPort to, String keyspace, RangesAtEndpoint replicas, String... columnFamilies) { StreamSession session = coordinator.getOrCreateOutboundSession(to); - session.addTransferRanges(keyspace, replicas, Arrays.asList(columnFamilies), flushBeforeTransfer); + + // Automatically include mutation logs for tracked keyspaces + ReconciledKeyspaceOffsets reconciledKeyspaceOffsets = isTrackedReplicationEnabled(keyspace) + ? session.addMutationLogTransfer(keyspace, replicas) + : null; + + session.addTransferRanges(keyspace, replicas, Arrays.asList(columnFamilies), flushBeforeTransfer, reconciledKeyspaceOffsets); + return this; } @@ -254,4 +270,15 @@ public static boolean hasAccordTables(KeyspaceMetadata ksm) { return ksm.tables.stream().anyMatch(TableMetadata::isAccordEnabled); } + + /** + * Check if the given keyspace uses tracked replication, which requires mutation log streaming. + * + * @param keyspace the keyspace name + * @return true if the keyspace uses tracked replication + */ + private boolean isTrackedReplicationEnabled(String keyspace) + { + return ClusterMetadata.current().schema.getKeyspaceMetadata(keyspace).useMutationTracking(); + } } diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java index 447621c78c9f..e93dec699f2b 100644 --- a/src/java/org/apache/cassandra/streaming/StreamSession.java +++ b/src/java/org/apache/cassandra/streaming/StreamSession.java @@ -39,6 +39,7 @@ import javax.annotation.Nullable; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -66,12 +67,19 @@ import org.apache.cassandra.locator.RangesAtEndpoint; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.metrics.StreamingMetrics; +import org.apache.cassandra.replication.MutationJournal; +import org.apache.cassandra.replication.MutationTrackingService; +import org.apache.cassandra.replication.ReconciledLogSnapshot; +import org.apache.cassandra.replication.ReconciledKeyspaceOffsets; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.async.StreamingMultiplexedChannel; import org.apache.cassandra.streaming.messages.CompleteMessage; +import org.apache.cassandra.streaming.messages.IncomingMutationLogStreamMessage; import org.apache.cassandra.streaming.messages.IncomingStreamMessage; +import org.apache.cassandra.streaming.messages.MutationLogReceivedMessage; +import org.apache.cassandra.streaming.messages.OutgoingMutationLogStreamMessage; import org.apache.cassandra.streaming.messages.OutgoingStreamMessage; import org.apache.cassandra.streaming.messages.PrepareAckMessage; import org.apache.cassandra.streaming.messages.PrepareSynAckMessage; @@ -199,6 +207,9 @@ public enum PrepareDirection { SEND, ACK } protected final ConcurrentHashMap transfers = new ConcurrentHashMap<>(); // data receivers, filled after receiving prepare message private final Map receivers = new ConcurrentHashMap<>(); + // log streaming tasks + private LogTransferTask logTransfer = null; + private LogReceiveTask logReceive = null; private final StreamingMetrics metrics; final Map>> transferredRangesPerKeyspace = new HashMap<>(); @@ -442,6 +453,30 @@ public void addStreamRequest(String keyspace, RangesAtEndpoint fullRanges, Range requests.add(new StreamRequest(keyspace, fullRanges, transientRanges, columnFamilies)); } + /** + * Request mutation log data from this session. + * + * @param keyspace Requesting keyspace + * @param fullRanges Ranges to retrieve mutation logs for + * @param transientRanges Ranges to retrieve mutation logs for + */ + public synchronized void addMutationLogRequest(String keyspace, RangesAtEndpoint fullRanges, RangesAtEndpoint transientRanges) + { + //It should either be a dummy address for repair or if it's a bootstrap/move/rebuild it should be this node + assert all(fullRanges, Replica::isSelf) || RangesAtEndpoint.isDummyList(fullRanges) : fullRanges.toString(); + assert all(transientRanges, Replica::isSelf) || RangesAtEndpoint.isDummyList(transientRanges) : transientRanges.toString(); + + // Create log receive task for the combined ranges + RangesAtEndpoint allRanges = RangesAtEndpoint.concat(fullRanges, transientRanges); + + if (logReceive == null) + { + logReceive = new LogReceiveTask(this, peer); + logger.trace("[Stream #{}] Created log receive task for peer {}", planId(), peer); + } + logReceive.addKeyspaceRanges(keyspace, allRanges); + } + /** * Set up transfer for specific keyspace/ranges/CFs * @@ -450,7 +485,7 @@ public void addStreamRequest(String keyspace, RangesAtEndpoint fullRanges, Range * @param columnFamilies Transfer ColumnFamilies * @param flushTables flush tables? */ - synchronized void addTransferRanges(String keyspace, RangesAtEndpoint replicas, Collection columnFamilies, boolean flushTables) + synchronized void addTransferRanges(String keyspace, RangesAtEndpoint replicas, Collection columnFamilies, boolean flushTables, ReconciledKeyspaceOffsets reconciledKeyspaceOffsets) { failIfFinished(); Collection stores = getColumnFamilyStores(keyspace, columnFamilies); @@ -461,7 +496,7 @@ synchronized void addTransferRanges(String keyspace, RangesAtEndpoint replicas, //Do we need to unwrap here also or is that just making it worse? //Range and if it's transient RangesAtEndpoint unwrappedRanges = replicas.unwrap(); - List streams = getOutgoingStreamsForRanges(unwrappedRanges, stores, pendingRepair, previewKind); + List streams = getOutgoingStreamsForRanges(unwrappedRanges, stores, pendingRepair, previewKind, reconciledKeyspaceOffsets); addTransferStreams(streams); Set> toBeUpdated = transferredRangesPerKeyspace.get(keyspace); @@ -473,6 +508,46 @@ synchronized void addTransferRanges(String keyspace, RangesAtEndpoint replicas, transferredRangesPerKeyspace.put(keyspace, toBeUpdated); } + private LogTransferTask createLogTransferTask() + { + ReconciledLogSnapshot reconciled = MutationTrackingService.instance.snapshotReconciledLogs(); + + // TODO: consider tradeoffs of eagerly reading the index of each segment and filtering out the ones that + // only contain fully reconciled ids vs just filtering out fully reconciled when reading out of the + // snapshot for streaming + MutationJournal.Snapshot snapshot = MutationJournal.instance.snapshot(); + try + { + // TODO: grab references to all current segments and the relevant reconciled sets + // TODO: Journal has a select and reference method we could use + LogTransferTask task = new LogTransferTask(this, peer, reconciled, snapshot); + logger.trace("[Stream #{}] Created log transfer task for peer {}", planId(), peer); + return task; + } + catch (Throwable t) + { + snapshot.close(); + throw t; + } + } + /** + * Set up mutation log transfer for specific keyspace and ranges. + * + * @param keyspace Transfer keyspace + * @param replicas Transfer ranges + */ + synchronized ReconciledKeyspaceOffsets addMutationLogTransfer(String keyspace, RangesAtEndpoint replicas) + { + failIfFinished(); + + if (logTransfer == null) + logTransfer = createLogTransferTask(); + + Collection> ranges = replicas.ranges(); + logTransfer.addKeyspaceRanges(keyspace, ranges); + return logTransfer.reconciled(keyspace, ranges); + } + private void failIfFinished() { if (state().isFinalState()) @@ -496,14 +571,14 @@ private Collection getColumnFamilyStores(String keyspace, Col } @VisibleForTesting - public List getOutgoingStreamsForRanges(RangesAtEndpoint replicas, Collection stores, TimeUUID pendingRepair, PreviewKind previewKind) + public List getOutgoingStreamsForRanges(RangesAtEndpoint replicas, Collection stores, TimeUUID pendingRepair, PreviewKind previewKind, ReconciledKeyspaceOffsets reconciledKeyspaceOffsets) { List streams = new ArrayList<>(); try { for (ColumnFamilyStore cfs: stores) { - streams.addAll(cfs.getStreamManager().createOutgoingStreams(this, replicas, pendingRepair, previewKind)); + streams.addAll(cfs.getStreamManager().createOutgoingStreams(this, replicas, pendingRepair, previewKind, reconciledKeyspaceOffsets)); } } catch (Throwable t) @@ -587,6 +662,8 @@ private void abortTasks() { receivers.values().forEach(StreamReceiveTask::abort); transfers.values().forEach(StreamTransferTask::abort); + if (logReceive != null) logReceive.abort(); + if (logTransfer != null) logTransfer.abort(); } catch (Exception e) { @@ -656,7 +733,7 @@ public synchronized void messageReceived(StreamMessage message) case PREPARE_SYN: // at follower PrepareSynMessage msg = (PrepareSynMessage) message; - prepare(msg.requests, msg.summaries); + prepare(msg.requests, msg.summaries, msg.logRequest, msg.logSummary); break; case PREPARE_SYNACK: // at initiator @@ -683,6 +760,12 @@ public synchronized void messageReceived(StreamMessage message) case SESSION_FAILED: sessionFailed(); break; + case MUTATION_LOG_STREAM: + receiveMutationLog((IncomingMutationLogStreamMessage) message); + break; + case MUTATION_LOG_RECEIVED: + mutationLogReceived((MutationLogReceivedMessage) message); + break; default: throw new AssertionError("unhandled StreamMessage type: " + message.getClass().getName()); } @@ -702,6 +785,9 @@ public void onInitializationComplete() prepare.summaries.add(task.getSummary()); } + prepare.logRequest = logReceive != null ? logReceive.getManifest() : null; + prepare.logSummary = logTransfer != null ? logTransfer.getManifest() : null; + sendControlMessage(prepare).syncUninterruptibly(); } @@ -777,14 +863,15 @@ private void logError(Throwable e) * * @return the prepare future for testing */ - public Future prepare(Collection requests, Collection summaries) + public Future prepare(Collection requests, Collection summaries, + LogStreamManifest logRequest, LogStreamManifest logSummary) { // prepare tasks state(State.PREPARING); return ScheduledExecutors.nonPeriodicTasks.submit(() -> { try { - prepareAsync(requests, summaries); + prepareAsync(requests, summaries, logRequest, logSummary); return null; } catch (Exception e) @@ -805,7 +892,8 @@ public void countStreamedIn(boolean isEntireSSTable) * so the logic should not execute on the main IO thread (read: netty event loop). */ @VisibleForTesting - void prepareAsync(Collection requests, Collection summaries) + void prepareAsync(Collection requests, Collection summaries, + LogStreamManifest logRequest, LogStreamManifest logSummary) { if (StreamOperation.REPAIR == streamOperation()) checkAvailableDiskSpaceAndCompactions(summaries); @@ -813,11 +901,20 @@ void prepareAsync(Collection requests, Collection for (StreamSummary summary : summaries) prepareReceiving(summary); + // Process mutation log manifests + if (logRequest != null) + prepareLogTransferring(logRequest); + if (logSummary != null) + prepareLogReceiving(logSummary); + PrepareSynAckMessage prepareSynAck = new PrepareSynAckMessage(); if (!peer.equals(FBUtilities.getBroadcastAddressAndPort())) for (StreamTransferTask task : transfers.values()) prepareSynAck.summaries.add(task.getSummary()); + // Include mutation log summary if we have log transfer task + prepareSynAck.logSummary = logTransfer != null ? logTransfer.getManifest() : null; + streamResult.handleSessionPrepared(this, PrepareDirection.SEND); // After sending the message the initiator can close the channel which will cause a ClosedChannelException // in buffer logic, this then gets sent to onError which validates the state isFinalState, if not fails @@ -848,6 +945,10 @@ private void prepareSynAck(PrepareSynAckMessage msg) sendControlMessage(new PrepareAckMessage()).syncUninterruptibly(); } + // Process mutation log summary if present + if (msg.logSummary != null) + prepareLogReceiving(msg.logSummary); + if (isPreview()) completePreview(); else @@ -875,22 +976,50 @@ private void processStreamRequests(Collection requests) requests.forEach(r -> requestsByKeyspace.put(r.keyspace, r)); requestsByKeyspace.asMap().forEach((ks, reqs) -> - { - OwnedRanges ownedRanges = StorageService.instance.getNormalizedLocalRanges(ks, getBroadcastAddressAndPort()); - - reqs.forEach(req -> - { - RangesAtEndpoint allRangesAtEndpoint = RangesAtEndpoint.concat(req.full, req.transientReplicas); - if (ownedRanges.validateRangeRequest(allRangesAtEndpoint.ranges(), "Stream #" + planId(), "stream request", peer)) - addTransferRanges(req.keyspace, allRangesAtEndpoint, req.columnFamilies, true); // always flush on stream request - else - rejectedRequests.add(req); - }); - }); + { + OwnedRanges ownedRanges = StorageService.instance.getNormalizedLocalRanges(ks, getBroadcastAddressAndPort()); + + reqs.forEach(req -> + { + RangesAtEndpoint allRangesAtEndpoint = RangesAtEndpoint.concat(req.full, req.transientReplicas); + if (ownedRanges.validateRangeRequest(allRangesAtEndpoint.ranges(), "Stream #" + planId(), "stream request", peer)) + { + + ReconciledKeyspaceOffsets reconciledKeyspaceOffsets = null; + if (logTransfer != null) + reconciledKeyspaceOffsets = logTransfer.reconciled(req.keyspace, allRangesAtEndpoint.ranges()); + addTransferRanges(req.keyspace, allRangesAtEndpoint, req.columnFamilies, true, reconciledKeyspaceOffsets); // always flush on stream request + } + else + rejectedRequests.add(req); + }); + }); if (!rejectedRequests.isEmpty()) throw new StreamRequestOutOfTokenRangeException(rejectedRequests); } + + private void prepareLogReceiving(LogStreamManifest manifest) + { + // Create log receive task based on manifest + if (logReceive == null) + logReceive = new LogReceiveTask(this, peer); + + // Add keyspace ranges from manifest + manifest.keyspaceRanges.forEach((keyspace, ranges) -> + logReceive.addKeyspaceRanges(keyspace, ranges)); + } + + private void prepareLogTransferring(LogStreamManifest manifest) + { + // Create log transfer task based on manifest + if (logTransfer == null) + logTransfer = createLogTransferTask(); + + // Add keyspace ranges from manifest + manifest.keyspaceRanges.forEach((keyspace, ranges) -> + logTransfer.addKeyspaceRanges(keyspace, ranges)); + } /** * In the case where we have an error checking disk space we allow the Operation to continue. * In the case where we do _not_ have available space, this method raises a RTE. @@ -1073,6 +1202,17 @@ public void streamSent(OutgoingStreamMessage message) } } + /** + * Call back after sending OutgoingMutationLogStreamMessage. + * + * @param message sent mutation log stream message + */ + public void logStreamSent(OutgoingMutationLogStreamMessage message) + { + if (logTransfer != null) + logTransfer.scheduleTimeout(); + } + /** * Call back after receiving a stream. * @@ -1114,6 +1254,26 @@ public void receive(IncomingStreamMessage message) } } + /** + * Call back after receiving a mutation log stream. + * + * @param message received mutation log stream + */ + public void receiveMutationLog(IncomingMutationLogStreamMessage message) + { + if (isPreview()) + { + throw new RuntimeException(String.format("[Stream #%s] Cannot receive mutation log stream for preview session", planId())); + } + + logger.debug("[Stream #{}] Received {}", planId(), message); + // Mutations are already applied during deserialization + + // Create and track the log receive task, then let it handle the message + if (logReceive != null) + logReceive.received(message); + } + public void progress(String filename, ProgressInfo.Direction direction, long bytes, long delta, long total) { if (delta < 0) @@ -1129,6 +1289,12 @@ public void received(TableId tableId, int sequenceNumber) transfers.get(tableId).complete(sequenceNumber); } + public void mutationLogReceived(MutationLogReceivedMessage message) + { + if (logTransfer != null) + logTransfer.complete(); + } + /** * Check if session is completed on receiving {@code StreamMessage.Type.COMPLETE} message. */ @@ -1152,7 +1318,16 @@ public synchronized void complete() */ private synchronized boolean maybeCompleted() { - if (!(receivers.isEmpty() && transfers.isEmpty())) + if (!receivers.isEmpty()) + return false; + + if (!transfers.isEmpty()) + return false; + + if (logReceive != null && !logReceive.isCompleted()) + return false; + + if (logTransfer != null && !logTransfer.isCompleted()) return false; // if already executed once, skip it @@ -1235,6 +1410,22 @@ public synchronized void taskCompleted(StreamTransferTask completedTask) maybeCompleted(); } + public synchronized void taskCompleted(LogReceiveTask completedTask) + { + Preconditions.checkState(logReceive == completedTask); + logger.trace("[Stream #{}] Log receive task completed, clearing reference", planId()); + logReceive = null; + maybeCompleted(); + } + + public synchronized void taskCompleted(LogTransferTask completedTask) + { + Preconditions.checkState(logTransfer == completedTask); + logger.trace("[Stream #{}] Log transfer task completed, clearing reference", planId()); + logTransfer = null; + maybeCompleted(); + } + private void completePreview() { try @@ -1249,6 +1440,9 @@ private void completePreview() // expected streaming, but don't leak any resources held by the task for (StreamTask task : Iterables.concat(receivers.values(), transfers.values())) task.abort(); + + if (logReceive != null) logReceive.abort(); + if (logTransfer != null) logTransfer.abort(); } } @@ -1279,6 +1473,8 @@ private void startStreamingFiles(@Nullable PrepareDirection prepareDirection) state(State.STREAMING); + startLogStreaming(); + for (StreamTransferTask task : transfers.values()) { Collection messages = task.getFileMessages(); @@ -1300,6 +1496,14 @@ private void startStreamingFiles(@Nullable PrepareDirection prepareDirection) maybeCompleted(); } + private void startLogStreaming() + { + if (logTransfer != null) + { + sendControlMessage(logTransfer.getMessage(this)); + } + } + @VisibleForTesting public int getNumRequests() { diff --git a/src/java/org/apache/cassandra/streaming/TableStreamManager.java b/src/java/org/apache/cassandra/streaming/TableStreamManager.java index d19064c9577e..e4145415d8aa 100644 --- a/src/java/org/apache/cassandra/streaming/TableStreamManager.java +++ b/src/java/org/apache/cassandra/streaming/TableStreamManager.java @@ -24,6 +24,7 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.locator.RangesAtEndpoint; +import org.apache.cassandra.replication.ReconciledKeyspaceOffsets; import org.apache.cassandra.streaming.messages.StreamMessageHeader; import org.apache.cassandra.utils.TimeUUID; @@ -55,5 +56,6 @@ public interface TableStreamManager Collection createOutgoingStreams(StreamSession session, RangesAtEndpoint replicas, TimeUUID pendingRepair, - PreviewKind previewKind); + PreviewKind previewKind, + ReconciledKeyspaceOffsets reconciledKeyspaceOffsets); } diff --git a/src/java/org/apache/cassandra/streaming/async/StreamingMultiplexedChannel.java b/src/java/org/apache/cassandra/streaming/async/StreamingMultiplexedChannel.java index 751928c19096..90c50fe9cc18 100644 --- a/src/java/org/apache/cassandra/streaming/async/StreamingMultiplexedChannel.java +++ b/src/java/org/apache/cassandra/streaming/async/StreamingMultiplexedChannel.java @@ -47,6 +47,7 @@ import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.streaming.messages.IncomingStreamMessage; import org.apache.cassandra.streaming.messages.KeepAliveMessage; +import org.apache.cassandra.streaming.messages.OutgoingMutationLogStreamMessage; import org.apache.cassandra.streaming.messages.OutgoingStreamMessage; import org.apache.cassandra.streaming.messages.StreamMessage; import org.apache.cassandra.utils.concurrent.ImmediateFuture; @@ -212,7 +213,7 @@ public Future sendMessage(StreamingChannel channel, StreamMessage message) if (closed) throw new RuntimeException("stream has been closed, cannot send " + message); - if (message instanceof OutgoingStreamMessage) + if (message instanceof OutgoingStreamMessage || message instanceof OutgoingMutationLogStreamMessage) { if (session.isPreview()) throw new RuntimeException("Cannot send stream data messages for preview streaming sessions"); @@ -220,7 +221,10 @@ public Future sendMessage(StreamingChannel channel, StreamMessage message) logger.debug("{} Sending {}", createLogTag(session), message); InetAddressAndPort connectTo = factory.supportsPreferredIp() ? SystemKeyspace.getPreferredIP(to) : to; - return fileTransferExecutor.submit(new FileStreamTask((OutgoingStreamMessage) message, connectTo)); + FileStreamTask task = message instanceof OutgoingStreamMessage + ? new FileStreamTask((OutgoingStreamMessage) message, connectTo) + : new FileStreamTask((OutgoingMutationLogStreamMessage) message, connectTo); + return fileTransferExecutor.submit(task); } try @@ -288,7 +292,13 @@ class FileStreamTask implements Runnable private final InetAddressAndPort connectTo; - FileStreamTask(OutgoingStreamMessage ofm, InetAddressAndPort connectTo) + private FileStreamTask(OutgoingStreamMessage ofm, InetAddressAndPort connectTo) + { + this.msg = ofm; + this.connectTo = connectTo; + } + + FileStreamTask(OutgoingMutationLogStreamMessage ofm, InetAddressAndPort connectTo) { this.msg = ofm; this.connectTo = connectTo; diff --git a/src/java/org/apache/cassandra/streaming/messages/IncomingMutationLogStreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/IncomingMutationLogStreamMessage.java new file mode 100644 index 000000000000..fea433419e89 --- /dev/null +++ b/src/java/org/apache/cassandra/streaming/messages/IncomingMutationLogStreamMessage.java @@ -0,0 +1,112 @@ +/* + * 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.streaming.messages; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.apache.cassandra.db.Mutation; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.replication.MutationTrackingService; +import org.apache.cassandra.streaming.LogStreamHeader; +import org.apache.cassandra.streaming.StreamManager; +import org.apache.cassandra.streaming.StreamReceiveException; +import org.apache.cassandra.streaming.StreamSession; +import org.apache.cassandra.streaming.StreamingChannel; +import org.apache.cassandra.streaming.StreamingDataOutputPlus; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Incoming mutation log stream message for receiving mutation logs during bootstrap. + * + * This message handles deserialization and processing of mutation logs received from + * nodes during the bootstrap process. + */ +public class IncomingMutationLogStreamMessage extends MutationLogStreamMessage +{ + private static final Logger logger = LoggerFactory.getLogger(IncomingMutationLogStreamMessage.class); + + public static final StreamMessage.Serializer serializer = new IncomingMutationLogStreamMessageSerializer(); + + public final StreamSession session; + + public IncomingMutationLogStreamMessage(LogStreamHeader header, StreamSession session) + { + super(header); + this.session = session; + } + + @Override + public StreamSession getOrCreateAndAttachInboundSession(StreamingChannel channel, int messagingVersion) + { + session.attachInbound(channel); + return session; + } + + public static class IncomingMutationLogStreamMessageSerializer implements StreamMessage.Serializer + { + public IncomingMutationLogStreamMessage deserialize(DataInputPlus in, int version) throws IOException + { + LogStreamHeader header = LogStreamHeader.serializer.deserialize(in, version); + StreamSession session = StreamManager.instance.findSession(header.sender, header.planId, header.sessionIndex, header.sendByFollower); + if (session == null) + throw new IllegalStateException(String.format("unknown stream session: %s - %d", header.planId, header.sessionIndex)); + + try + { + while (in.readBoolean()) + { + int userVersion = in.readInt(); + ByteBuffer buffer = ByteBufferUtil.readWithVIntLength(in); + Mutation mutation = Mutation.serializer.deserialize(buffer, userVersion); + + if (logger.isTraceEnabled()) + logger.trace("Received mutation {}: session={}, keyspace={}, token={}", + mutation.id(), + session.planId(), + mutation.getKeyspaceName(), + mutation.key().getToken()); + + mutation.apply(); + } + + MutationTrackingService.instance.recordFullyReconciledOffsets(header.reconciled); + + return new IncomingMutationLogStreamMessage(header, session); + } + catch (Throwable t) + { + if (t instanceof StreamReceiveException) + throw (StreamReceiveException) t; + throw new StreamReceiveException(session, t); + } + } + + public void serialize(IncomingMutationLogStreamMessage message, StreamingDataOutputPlus out, int version, StreamSession session) + { + throw new UnsupportedOperationException("Not allowed to call serialize on an incoming stream"); + } + + public long serializedSize(IncomingMutationLogStreamMessage message, int version) + { + throw new UnsupportedOperationException("Not allowed to call serializedSize on an incoming stream"); + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/streaming/messages/MutationLogReceivedMessage.java b/src/java/org/apache/cassandra/streaming/messages/MutationLogReceivedMessage.java new file mode 100644 index 000000000000..e3dbf45d5223 --- /dev/null +++ b/src/java/org/apache/cassandra/streaming/messages/MutationLogReceivedMessage.java @@ -0,0 +1,51 @@ +/* + * 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.streaming.messages; + +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.streaming.StreamSession; +import org.apache.cassandra.streaming.StreamingDataOutputPlus; + +public class MutationLogReceivedMessage extends StreamMessage +{ + public static Serializer serializer = new Serializer() + { + public MutationLogReceivedMessage deserialize(DataInputPlus in, int version) + { + return new MutationLogReceivedMessage(); + } + + public void serialize(MutationLogReceivedMessage message, StreamingDataOutputPlus out, int version, StreamSession session) {} + + public long serializedSize(MutationLogReceivedMessage message, int version) + { + return 0; + } + }; + + public MutationLogReceivedMessage() + { + super(Type.MUTATION_LOG_RECEIVED); + } + + @Override + public String toString() + { + return "MutationLogReceived"; + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/streaming/messages/MutationLogStreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/MutationLogStreamMessage.java new file mode 100644 index 000000000000..4f11fb8b7118 --- /dev/null +++ b/src/java/org/apache/cassandra/streaming/messages/MutationLogStreamMessage.java @@ -0,0 +1,37 @@ +/* + * 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.streaming.messages; + +import org.apache.cassandra.streaming.LogStreamHeader; + +public abstract class MutationLogStreamMessage extends StreamMessage +{ + public final LogStreamHeader header; + + protected MutationLogStreamMessage(LogStreamHeader header) + { + super(Type.MUTATION_LOG_STREAM); + this.header = header; + } + + @Override + public String toString() + { + return String.format("%s{header=%s}", getClass().getSimpleName(), header); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/streaming/messages/OutgoingMutationLogStreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/OutgoingMutationLogStreamMessage.java new file mode 100644 index 000000000000..263a2c22621d --- /dev/null +++ b/src/java/org/apache/cassandra/streaming/messages/OutgoingMutationLogStreamMessage.java @@ -0,0 +1,135 @@ +/* + * 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.streaming.messages; + +import java.io.IOException; +import java.util.Set; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.Mutation; +import org.apache.cassandra.db.rows.DeserializationHelper; +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.DataInputPlus; +import org.apache.cassandra.replication.MutationJournal; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.streaming.LogStreamHeader; +import org.apache.cassandra.streaming.StreamSession; +import org.apache.cassandra.streaming.StreamingDataOutputPlus; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class OutgoingMutationLogStreamMessage extends MutationLogStreamMessage +{ + private static final Logger logger = LoggerFactory.getLogger(OutgoingMutationLogStreamMessage.class); + + public static final StreamMessage.Serializer serializer = new OutgoingMutationLogStreamMessageSerializer(); + + private final MutationJournal.Snapshot snapshot; + + public OutgoingMutationLogStreamMessage(LogStreamHeader header, MutationJournal.Snapshot snapshot) + { + super(header); + this.snapshot = snapshot; + } + + public void serialize(StreamingDataOutputPlus out, int version, StreamSession session) throws IOException + { + LogStreamHeader.serializer.serialize(header, out, version); + + try + { + // Stream mutations using the journal readAll method and filter by keyspace and token ranges + snapshot.readAll((segment, position, key, buffer, userVersion) -> { + try (DataInputBuffer in = new DataInputBuffer(buffer, true)) + { + Pair keyAndTableMetadata = Mutation.serializer.deserializeKeyAndTableMetadata(in, userVersion, DeserializationHelper.Flag.LOCAL); + DecoratedKey dk = keyAndTableMetadata.left; + String keyspace = keyAndTableMetadata.right.keyspace; + + // don't send fully reconciled mutations + if (header.reconciled.isFullyReconciled(keyspace, key)) + return; + + // Check if the mutation's keyspace and token are in our ranges + Set> ranges = header.manifest.keyspaceRanges.get(keyspace); + + if (ranges == null) + { + if (logger.isTraceEnabled()) + logger.trace("Mutation {} not sent: keyspace {} not in manifest ranges for session {}", key, keyspace, session.planId()); + return; + } + + if (!Range.isInRanges(dk.getToken(), ranges)) + { + if (logger.isTraceEnabled()) + logger.trace("Mutation {} not sent: token {} not in ranges for keyspace {} in session {}", key, dk.getToken(), keyspace, session.planId()); + return; + } + + if (logger.isTraceEnabled()) + logger.trace("Sending mutation {}: keyspace={}, token={}, session={}", key, keyspace, dk.getToken(), session.planId()); + + out.writeBoolean(true); + out.writeInt(userVersion); + ByteBufferUtil.writeWithVIntLength(buffer, out); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }); + } + finally + { + snapshot.close(); + } + + // end-of-stream marker + out.writeBoolean(false); + + session.logStreamSent(this); + } + + public long serializedSize(int version) + { + return 0; + } + + public static class OutgoingMutationLogStreamMessageSerializer implements StreamMessage.Serializer + { + public OutgoingMutationLogStreamMessage deserialize(DataInputPlus in, int version) + { + throw new UnsupportedOperationException("Not allowed to call deserialize on an outgoing stream"); + } + + public void serialize(OutgoingMutationLogStreamMessage message, StreamingDataOutputPlus out, int version, StreamSession session) throws IOException + { + message.serialize(out, version, session); + } + + public long serializedSize(OutgoingMutationLogStreamMessage message, int version) + { + return message.serializedSize(version); + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/streaming/messages/PrepareSynAckMessage.java b/src/java/org/apache/cassandra/streaming/messages/PrepareSynAckMessage.java index e052f4c3017b..095b82e9cffa 100644 --- a/src/java/org/apache/cassandra/streaming/messages/PrepareSynAckMessage.java +++ b/src/java/org/apache/cassandra/streaming/messages/PrepareSynAckMessage.java @@ -23,6 +23,8 @@ import java.util.Collection; import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.streaming.LogStreamManifest; import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.streaming.StreamSummary; import org.apache.cassandra.streaming.StreamingDataOutputPlus; @@ -36,6 +38,13 @@ public void serialize(PrepareSynAckMessage message, StreamingDataOutputPlus out, out.writeInt(message.summaries.size()); for (StreamSummary summary : message.summaries) StreamSummary.serializer.serialize(summary, out, version); + // log summary (optional, added in version 52) + if (version >= MessagingService.VERSION_52) + { + out.writeBoolean(message.logSummary != null); + if (message.logSummary != null) + LogStreamManifest.serializer.serialize(message.logSummary, out, version); + } } public PrepareSynAckMessage deserialize(DataInputPlus input, int version) throws IOException @@ -44,14 +53,27 @@ public PrepareSynAckMessage deserialize(DataInputPlus input, int version) throws int numSummaries = input.readInt(); for (int i = 0; i < numSummaries; i++) message.summaries.add(StreamSummary.serializer.deserialize(input, version)); + // log summary (optional, added in version 52) + if (version >= MessagingService.VERSION_52) + { + if (input.readBoolean()) + message.logSummary = LogStreamManifest.serializer.deserialize(input, version); + } return message; } public long serializedSize(PrepareSynAckMessage message, int version) { - long size = 4; // count of requests and count of summaries + long size = 4; // count of summaries for (StreamSummary summary : message.summaries) size += StreamSummary.serializer.serializedSize(summary, version); + // log summary (optional, added in version 52) + if (version >= MessagingService.VERSION_52) + { + size += 1; // boolean for logSummary presence + if (message.logSummary != null) + size += LogStreamManifest.serializer.serializedSize(message.logSummary, version); + } return size; } }; @@ -61,6 +83,11 @@ public long serializedSize(PrepareSynAckMessage message, int version) */ public final Collection summaries = new ArrayList<>(); + /** + * Optional summary of log stream tx + */ + public LogStreamManifest logSummary = null; + public PrepareSynAckMessage() { super(Type.PREPARE_SYNACK); diff --git a/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java b/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java index c856f469838f..cbae5781fa2a 100644 --- a/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java +++ b/src/java/org/apache/cassandra/streaming/messages/PrepareSynMessage.java @@ -22,6 +22,8 @@ import java.util.Collection; import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.streaming.LogStreamManifest; import org.apache.cassandra.streaming.StreamRequest; import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.streaming.StreamSummary; @@ -42,6 +44,16 @@ public PrepareSynMessage deserialize(DataInputPlus input, int version) throws IO int numSummaries = input.readInt(); for (int i = 0; i < numSummaries; i++) message.summaries.add(StreamSummary.serializer.deserialize(input, version)); + + if (version >= MessagingService.VERSION_52) + { + if (input.readBoolean()) + message.logRequest = LogStreamManifest.serializer.deserialize(input, version); + + if (input.readBoolean()) + message.logSummary = LogStreamManifest.serializer.deserialize(input, version); + } + return message; } @@ -52,6 +64,16 @@ public long serializedSize(PrepareSynMessage message, int version) size += StreamRequest.serializer.serializedSize(request, version); for (StreamSummary summary : message.summaries) size += StreamSummary.serializer.serializedSize(summary, version); + // log request and summary (optional, added in version 52) + if (version >= MessagingService.VERSION_52) + { + size += 1; // boolean for logRequest presence + if (message.logRequest != null) + size += LogStreamManifest.serializer.serializedSize(message.logRequest, version); + size += 1; // boolean for logSummary presence + if (message.logSummary != null) + size += LogStreamManifest.serializer.serializedSize(message.logSummary, version); + } return size; } @@ -65,6 +87,16 @@ public void serialize(PrepareSynMessage message, StreamingDataOutputPlus out, in out.writeInt(message.summaries.size()); for (StreamSummary summary : message.summaries) StreamSummary.serializer.serialize(summary, out, version); + // log request and summary (optional, added in version 52) + if (version >= MessagingService.VERSION_52) + { + out.writeBoolean(message.logRequest != null); + if (message.logRequest != null) + LogStreamManifest.serializer.serialize(message.logRequest, out, version); + out.writeBoolean(message.logSummary != null); + if (message.logSummary != null) + LogStreamManifest.serializer.serialize(message.logSummary, out, version); + } } }; @@ -78,6 +110,16 @@ public void serialize(PrepareSynMessage message, StreamingDataOutputPlus out, in */ public final Collection summaries = new ArrayList<>(); + /** + * Optional request for log stream + */ + public LogStreamManifest logRequest = null; + + /** + * Optional summary of log stream tx + */ + public LogStreamManifest logSummary = null; + public PrepareSynMessage() { super(Type.PREPARE_SYN); diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java index 186ac3274abd..4bc494bef612 100644 --- a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java +++ b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java @@ -61,15 +61,17 @@ public static interface Serializer /** StreamMessage types */ public enum Type { - PREPARE_SYN (1, 5, PrepareSynMessage.serializer ), - STREAM (2, 0, IncomingStreamMessage.serializer, OutgoingStreamMessage.serializer), - RECEIVED (3, 4, ReceivedMessage.serializer ), - COMPLETE (5, 1, CompleteMessage.serializer ), - SESSION_FAILED (6, 5, SessionFailedMessage.serializer), - KEEP_ALIVE (7, 5, KeepAliveMessage.serializer ), - PREPARE_SYNACK (8, 5, PrepareSynAckMessage.serializer), - PREPARE_ACK (9, 5, PrepareAckMessage.serializer ), - STREAM_INIT (10, 5, StreamInitMessage.serializer ); + PREPARE_SYN (1, 5, PrepareSynMessage.serializer ), + STREAM (2, 0, IncomingStreamMessage.serializer, OutgoingStreamMessage.serializer), + RECEIVED (3, 4, ReceivedMessage.serializer ), + COMPLETE (5, 1, CompleteMessage.serializer ), + SESSION_FAILED (6, 5, SessionFailedMessage.serializer), + KEEP_ALIVE (7, 5, KeepAliveMessage.serializer ), + PREPARE_SYNACK (8, 5, PrepareSynAckMessage.serializer), + PREPARE_ACK (9, 5, PrepareAckMessage.serializer ), + STREAM_INIT (10, 5, StreamInitMessage.serializer ), + MUTATION_LOG_STREAM (11, 0, IncomingMutationLogStreamMessage.serializer, OutgoingMutationLogStreamMessage.serializer), + MUTATION_LOG_RECEIVED(12, 4, MutationLogReceivedMessage.serializer); private static final Map idToTypeMap; diff --git a/src/java/org/apache/cassandra/tcm/Startup.java b/src/java/org/apache/cassandra/tcm/Startup.java index 7fc336809c53..e4bb76e99839 100644 --- a/src/java/org/apache/cassandra/tcm/Startup.java +++ b/src/java/org/apache/cassandra/tcm/Startup.java @@ -162,6 +162,7 @@ public static void initializeAsNonCmsNode(Function wrapPro ClusterMetadataService::state, logSpec)); ClusterMetadataService.instance().log().ready(); + MutationTrackingService.instance.registerTCMListener(); NodeId nodeId = ClusterMetadata.current().myNodeId(); UUID currentHostId = SystemKeyspace.getLocalHostId(); @@ -279,6 +280,7 @@ public static void initializeFromGossip(Function wrapProce logSpec)); ClusterMetadataService.instance().log().ready(); + MutationTrackingService.instance.registerTCMListener(); initMessaging.run(); try { @@ -390,6 +392,7 @@ public static void reinitializeWithClusterMetadata(String fileName, Function> M deserializeMap(DataInputPlus in, Unv return result; } + public static void deserializeMapToConsumer(DataInputPlus in, int version, IVersionedSerializer keySerializer, IVersionedSerializer valueSerializer, BiConsumer consumer) throws IOException + { + int size = in.readUnsignedVInt32(); + while (size-- > 0) + { + K key = keySerializer.deserialize(in, version); + V value = valueSerializer.deserialize(in, version); + consumer.accept(key, value); + } + } + public static > M deserializeMap(DataInputPlus in, int version, IVersionedSerializer keySerializer, IVersionedSerializer valueSerializer, IntFunction factory) throws IOException { int size = in.readUnsignedVInt32(); @@ -540,6 +553,13 @@ private static > C deserializeCollection(Data return result; } + public static void deserializeCollectionToConsumer(DataInputPlus in, int version, IVersionedSerializer serializer, Consumer consumer) throws IOException + { + int size = in.readUnsignedVInt32(); + while (size-- > 0) + consumer.accept(serializer.deserialize(in, version)); + } + /* * Private to push auto-complete to the convenience methods * Feel free to make public if there is a weird collection you want to use diff --git a/test/distributed/org/apache/cassandra/distributed/test/hostreplacement/TrackedHostReplacementTest.java b/test/distributed/org/apache/cassandra/distributed/test/hostreplacement/TrackedHostReplacementTest.java new file mode 100644 index 000000000000..589738a0097f --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/hostreplacement/TrackedHostReplacementTest.java @@ -0,0 +1,375 @@ +/* + * 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.hostreplacement; + +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.collect.Lists; +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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.IInvokableInstance; +import org.apache.cassandra.distributed.api.TokenSupplier; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.replication.MutationJournal; +import org.apache.cassandra.replication.MutationSummary; +import org.apache.cassandra.replication.MutationTrackingService; +import org.apache.cassandra.tcm.ClusterMetadata; + +import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows; +import static org.apache.cassandra.distributed.shared.AssertUtils.row; +import static org.apache.cassandra.distributed.shared.ClusterUtils.awaitRingJoin; +import static org.apache.cassandra.distributed.shared.ClusterUtils.replaceHostAndStart; +import static org.apache.cassandra.distributed.shared.ClusterUtils.stopUnchecked; +import static org.apache.cassandra.distributed.test.tracking.MutationTrackingUtils.assertMatchingSummaryForTable; +import static org.apache.cassandra.distributed.test.tracking.MutationTrackingUtils.summaryForTable; +import static org.assertj.core.api.Assertions.assertThat; + +public class TrackedHostReplacementTest extends TestBaseImpl +{ + private static final Logger logger = LoggerFactory.getLogger(TrackedHostReplacementTest.class); + + private static final String KEYSPACE = "test_ks"; + private static final String TABLE = "test_table"; + private static final String QUALIFIED_TABLE_NAME = KEYSPACE + '.' + TABLE; + + private static void pauseLogBroadcasts(Cluster cluster, boolean pause) + { + cluster.stream() + .filter(node -> !node.isShutdown()) + .forEach(node -> node.runOnInstance(() -> { + MutationTrackingService.instance.pauseOffsetBroadcast(pause); + })); + } + + private static void awaitFullReconciliation(Cluster cluster, int ids) throws InterruptedException + { + // await full reconciliation + boolean fullyReconciled = false; + for (int i = 0; i < 20; i++) + { + int attempt = i + 1; + fullyReconciled = cluster.stream().filter(node -> !node.isShutdown()).allMatch(node -> { + MutationSummary summary = summaryForTable(node, KEYSPACE, TABLE); + if (summary.unreconciledIds() == 0) + { + Assert.assertEquals(node.toString(), ids, summary.reconciledIds()); + return true; + } + else + { + logger.info("Not yet fully reconciled (reconciled: {}, unreconciled:{}) - attempt {} summary: {}", summary.reconciledIds(), summary.unreconciledIds(), attempt, summary); + } + return false; + }); + + if (!fullyReconciled) + Thread.sleep(1000); + } + + Assert.assertTrue(fullyReconciled); + } + @Test + public void testBasicTrackedHostReplacement() throws Exception + { + TokenSupplier even = TokenSupplier.evenlyDistributedTokens(3); + try (Cluster cluster = init(Cluster.build(3) + .withConfig(config -> config.with(Feature.NETWORK, Feature.GOSSIP)) + .withTokenSupplier(node -> even.token(node == 4 ? 3 : node)) + .start())) + { + setupTrackedKeyspace(cluster); + + // write some initial data + writeDataRange(cluster, 0, 10); + awaitFullReconciliation(cluster, 10); + verifyLocalDataContents(cluster, 0, 10); + + IInvokableInstance victimNode = cluster.get(3); + Set victimTokens = getNodeTokens(victimNode); + + // Stop the victim node + stopUnchecked(victimNode); + + // Write more data while victim is down + writeDataRange(cluster, 10, 20); + + // Verify remaining nodes have the new data + verifyLocalDataContents(Arrays.asList(cluster.get(1), cluster.get(2)), 10, 20); + + pauseLogBroadcasts(cluster, true); + MutationSummary expectedSummary = summaryForTable(cluster.get(1), KEYSPACE, TABLE); + Assert.assertEquals(10, expectedSummary.reconciledIds()); + Assert.assertEquals(10, expectedSummary.unreconciledIds()); + + // Replace the node + IInvokableInstance replacementNode = replaceHostAndStart(cluster, victimNode); + + // Wait for replacement to complete + awaitRingJoin(cluster.get(1), replacementNode); + awaitRingJoin(cluster.get(2), replacementNode); + awaitRingJoin(replacementNode, cluster.get(1)); + awaitRingJoin(replacementNode, cluster.get(2)); + + // confirm replacement node took over victim's token ranges + Set replacementTokens = getNodeTokens(replacementNode); + assertThat(replacementTokens).as("Replacement node should have same tokens as victim") + .isEqualTo(victimTokens); + + assertMatchingSummaryForTable(replacementNode, KEYSPACE, TABLE, expectedSummary); + + List remainingNodes = List.of(cluster.get(1), cluster.get(2), replacementNode); + // Verify all nodes have all writes + verifyLocalDataContents(remainingNodes, 0, 20); + + + // unpause id broadcast. all nodes should now reach full reconciliation, even though the replica set has changed + pauseLogBroadcasts(cluster, false); + awaitFullReconciliation(cluster, 20); + + // Write new data and verify replacement node handles writes for its ranges + writeDataRange(cluster, 20, 25); + + awaitFullReconciliation(cluster, 25); + verifyLocalDataContents(remainingNodes, 20, 25); + } + } + + /** + * Test host replacement with writes to the cluster during replacement bootstrap. + */ + @Test + public void testTrackedHostReplacementWithOngoingWrites() throws Exception + { + TokenSupplier even = TokenSupplier.evenlyDistributedTokens(3); + try (Cluster cluster = init(Cluster.build(3) + .withConfig(config -> config.with(Feature.NETWORK, Feature.GOSSIP)) + .withTokenSupplier(node -> even.token(node == 4 ? 3 : node)) + .start())) + { + final int numInitialWrites = 10; + setupTrackedKeyspace(cluster); + + // Phase 1: Establish baseline mutation tracking state + writeDataRange(cluster, 0, numInitialWrites); + awaitFullReconciliation(cluster, numInitialWrites); + verifyLocalDataContents(cluster, 0, numInitialWrites); + + // Capture victim's exact mutation tracking state for streaming validation + IInvokableInstance victim = cluster.get(3); + Set victimTokens = getNodeTokens(victim); + + // Phase 2: Stop victim and start replacement bootstrap + AtomicBoolean replacementCompleted = new AtomicBoolean(false); + AtomicInteger totalWrites = new AtomicInteger(numInitialWrites); + + + Thread thread = new Thread(() -> { + while (!replacementCompleted.get()) + { + int key = totalWrites.getAndIncrement(); + writeDataToCluster(cluster.coordinator((key % 2) + 1), key, key * 10); + } + }); + thread.start(); + + stopUnchecked(victim); + IInvokableInstance replacementNode = replaceHostAndStart(cluster, victim); + + // Wait for replacement to complete + awaitRingJoin(cluster.get(1), replacementNode); + awaitRingJoin(cluster.get(2), replacementNode); + awaitRingJoin(replacementNode, cluster.get(1)); + awaitRingJoin(replacementNode, cluster.get(2)); + + // confirm replacement node took over victim's token ranges + Set replacementTokens = getNodeTokens(replacementNode); + assertThat(replacementTokens).as("Replacement node should have same tokens as victim") + .isEqualTo(victimTokens); + + // stop concurrent writes + replacementCompleted.set(true); + thread.join(); + + if (totalWrites.get() == numInitialWrites) + throw new AssertionError("No concurrent writes were performed during replacement"); + + logger.info("Total writes performed: {} ", totalWrites.get()); + + List remainingNodes = List.of(cluster.get(1), cluster.get(2), replacementNode); + + // wait for all nodes to reach full reconciliation and verify data + awaitFullReconciliation(cluster, totalWrites.get()); + verifyLocalDataContents(remainingNodes, 0, totalWrites.get()); + } + } + + @Test + public void testTrackedHostReplacementWithLargeDataSet() throws Exception + { + TokenSupplier even = TokenSupplier.evenlyDistributedTokens(3); + try (Cluster cluster = init(Cluster.build(3) + .withConfig(config -> config.with(Feature.NETWORK, Feature.GOSSIP)) + .withTokenSupplier(node -> even.token(node == 4 ? 3 : node)) + .start())) + { + setupTrackedKeyspace(cluster); + + // Phase 1: Create initial SSTable generation (Generation 1) + writeDataRange(cluster, 0, 30); + flushAllNodes(cluster); // Creates first SSTable generation + advanceMutationLogSegment(cluster); + + // Phase 2: Add data and create overlapping SSTable generation (Generation 2) + writeDataRange(cluster, 30, 60); + flushAllNodes(cluster); // Creates second SSTable generation with overlapping keys + advanceMutationLogSegment(cluster); + + // Phase 3: Create final SSTable generation with different key distribution (Generation 3) + writeDataRange(cluster, 60, 90); + flushAllNodes(cluster); // Creates third SSTable generation + advanceMutationLogSegment(cluster); + + // Capture victim mutation tracking baseline before replacement + IInvokableInstance nodeToReplace = cluster.get(3); + Set victimTokens = getNodeTokens(nodeToReplace); + + // Stop victim node and write additional data to create streaming complexity + nodeToReplace.shutdown().get(); + + // Write additional data while victim is down + writeDataRange(cluster, 90, 120); + // don't flush + + + // Replace the node + IInvokableInstance replacementNode = replaceHostAndStart(cluster, nodeToReplace); + + // Wait for replacement to complete + awaitRingJoin(cluster.get(1), replacementNode); + awaitRingJoin(cluster.get(2), replacementNode); + awaitRingJoin(replacementNode, cluster.get(1)); + awaitRingJoin(replacementNode, cluster.get(2)); + + // confirm replacement node took over victim's token ranges + Set replacementTokens = getNodeTokens(replacementNode); + assertThat(replacementTokens).as("Replacement node should have same tokens as victim") + .isEqualTo(victimTokens); + + List remainingNodes = List.of(cluster.get(1), cluster.get(2), replacementNode); + + // wait for all nodes to reach full reconciliation and verify data + awaitFullReconciliation(cluster, 120); + verifyLocalDataContents(remainingNodes, 0, 120); + } + } + + private void setupTrackedKeyspace(Cluster cluster) + { + cluster.schemaChange(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH REPLICATION={'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked'", KEYSPACE)); + cluster.schemaChange(String.format("CREATE TABLE IF NOT EXISTS %s (k int PRIMARY KEY, v int)", QUALIFIED_TABLE_NAME)); + } + + /** + * Write data using proper coordinator pattern, not direct node access. + * This ensures replication happens correctly. + */ + private void writeDataToCluster(Cluster cluster, int key, int value) + { + writeDataToCluster(cluster.coordinator(1), key, value); + } + + private void writeDataToCluster(ICoordinator coordinator, int key, int value) + { + // Use coordinator to write with QUORUM - this will replicate properly + coordinator.execute(String.format("INSERT INTO %s (k, v) VALUES (?, ?)", QUALIFIED_TABLE_NAME), ConsistencyLevel.QUORUM, key, value); + } + + /** + * Write multiple keys to establish baseline mutation tracking state + */ + private void writeDataRange(Cluster cluster, int startKey, int endKey) + { + for (int i = startKey; i < endKey; i++) + { + writeDataToCluster(cluster, i, i * 10); + } + } + + private void flushAllNodes(Cluster cluster) + { + for (int i = 1; i <= cluster.size(); i++) + { + cluster.get(i).flush(KEYSPACE); + } + } + + private void advanceMutationLogSegment(Cluster cluster) + { + cluster.stream().filter(node -> !node.isShutdown()).forEach( node -> { + node.runOnInstance(() -> { + MutationJournal.instance.advanceSegment(); + }); + }); + } + + private void verifyLocalDataContents(Cluster cluster, int startKey, int endKey) + { + // Inline simple iteration over cluster nodes instead of using getAllNodes() helper + List nodes = new java.util.ArrayList<>(); + for (int i = 1; i <= cluster.size(); i++) + { + nodes.add(cluster.get(i)); + } + verifyLocalDataContents(Lists.newArrayList(cluster), startKey, endKey); + } + + private void verifyLocalDataContents(List nodes, int startKey, int endKey) + { + for (int key = startKey; key < endKey; key++) + { + int expectedValue = key * 10; + for (IInvokableInstance node : nodes) + { + Object[][] result = node.executeInternal(String.format("SELECT k, v FROM %s WHERE k = ?", QUALIFIED_TABLE_NAME), key); + assertRows(result, row(key, expectedValue)); + } + } + } + + private Set getNodeTokens(IInvokableInstance node) + { + return node.callOnInstance(() -> { + ClusterMetadata metadata = ClusterMetadata.current(); + return metadata.tokenMap.tokens(metadata.myNodeId()).stream() + .map(Object::toString) + .collect(java.util.stream.Collectors.toSet()); + }); + } +} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/distributed/test/replication/MutationTrackingServiceTest.java b/test/distributed/org/apache/cassandra/distributed/test/replication/MutationTrackingServiceTest.java new file mode 100644 index 000000000000..4f75a00ea885 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/replication/MutationTrackingServiceTest.java @@ -0,0 +1,58 @@ +/* + * 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.replication; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.replication.MutationTrackingService; +import org.junit.Test; + +import static org.junit.Assert.assertTrue; + +public class MutationTrackingServiceTest extends TestBaseImpl +{ + private static final String KS_NAME = "ks"; + private static final String TBL_NAME = "tbl"; + + @Test + public void testIrrelevantTopologyChange() throws Throwable + { + try (Cluster cluster = builder().withNodes(3).start()) + { + // Create tracked keyspace and table + cluster.schemaChange("CREATE KEYSPACE " + KS_NAME + " WITH replication = " + + "{'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='tracked'"); + cluster.schemaChange("CREATE TABLE " + KS_NAME + "." + TBL_NAME + " (k int PRIMARY KEY, v int)"); + + // Create an untracked keyspace (should not affect tracked keyspace shards) + cluster.schemaChange("CREATE KEYSPACE untracked WITH replication = " + + "{'class': 'SimpleStrategy', 'replication_factor': 3} AND replication_type='untracked'"); + + // Test that shards remain the same object (no reconfiguration) + Boolean shardsUnchanged = cluster.get(1).callOnInstance(() -> { + var service = MutationTrackingService.instance; + Object initialShards = MutationTrackingService.TestAccess.getKeyspaceShards(service, KS_NAME); + Object newShards = MutationTrackingService.TestAccess.getKeyspaceShards(service, KS_NAME); + return initialShards == newShards; // Same object reference + }); + + assertTrue("Keyspace shards should not change for irrelevant topology changes", shardsUnchanged); + } + } +} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/distributed/test/tracking/MutationTrackingUtils.java b/test/distributed/org/apache/cassandra/distributed/test/tracking/MutationTrackingUtils.java index 091bda1338f9..e1168dcf17b9 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/tracking/MutationTrackingUtils.java +++ b/test/distributed/org/apache/cassandra/distributed/test/tracking/MutationTrackingUtils.java @@ -191,47 +191,29 @@ public static void assertIdsForKey(IInvokableInstance node, String keyspaceName, assertSummaryContents(summary, expected); } - public static void assertMatchingSummaryForKey(IInvokableInstance node, String keyspaceName, String tableName, int key, MutationSummary expected) - { - byte[] encodedExpected = encodeSummary(expected); - node.runOnInstance(() -> { - MutationSummary decodedExpected = decodeSummary(encodedExpected); - MutationSummary actual = summaryForKey(keyspaceName, tableName, key); - Assert.assertEquals(decodedExpected, actual); - }); - } - /** * Checks that nodes have seen the same ids, regardless of whether they agree on their reconciliation status */ public static void assertMatchingSummaryIdSpaceForKey(IInvokableInstance node, String keyspaceName, String tableName, int key, MutationSummary expected) { - byte[] encodedExpected = encodeSummary(expected); - node.runOnInstance(() -> { - MutationSummary decodedExpected = decodeSummary(encodedExpected); - MutationSummary actual = summaryForKey(keyspaceName, tableName, key); - Assert.assertEquals(summaryIdSpace(decodedExpected), summaryIdSpace(actual)); - }); + byte[] encodedActual = node.callOnInstance(() -> encodeSummary(summaryForKey(keyspaceName, tableName, key))); + MutationSummary actual = decodeSummary(encodedActual); + Assert.assertEquals(summaryIdSpace(expected), summaryIdSpace(actual)); } public static void assertMatchingSummaryForTable(IInvokableInstance node, String keyspaceName, String tableName, MutationSummary expected) { - byte[] encodedExpected = encodeSummary(expected); - node.runOnInstance(() -> { - MutationSummary decodedExpected = decodeSummary(encodedExpected); - MutationSummary actual = summaryForTable(keyspaceName, tableName); - Assert.assertEquals(decodedExpected, actual); - }); + byte[] encodedActual = node.callOnInstance(() -> encodeSummary(summaryForTable(keyspaceName, tableName))); + + MutationSummary actual = decodeSummary(encodedActual); + Assert.assertEquals(expected, actual); } public static void assertMatchingSummaryIdSpaceForTable(IInvokableInstance node, String keyspaceName, String tableName, MutationSummary expected) { - byte[] encodedExpected = encodeSummary(expected); - node.runOnInstance(() -> { - MutationSummary decodedExpected = decodeSummary(encodedExpected); - MutationSummary actual = summaryForTable(keyspaceName, tableName); - Assert.assertEquals(summaryIdSpace(decodedExpected), summaryIdSpace(actual)); - }); + byte[] encodedActual = node.callOnInstance(() -> encodeSummary(summaryForTable(keyspaceName, tableName))); + MutationSummary actual = decodeSummary(encodedActual); + Assert.assertEquals(summaryIdSpace(expected), summaryIdSpace(actual)); } public static void assertOffsetsIsSuperSet(Offsets expectedSuperset, Offsets expectedSubset) diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java index 7085ddf18a47..3c8f2c369855 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java @@ -158,7 +158,8 @@ private Set getReadersForRange(Range range) Collection streams = cfs.getStreamManager().createOutgoingStreams(session(NO_PENDING_REPAIR), RangesAtEndpoint.toDummyList(Collections.singleton(range)), NO_PENDING_REPAIR, - PreviewKind.NONE); + PreviewKind.NONE, + null); return sstablesFromStreams(streams); } @@ -166,7 +167,7 @@ private Set selectReaders(TimeUUID pendingRepair) { IPartitioner partitioner = DatabaseDescriptor.getPartitioner(); Collection> ranges = Lists.newArrayList(new Range(partitioner.getMinimumToken(), partitioner.getMinimumToken())); - Collection streams = cfs.getStreamManager().createOutgoingStreams(session(pendingRepair), RangesAtEndpoint.toDummyList(ranges), pendingRepair, PreviewKind.NONE); + Collection streams = cfs.getStreamManager().createOutgoingStreams(session(pendingRepair), RangesAtEndpoint.toDummyList(ranges), pendingRepair, PreviewKind.NONE, null); return sstablesFromStreams(streams); } diff --git a/test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java b/test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java index 41c37ae40d3d..2dd58b45605a 100644 --- a/test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java @@ -214,7 +214,7 @@ private void testStreamWithConcurrentComponentMutation(Callable runBeforeStre ByteBuf serializedFile = Unpooled.buffer(8192); InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort(); StreamSession session = setupStreamingSessionForTest(); - Collection outgoingStreams = store.getStreamManager().createOutgoingStreams(session, rangesAtEndpoint, NO_PENDING_REPAIR, PreviewKind.NONE); + Collection outgoingStreams = store.getStreamManager().createOutgoingStreams(session, rangesAtEndpoint, NO_PENDING_REPAIR, PreviewKind.NONE, null); CassandraOutgoingFile outgoingFile = (CassandraOutgoingFile) Iterables.getOnlyElement(outgoingStreams); Future streaming = executeAsync(() -> { diff --git a/test/unit/org/apache/cassandra/journal/JournalTest.java b/test/unit/org/apache/cassandra/journal/JournalTest.java index c28126a6ceaf..90b394518cb1 100644 --- a/test/unit/org/apache/cassandra/journal/JournalTest.java +++ b/test/unit/org/apache/cassandra/journal/JournalTest.java @@ -19,12 +19,15 @@ import java.io.IOException; import java.nio.file.Files; +import java.util.HashMap; +import java.util.Map; import org.junit.BeforeClass; import org.junit.Test; import org.apache.cassandra.ServerTestUtils; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.io.util.File; @@ -81,6 +84,45 @@ public void testSimpleReadWrite() throws IOException journal.shutdown(); } + @Test + public void testReadAll() throws IOException + { + File directory = new File(Files.createTempDirectory("JournalTestReadAll")); + directory.deleteRecursiveOnExit(); + + Journal journal = + new Journal<>("TestJournalReadAll", directory, TestParams.ACCORD, TimeUUIDKeySupport.INSTANCE, LongSerializer.INSTANCE, SegmentCompactor.noop()); + + journal.start(); + + TimeUUID id1 = nextTimeUUID(); + TimeUUID id2 = nextTimeUUID(); + TimeUUID id3 = nextTimeUUID(); + + journal.blockingWrite(id1, 10L); + journal.blockingWrite(id2, 20L); + journal.blockingWrite(id3, 30L); + + Map readValues = new HashMap<>(); + journal.readAll((segment, position, key, buffer, userVersion) -> { + try (DataInputBuffer in = new DataInputBuffer(buffer, true)) + { + readValues.put(key, LongSerializer.INSTANCE.deserialize(key, in, userVersion)); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }); + + assertEquals(3, readValues.size()); + assertEquals(10L, (long) readValues.get(id1)); + assertEquals(20L, (long) readValues.get(id2)); + assertEquals(30L, (long) readValues.get(id3)); + + journal.shutdown(); + } + static class LongSerializer implements ValueSerializer { static final LongSerializer INSTANCE = new LongSerializer(); diff --git a/test/unit/org/apache/cassandra/replication/CoordinatorLogTest.java b/test/unit/org/apache/cassandra/replication/CoordinatorLogTest.java index 5122683b41ab..7da22a6c1fa2 100644 --- a/test/unit/org/apache/cassandra/replication/CoordinatorLogTest.java +++ b/test/unit/org/apache/cassandra/replication/CoordinatorLogTest.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.cassandra.dht.Murmur3Partitioner; import org.junit.BeforeClass; import org.junit.Test; @@ -213,4 +214,92 @@ private static void validatePersistAndLoadRoundtrip(CoordinatorLog log) assertTrue(log.unreconciledMutations.equalsForTesting(loaded.unreconciledMutations)); } + + @Test + public void withParticipantsSameParticipantsTest() + { + Token tk = Murmur3Partitioner.instance.getMinimumToken(); + CoordinatorLogPrimary log = new CoordinatorLogPrimary(KEYSPACE, new Range<>(tk, tk), LOCAL_HOST_ID, LOCAL_LOG_ID, PARTICIPANTS); + + CoordinatorLog newLog = log.withParticipants(PARTICIPANTS); + + assertSame("Same participants should return same instance", log, newLog); + } + + @Test + public void withParticipantsReplicaTest() + { + Token tk = Murmur3Partitioner.instance.getMinimumToken(); + CoordinatorLogPrimary log = new CoordinatorLogPrimary(KEYSPACE, new Range<>(tk, tk), LOCAL_HOST_ID, LOCAL_LOG_ID, PARTICIPANTS); + + log.witnessedOffsets.get(2).add(300); + + Participants newParticipants = new Participants(List.of(LOCAL_HOST_ID, 5, 6)); + CoordinatorLog newLog = log.withParticipants(newParticipants); + + assertTrue("Should be CoordinatorLogPrimary", newLog instanceof CoordinatorLog.CoordinatorLogPrimary); + assertTrue("Should have empty witness state for new participants", newLog.witnessedOffsets.get(5).isEmpty()); + assertTrue("Should have empty witness state for new participants", newLog.witnessedOffsets.get(6).isEmpty()); + } + + private static Offsets offsetsWithOffsets(CoordinatorLogId logId, int... offsets) + { + Offsets.Mutable result = new Offsets.Mutable(logId); + for (int offset : offsets) + result.add(offset); + return result; + } + + private static Offsets offsetsWithOffsets(int... offsets) + { + return offsetsWithOffsets(LOCAL_LOG_ID, offsets); + } + + @Test + public void withParticipantsAddNewParticipantTest() + { + Token tk = Murmur3Partitioner.instance.getMinimumToken(); + CoordinatorLogPrimary log = new CoordinatorLogPrimary(KEYSPACE, new Range<>(tk, tk), LOCAL_HOST_ID, LOCAL_LOG_ID, PARTICIPANTS); + + log.witnessedOffsets.get(LOCAL_HOST_ID).add(1); + log.witnessedOffsets.get(2).add(2); + log.witnessedOffsets.get(3).add(3); + + log.witnessedOffsets.get(LOCAL_HOST_ID).add(5); + log.witnessedOffsets.get(2).add(5); + log.witnessedOffsets.get(3).add(5); + log.reconciledOffsets.add(5); + + Participants expandedParticipants = new Participants(List.of(LOCAL_HOST_ID, 2, 3, 4)); + CoordinatorLog newLog = log.withParticipants(expandedParticipants); + + assertEquals(offsetsWithOffsets(1, 5), newLog.witnessedOffsets.get(LOCAL_HOST_ID)); + assertEquals(offsetsWithOffsets(2, 5), newLog.witnessedOffsets.get(2)); + assertEquals(offsetsWithOffsets(3, 5), newLog.witnessedOffsets.get(3)); + assertEquals(offsetsWithOffsets( 5), newLog.witnessedOffsets.get(4)); + assertEquals(offsetsWithOffsets(5), newLog.reconciledOffsets); + } + + @Test + public void withParticipantsRemoveParticipantTest() + { + Token tk = Murmur3Partitioner.instance.getMinimumToken(); + CoordinatorLogPrimary log = new CoordinatorLogPrimary(KEYSPACE, new Range<>(tk, tk), LOCAL_HOST_ID, LOCAL_LOG_ID, PARTICIPANTS); + + log.witnessedOffsets.get(LOCAL_HOST_ID).addAll(offsetsWithOffsets(10, 40)); + log.witnessedOffsets.get(2).addAll(offsetsWithOffsets(10, 40)); + log.witnessedOffsets.get(3).addAll(offsetsWithOffsets(30, 40)); + log.reconciledOffsets.add(40); + + Participants reducedParticipants = new Participants(List.of(LOCAL_HOST_ID, 2)); + CoordinatorLog newLog = log.withParticipants(reducedParticipants); + + assertEquals(2, newLog.participants.size()); + assertEquals(offsetsWithOffsets(10, 40), newLog.witnessedOffsets.get(LOCAL_HOST_ID)); + assertEquals(offsetsWithOffsets(10, 40), newLog.witnessedOffsets.get(2)); + + // offset 10 should be promoted to reconciled since the node without it (3) has been removed + assertEquals(offsetsWithOffsets(10, 40), newLog.reconciledOffsets); + assertTrue(newLog.unreconciledMutations.isEmpty()); + } } diff --git a/test/unit/org/apache/cassandra/streaming/EntireSSTableStreamingCorrectFilesCountTest.java b/test/unit/org/apache/cassandra/streaming/EntireSSTableStreamingCorrectFilesCountTest.java index 80dd71f2734c..58d02dc4298b 100644 --- a/test/unit/org/apache/cassandra/streaming/EntireSSTableStreamingCorrectFilesCountTest.java +++ b/test/unit/org/apache/cassandra/streaming/EntireSSTableStreamingCorrectFilesCountTest.java @@ -122,7 +122,8 @@ public void test() throws Exception Collection outgoingStreams = store.getStreamManager().createOutgoingStreams(session, rangesAtEndpoint, NO_PENDING_REPAIR, - PreviewKind.NONE); + PreviewKind.NONE, + null); session.addTransferStreams(outgoingStreams); AsyncStreamingOutputPlus out = constructDataOutputStream(); diff --git a/test/unit/org/apache/cassandra/streaming/StreamSessionOwnedRangesTest.java b/test/unit/org/apache/cassandra/streaming/StreamSessionOwnedRangesTest.java index 83f0b33dc742..94af0f5d158f 100644 --- a/test/unit/org/apache/cassandra/streaming/StreamSessionOwnedRangesTest.java +++ b/test/unit/org/apache/cassandra/streaming/StreamSessionOwnedRangesTest.java @@ -144,7 +144,7 @@ private static void tryPrepareExpectingSuccess(Collection request long startMetricCount = StorageMetrics.totalOpsForInvalidToken.getCount(); session.state(StreamSession.State.PREPARING); - session.prepareAsync(requests, Collections.emptySet()); + session.prepareAsync(requests, Collections.emptySet(), null, null); assertEquals(2, sent.size()); assertEquals(PREPARE_SYNACK, sent.get(0).type); @@ -161,7 +161,7 @@ private static void tryPrepareExpectingFailure(Collection request long startMetricCount = StorageMetrics.totalOpsForInvalidToken.getCount(); session.state(StreamSession.State.PREPARING); - java.util.concurrent.Future f = session.prepare(requests, Collections.emptySet()); + java.util.concurrent.Future f = session.prepare(requests, Collections.emptySet(), null, null); Exception ex = f.get(); assertNotNull(ex); if (!(ex instanceof StreamRequestOutOfTokenRangeException))