You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "bdeggleston (via GitHub)" <gi...@apache.org> on 2023/06/23 19:26:43 UTC

[GitHub] [cassandra-accord] bdeggleston commented on a diff in pull request #50: State Eviction

bdeggleston commented on code in PR #50:
URL: https://github.com/apache/cassandra-accord/pull/50#discussion_r1237412874


##########
accord-core/src/main/java/accord/topology/TopologyManager.java:
##########
@@ -59,29 +60,75 @@ static class EpochState
         private final Topology global;
         private final Topology local;
         private final QuorumTracker syncTracker;
-        private boolean syncComplete;
-        private boolean prevSynced;
+        private final boolean[] curShardSyncComplete;

Review Comment:
   can this be a bitset?



##########
accord-core/src/main/java/accord/topology/TopologyManager.java:
##########
@@ -188,30 +240,86 @@ public void syncComplete(Id node, long epoch)
             checkArgument(epoch > 0);
             if (epoch > currentEpoch)
             {
-                int idx = (int) (epoch - (1 + currentEpoch));
-                for (int i=pendingSyncComplete.size(); i<=idx; i++)
-                    pendingSyncComplete.add(new HashSet<>());
+                pending(epoch).syncComplete.add(node);
+            }
+            else
+            {
+                int i = indexOf(epoch);
+                if (i < 0 || !epochs[i].recordSyncComplete(node))
+                    return;
+
+                while (--i >= 0 && epochs[i].markPrevSynced(epochs[i + 1].syncComplete)) {}
+            }
+        }
 
-                pendingSyncComplete.get(idx).add(node);
+        /**
+         * Mark sync complete for the given node/epoch, and if this epoch
+         * is now synced, update the prevSynced flag on superseding epochs

Review Comment:
   update comment



##########
accord-core/src/main/java/accord/topology/TopologyManager.java:
##########
@@ -188,30 +240,86 @@ public void syncComplete(Id node, long epoch)
             checkArgument(epoch > 0);
             if (epoch > currentEpoch)
             {
-                int idx = (int) (epoch - (1 + currentEpoch));
-                for (int i=pendingSyncComplete.size(); i<=idx; i++)
-                    pendingSyncComplete.add(new HashSet<>());
+                pending(epoch).syncComplete.add(node);
+            }
+            else
+            {
+                int i = indexOf(epoch);
+                if (i < 0 || !epochs[i].recordSyncComplete(node))
+                    return;
+
+                while (--i >= 0 && epochs[i].markPrevSynced(epochs[i + 1].syncComplete)) {}
+            }
+        }
 
-                pendingSyncComplete.get(idx).add(node);
+        /**
+         * Mark sync complete for the given node/epoch, and if this epoch
+         * is now synced, update the prevSynced flag on superseding epochs
+         */
+        public void epochClosed(Ranges ranges, long epoch)
+        {
+            checkArgument(epoch > 0);
+            int i;
+            if (epoch > currentEpoch)
+            {
+                Notifications notifications = pending(epoch);
+                notifications.closed = notifications.closed.with(ranges);
+                i = 0;
             }
             else
             {
-                EpochState state = get(epoch);
-                state.recordSyncComplete(node);
-                for (epoch++ ; state.syncComplete() && epoch <= currentEpoch; epoch++)
-                {
-                    state = get(epoch);
-                    state.markPrevSynced();
-                }
+                i = indexOf(epoch);
+            }
+            while (epochs[i].recordClosed(ranges) && ++i < epochs.length) {}
+        }
+
+        /**
+         * Mark sync complete for the given node/epoch, and if this epoch
+         * is now synced, update the prevSynced flag on superseding epochs

Review Comment:
   update comment



##########
accord-core/src/main/java/accord/local/Commands.java:
##########
@@ -610,247 +665,236 @@ private static boolean maybeExecute(SafeCommandStore safeStore, SafeCommand safe
         }
     }
 
-    protected static WaitingOn populateWaitingOn(SafeCommandStore safeStore, TxnId txnId, Timestamp executeAt, PartialDeps partialDeps)

Review Comment:
   no action / change needed, just wanted to say I like the streamlining of waiting on / listener registration here



##########
accord-core/src/main/java/accord/topology/TopologyManager.java:
##########
@@ -188,30 +240,86 @@ public void syncComplete(Id node, long epoch)
             checkArgument(epoch > 0);
             if (epoch > currentEpoch)
             {
-                int idx = (int) (epoch - (1 + currentEpoch));
-                for (int i=pendingSyncComplete.size(); i<=idx; i++)
-                    pendingSyncComplete.add(new HashSet<>());
+                pending(epoch).syncComplete.add(node);
+            }
+            else
+            {
+                int i = indexOf(epoch);
+                if (i < 0 || !epochs[i].recordSyncComplete(node))
+                    return;
+
+                while (--i >= 0 && epochs[i].markPrevSynced(epochs[i + 1].syncComplete)) {}
+            }
+        }
 
-                pendingSyncComplete.get(idx).add(node);
+        /**
+         * Mark sync complete for the given node/epoch, and if this epoch
+         * is now synced, update the prevSynced flag on superseding epochs
+         */
+        public void epochClosed(Ranges ranges, long epoch)
+        {
+            checkArgument(epoch > 0);
+            int i;
+            if (epoch > currentEpoch)
+            {
+                Notifications notifications = pending(epoch);
+                notifications.closed = notifications.closed.with(ranges);
+                i = 0;
             }
             else
             {
-                EpochState state = get(epoch);
-                state.recordSyncComplete(node);
-                for (epoch++ ; state.syncComplete() && epoch <= currentEpoch; epoch++)
-                {
-                    state = get(epoch);
-                    state.markPrevSynced();
-                }
+                i = indexOf(epoch);
+            }
+            while (epochs[i].recordClosed(ranges) && ++i < epochs.length) {}
+        }
+
+        /**
+         * Mark sync complete for the given node/epoch, and if this epoch
+         * is now synced, update the prevSynced flag on superseding epochs
+         */
+        public void epochRedundant(Ranges ranges, long epoch)
+        {
+            checkArgument(epoch > 0);
+            int i;
+            if (epoch > currentEpoch)
+            {
+                Notifications notifications = pending(epoch);
+                notifications.complete = notifications.complete.with(ranges);
+                i = 0; // record these ranges as complete for all earlier epochs as well
             }
+            else
+            {
+                i = indexOf(epoch);
+                if (i < 0)
+                    return;
+            }
+            while (epochs[i].recordComplete(ranges) && ++i < epochs.length) {}
+        }
+
+        private Notifications pending(long epoch)
+        {
+            int idx = (int) (epoch - (1 + currentEpoch));

Review Comment:
   can we have a sanity check that epoch>currentEpoch?



##########
accord-core/src/main/java/accord/utils/SimpleBitSet.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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 accord.utils;
+
+import java.util.Arrays;
+
+import static java.lang.Long.highestOneBit;
+import static java.lang.Long.lowestOneBit;
+import static java.lang.Long.numberOfTrailingZeros;
+
+public class SimpleBitSet

Review Comment:
   why not use the java bitset?



##########
accord-core/src/main/java/accord/impl/AbstractConfigurationService.java:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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 accord.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.primitives.Ints;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import accord.api.ConfigurationService;
+import accord.local.Node;
+import accord.primitives.Ranges;
+import accord.topology.Topology;
+import accord.utils.Invariants;
+import accord.utils.async.AsyncChain;
+import accord.utils.async.AsyncResult;
+import accord.utils.async.AsyncResults;
+
+public abstract class AbstractConfigurationService<EpochState extends AbstractConfigurationService.AbstractEpochState,
+                                                   EpochHistory extends AbstractConfigurationService.AbstractEpochHistory<EpochState>>
+                      implements ConfigurationService
+{
+    private static final Logger logger = LoggerFactory.getLogger(AbstractConfigurationService.class);
+
+    protected final Node.Id localId;
+
+    protected final EpochHistory epochs = createEpochHistory();
+
+    protected final List<Listener> listeners = new ArrayList<>();
+
+    public abstract static class AbstractEpochState
+    {
+        protected final long epoch;
+        protected final AsyncResult.Settable<Topology> received = AsyncResults.settable();
+        protected final AsyncResult.Settable<Void> acknowledged = AsyncResults.settable();
+        protected AsyncResult<Void> reads = null;
+
+        protected Topology topology = null;
+
+        public AbstractEpochState(long epoch)
+        {
+            this.epoch = epoch;
+        }
+
+        public long epoch()
+        {
+            return epoch;
+        }
+
+        @Override
+        public String toString()
+        {
+            return "EpochState{" + epoch + '}';
+        }
+    }
+
+    @VisibleForTesting
+    public abstract static class AbstractEpochHistory<EpochState extends AbstractEpochState>
+    {
+        // TODO (low priority): move pendingEpochs / FetchTopology into here?
+        private List<EpochState> epochs = new ArrayList<>();
+
+        protected long lastReceived = 0;
+        protected long lastAcknowledged = 0;
+
+        protected abstract EpochState createEpochState(long epoch);
+
+        public long minEpoch()
+        {
+            return epochs.isEmpty() ? 0L : epochs.get(0).epoch;
+        }
+
+        public long maxEpoch()
+        {
+            int size = epochs.size();
+            return size == 0 ? 0L : epochs.get(size - 1).epoch;
+        }
+
+        @VisibleForTesting
+        EpochState atIndex(int idx)
+        {
+            return epochs.get(idx);
+        }
+
+        @VisibleForTesting
+        int size()
+        {
+            return epochs.size();
+        }
+
+        EpochState getOrCreate(long epoch)
+        {
+            Invariants.checkArgument(epoch > 0, "Epoch must be positive but given %d", epoch);
+            if (epochs.isEmpty())
+            {
+                EpochState state = createEpochState(epoch);
+                epochs.add(state);
+                return state;
+            }
+
+            long minEpoch = minEpoch();
+            if (epoch < minEpoch)
+            {
+                int prepend = Ints.checkedCast(minEpoch - epoch);
+                List<EpochState> next = new ArrayList<>(epochs.size() + prepend);
+                for (long addEpoch=epoch; addEpoch<minEpoch; addEpoch++)
+                    next.add(createEpochState(addEpoch));
+                next.addAll(epochs);
+                epochs = next;
+                minEpoch = minEpoch();
+                Invariants.checkState(minEpoch == epoch);
+            }
+            long maxEpoch = maxEpoch();
+            int idx = Ints.checkedCast(epoch - minEpoch);
+
+            // add any missing epochs
+            for (long addEpoch = maxEpoch + 1; addEpoch <= epoch; addEpoch++)
+                epochs.add(createEpochState(addEpoch));
+
+            return epochs.get(idx);
+        }
+
+        public void receive(Topology topology)
+        {
+            long epoch = topology.epoch();
+            Invariants.checkState(lastReceived == epoch - 1 || epoch == 0 || lastReceived == 0,
+                                  "Epoch %d != %d + 1", epoch, lastReceived);
+            lastReceived = epoch;
+            EpochState state = getOrCreate(epoch);
+            state.topology = topology;
+            state.received.setSuccess(topology);
+        }
+
+        AsyncResult<Topology> receiveFuture(long epoch)
+        {
+            return getOrCreate(epoch).received;
+        }
+
+        Topology topologyFor(long epoch)
+        {
+            return getOrCreate(epoch).topology;
+        }
+
+        public void acknowledge(EpochReady ready)
+        {
+            long epoch = ready.epoch;
+            Invariants.checkState(lastAcknowledged == epoch - 1 || epoch == 0 || lastAcknowledged == 0,
+                                  "Epoch %d != %d + 1", epoch, lastAcknowledged);
+            lastAcknowledged = epoch;
+            EpochState state = getOrCreate(epoch);
+            Invariants.checkState(state.reads == null, "Reads result was already set for epoch", epoch);
+            state.reads = ready.reads;
+            state.acknowledged.setSuccess(null);
+        }
+
+        AsyncResult<Void> acknowledgeFuture(long epoch)
+        {
+            return getOrCreate(epoch).acknowledged;
+        }
+
+        void truncateUntil(long epoch)
+        {
+            Invariants.checkArgument(epoch <= maxEpoch(), "epoch %d > %d", epoch, maxEpoch());
+            long minEpoch = minEpoch();
+            int toTrim = Ints.checkedCast(epoch - minEpoch);
+            if (toTrim <=0)
+                return;
+
+            epochs = new ArrayList<>(epochs.subList(toTrim, epochs.size()));
+        }
+    }
+
+    public AbstractConfigurationService(Node.Id localId)
+    {
+        this.localId = localId;
+    }
+
+    protected abstract EpochHistory createEpochHistory();
+
+    protected EpochState getOrCreateEpochState(long epoch)
+    {
+        return epochs.getOrCreate(epoch);
+    }
+
+    @Override
+    public synchronized void registerListener(Listener listener)
+    {
+        listeners.add(listener);
+    }
+
+    @Override
+    public synchronized Topology currentTopology()
+    {
+        return epochs.topologyFor(epochs.lastReceived);
+    }
+
+    @Override
+    public synchronized Topology getTopologyForEpoch(long epoch)
+    {
+        return epochs.topologyFor(epoch);
+    }
+
+    protected abstract void fetchTopologyInternal(long epoch);
+
+    @Override
+    public synchronized void fetchTopologyForEpoch(long epoch)
+    {
+        if (epoch <= epochs.lastReceived)
+            return;
+
+        fetchTopologyInternal(epoch);
+    }
+
+    protected abstract void localSyncComplete(Topology topology);
+
+    @Override
+    public synchronized void acknowledgeEpoch(EpochReady ready)
+    {
+        ready.metadata.addCallback(() -> epochs.acknowledge(ready));
+        ready.coordination.addCallback(() ->  localSyncComplete(epochs.getOrCreate(ready.epoch).topology));
+    }
+
+    protected void topologyUpdatePreListenerNotify(Topology topology) {}
+    protected void topologyUpdatePostListenerNotify(Topology topology) {}
+
+    public synchronized void reportTopology(Topology topology, boolean startSync)
+    {
+        long lastReceived = epochs.lastReceived;
+        if (topology.epoch() <= lastReceived)
+            return;
+
+        if (lastReceived > 0 && topology.epoch() > lastReceived + 1)
+        {
+            fetchTopologyForEpoch(lastReceived + 1);
+            epochs.receiveFuture(lastReceived + 1).addCallback(() -> reportTopology(topology, startSync));
+            return;
+        }
+
+        long lastAcked = epochs.lastAcknowledged;
+        if (lastAcked > 0 && topology.epoch() > lastAcked + 1)
+        {
+            epochs.acknowledgeFuture(lastAcked + 1).addCallback(() -> reportTopology(topology, startSync));
+            return;
+        }
+        logger.trace("Epoch {} received by {}", topology.epoch(), localId);
+
+        epochs.receive(topology);
+        topologyUpdatePreListenerNotify(topology);
+        for (Listener listener : listeners)
+            listener.onTopologyUpdate(topology, startSync);
+        topologyUpdatePostListenerNotify(topology);
+    }
+
+    public synchronized void reportTopology(Topology topology)
+    {
+        reportTopology(topology, true);
+    }
+
+    protected void receiveSyncCompletePreListenerNotify(Node.Id node, long epoch) {}

Review Comment:
   can we preserve the local/remote naming?



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org