aweisberg commented on code in PR #4192:
URL: https://github.com/apache/cassandra/pull/4192#discussion_r2152640113


##########
src/java/org/apache/cassandra/replication/Shard.java:
##########
@@ -158,7 +158,25 @@ private CoordinatorLog getOrCreate(CoordinatorLogId logId)
     private CoordinatorLog getOrCreate(long logId)
     {
         CoordinatorLog log = logs.get(logId);
-        return log != null
-             ? log : logs.computeIfAbsent(logId, ignore -> 
CoordinatorLog.create(localHostId, new CoordinatorLogId(logId), participants));
+        if (log != null)
+            return log;
+        CoordinatorLog newLog = logs.computeIfAbsent(logId, ignore -> 
CoordinatorLog.create(localHostId, new CoordinatorLogId(logId), participants));
+        for (Subscriber subscriber : subscribers)
+            subscriber.onLogCreation(newLog);
+        return newLog;
+    }
+
+    private final List<Subscriber> subscribers = new ArrayList<>();
+
+    public interface Subscriber

Review Comment:
   OK so if this is new then we definitely need to think through the lifecycle 
of creation and then expiration.



##########
src/java/org/apache/cassandra/replication/MutationTrackingService.java:
##########
@@ -209,7 +238,11 @@ static KeyspaceShards make(KeyspaceMetadata keyspace, 
ClusterMetadata cluster, I
             this.shards = shards;
 
             this.ppShards = new HashMap<>();
-            shards.forEach((range, shard) -> 
ppShards.put(Range.makeRowRange(range), shard));
+            this.logs = new HashMap<>();
+            shards.forEach((range, shard) -> {
+                ppShards.put(Range.makeRowRange(range), shard);
+                shard.addSubscriber(this);

Review Comment:
   Curious with shard subscription when logs get removed once they are not in 
use anymore.



##########
src/java/org/apache/cassandra/replication/CoordinatorLogOffsetsMap.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.replication;
+
+import java.util.Iterator;
+import java.util.Objects;
+import java.util.concurrent.locks.ReentrantLock;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.ThreadSafe;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.memtable.Memtable;
+import org.apache.cassandra.db.memtable.SkipListMemtable;
+import org.apache.cassandra.db.memtable.TrieMemtable;
+import org.jctools.maps.NonBlockingHashMapLong;
+import org.jctools.queues.MpscUnboundedArrayQueue;
+
+/**
+ * A replica can only receive writes from another replica it shares ranges 
with, and tracked writes are executed by
+ * coordinators, so this should contain up to (2*RF - 1) keys.
+ * <p>
+ * This is different from {@link Log2OffsetsMap} because it's focused on 
supporting fast, frequent updates from multiple
+ * threads at {@link Memtable#put}, and infrequent reads at {@link 
Memtable#getFlushSet}.
+ */
+@ThreadSafe
+class CoordinatorLogOffsetsMap extends 
NonBlockingHashMapLong<CoordinatorLogOffsetsMap.Entry> implements 
MutableCoordinatorLogOffsets
+{
+    /**
+     * 0 if no contentions.
+     */
+    private final int contentions;
+
+    protected CoordinatorLogOffsetsMap(boolean assumeExclusive)
+    {
+        super();
+        if (assumeExclusive)
+            this.contentions = 0;
+        else
+            this.contentions = DatabaseDescriptor.getConcurrentWriters();
+    }
+
+    public CoordinatorLogOffsetsMap()
+    {
+        super();
+        this.contentions = 0;
+    }
+
+    static class Entry

Review Comment:
   I would style this slightly differently. Probably a common interface or base 
class for `Entry` and a lock and no-lock version. This seems to smush together 
the lock and no-lock version and this avoids the base class or interface, but 
not really worth it IMO? You still have a bunch of logic to determine what to 
do between the two versions so you might as well just do bi-morphic dispatch to 
get the right behavior.
   
   I hate to ask for it because this does the thing it just ends up being odd 
to parse because it's mixing the two approaches in what I would call the "wrong 
way" where a single method contains both versions.



##########
test/unit/org/apache/cassandra/replication/CoordinatorLogOffsetsTest.java:
##########
@@ -0,0 +1,282 @@
+/*
+ * 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 org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper;
+import org.apache.cassandra.distributed.test.tracking.MutationTrackingUtils;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.tcm.ClusterMetadata;
+import org.apache.cassandra.tcm.ClusterMetadataService;
+import org.apache.cassandra.tcm.StubClusterMetadataService;
+import org.apache.cassandra.tcm.membership.Directory;
+import org.apache.cassandra.tcm.membership.Location;
+import org.apache.cassandra.tcm.membership.NodeId;
+import org.apache.cassandra.tcm.transformations.AlterSchema;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import accord.utils.Gen;
+import accord.utils.Gens;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.net.MessagingService;
+import org.assertj.core.api.Assertions;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import com.google.common.collect.Sets;
+
+import static accord.utils.Property.qt;
+
+public class CoordinatorLogOffsetsTest

Review Comment:
   Paramaterize the test for locking and not locking versions?



##########
src/java/org/apache/cassandra/replication/Shard.java:
##########
@@ -158,7 +158,25 @@ private CoordinatorLog getOrCreate(CoordinatorLogId logId)
     private CoordinatorLog getOrCreate(long logId)
     {
         CoordinatorLog log = logs.get(logId);
-        return log != null
-             ? log : logs.computeIfAbsent(logId, ignore -> 
CoordinatorLog.create(localHostId, new CoordinatorLogId(logId), participants));
+        if (log != null)
+            return log;
+        CoordinatorLog newLog = logs.computeIfAbsent(logId, ignore -> 
CoordinatorLog.create(localHostId, new CoordinatorLogId(logId), participants));

Review Comment:
   `CoordinatorLogId` being a POJO instead of a packing it into a long and 
using it that way makes me go hmm in the micro-optimization sense. It's one of 
those death by a thousand cuts things where no chance you can measure the 
impact of having it be a POJO.
   
   Maybe we don't loop enough on maps with it as a key for it to matter and 
don't serialize/deserialize it that often. It might not be the right order of 
magnitude frequency to matter.



##########
src/java/org/apache/cassandra/replication/CoordinatorLogOffsetsMap.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.replication;
+
+import java.util.Iterator;
+import java.util.Objects;
+import java.util.concurrent.locks.ReentrantLock;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.ThreadSafe;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.memtable.Memtable;
+import org.apache.cassandra.db.memtable.SkipListMemtable;
+import org.apache.cassandra.db.memtable.TrieMemtable;
+import org.jctools.maps.NonBlockingHashMapLong;
+import org.jctools.queues.MpscUnboundedArrayQueue;
+
+/**
+ * A replica can only receive writes from another replica it shares ranges 
with, and tracked writes are executed by
+ * coordinators, so this should contain up to (2*RF - 1) keys.
+ * <p>
+ * This is different from {@link Log2OffsetsMap} because it's focused on 
supporting fast, frequent updates from multiple
+ * threads at {@link Memtable#put}, and infrequent reads at {@link 
Memtable#getFlushSet}.
+ */
+@ThreadSafe
+class CoordinatorLogOffsetsMap extends 
NonBlockingHashMapLong<CoordinatorLogOffsetsMap.Entry> implements 
MutableCoordinatorLogOffsets
+{
+    /**
+     * 0 if no contentions.
+     */
+    private final int contentions;
+
+    protected CoordinatorLogOffsetsMap(boolean assumeExclusive)
+    {
+        super();
+        if (assumeExclusive)
+            this.contentions = 0;
+        else
+            this.contentions = DatabaseDescriptor.getConcurrentWriters();
+    }
+
+    public CoordinatorLogOffsetsMap()
+    {
+        super();
+        this.contentions = 0;
+    }
+
+    static class Entry
+    {
+        /**
+         * {@link SkipListMemtable} requires a lock because otherwise it uses 
atomics, but {@link TrieMemtable} does not,
+         * because each shard is already protected by a lock. In the contended 
case, don't wait for lock acquisition on
+         * update.
+         */
+        @Nullable private final ReentrantLock lock;
+        @Nullable private final MpscUnboundedArrayQueue<Integer> contended;
+
+        private final Offsets.Mutable base;
+
+        private Entry(CoordinatorLogId logId, int contentions)
+        {
+            this.base = new Offsets.Mutable(logId);
+
+            if (contentions == 0)
+            {
+                this.lock = null;
+                this.contended = null;
+            }
+            else
+            {
+                this.lock = new ReentrantLock();
+                this.contended = new MpscUnboundedArrayQueue<>(contentions);

Review Comment:
   Where is it documented that chunkSize should be contentions (number of 
expected threads?)



##########
src/java/org/apache/cassandra/replication/MutationTrackingService.java:
##########
@@ -180,12 +180,41 @@ int nextHostLogId()
     }
     private final AtomicInteger nextHostLogId = new AtomicInteger();
 
-    private static class KeyspaceShards
+    public boolean isDurablyReconciled(String keyspace, 
CoordinatorLogOffsets<Offsets.Immutable> 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.
+
+        KeyspaceShards keyspaceShards = shards.get(keyspace);
+        if (keyspaceShards == null)
+        {
+            logger.debug("Could not find shards for keyspace {}", keyspace);
+            return false;
+        }
+
+        for (Long logId : logOffsets)
+        {
+            CoordinatorLogId coordinatorLogId = new CoordinatorLogId(logId);
+            CoordinatorLog log = keyspaceShards.logs.get(coordinatorLogId);
+            if (log == null)
+            {
+                logger.debug("Could not find log {}", coordinatorLogId);

Review Comment:
   Is this the right way to handle this long term? When does this happen and it 
is more of a warn or error condition?



-- 
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