aratno commented on code in PR #4396:
URL: https://github.com/apache/cassandra/pull/4396#discussion_r2383794816


##########
src/java/org/apache/cassandra/replication/MutationTrackingService.java:
##########
@@ -92,8 +96,24 @@ public class MutationTrackingService
     public static final MutationTrackingService instance = new 
MutationTrackingService();
 
     private final TrackedLocalReads localReads = new TrackedLocalReads();
-    private final ConcurrentHashMap<String, KeyspaceShards> keyspaceShards = 
new ConcurrentHashMap<>();
-    private final ConcurrentHashMap<CoordinatorLogId, Shard> log2ShardMap = 
new ConcurrentHashMap<>();
+    private ConcurrentHashMap<String, KeyspaceShards> keyspaceShards = new 
ConcurrentHashMap<>();
+    private ConcurrentHashMap<CoordinatorLogId, Shard> 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();

Review Comment:
   Practically it should be fine to keep this lock unfair, but I wonder if 
we'll find workloads with high read and write query throughput to starve 
topology changes. Could be worth using StampedLock here, we don't seem to 
require reentrancy.



##########
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<IncomingMutationLogStreamMessage> 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<IncomingMutationLogStreamMessage>
+    {
+        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();

Review Comment:
   I was thinking we'd receive all the mutation logs, then do replay to apply 
all before completing the session, rather than deserializing and applying each 
at a time



##########
src/java/org/apache/cassandra/streaming/StreamPlan.java:
##########
@@ -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();

Review Comment:
   Null check for dropped keyspace?



##########
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<OutgoingMutationLogStreamMessage> 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<DecoratedKey, TableMetadata> 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<Range<Token>> 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;

Review Comment:
   Intentional?



##########
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<OutgoingMutationLogStreamMessage> 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<DecoratedKey, TableMetadata> 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<Range<Token>> 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);

Review Comment:
   Should we avoid side effects like timeout scheduling during serde?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to