josh-mckenzie commented on a change in pull request #1213:
URL: https://github.com/apache/cassandra/pull/1213#discussion_r734670222



##########
File path: src/java/org/apache/cassandra/schema/PartitionDenylist.java
##########
@@ -0,0 +1,519 @@
+/*
+ * 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.schema;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.ImmutableSortedSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.github.benmanes.caffeine.cache.CacheLoader;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.service.reads.range.RangeCommands;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.NoSpamLogger;
+
+import static 
org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
+import static org.apache.cassandra.cql3.QueryProcessor.process;
+import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
+
+/**
+ * PartitionDenylist uses the system_distributed.partition_denylist table to 
maintain a list of denylisted partition keys
+ * for each keyspace/table.
+ *
+ * Keys can be entered manually into the partition_denylist table or via the 
JMX operation StorageProxyMBean.denylistKey
+ *
+ * The denylist is stored as one CQL partition per table, and the denylisted 
keys are column names in that partition. The denylisted
+ * keys for each table are cached in memory, and reloaded from the 
partition_denylist table every 24 hours (default) or when the
+ * StorageProxyMBean.loadPartitionDenylist is called via JMX.
+ *
+ * Concurrency of the cache is provided by the concurrency semantics of the 
guava LoadingCache. All values (DenylistEntry) are
+ * immutable collections of keys/tokens which are replaced in whole when the 
cache refreshes from disk.
+ *
+ * The intersection of the denylist and the configurable (though not 
conveniently exposed) ConsistencyLevel is subtle; essentially
+ * we prioritize operator flexibility in the face of degraded cluster state 
over the consistency of the denylist cache across
+ * the cluster. What this means in effect is that we warn when we have a 
degraded availability scenario rather than freezing
+ * the state of the denylist or, worse, presenting an empty denylist for the 
owned ks/table set.
+ *
+ * For example, in a scenario where you have RF=3 with 2 nodes down, a 
CL.QUORUM required CL for your denylist would mean
+ * you could effectively not change the denylist configuration for your only 
available node until you recovered one of the
+ * other nodes in this replica set owning the denylist data. Rather than 
forcing operators to change the CL of their denylist
+ * table, revise the denylist, reload the cache, then revert the CL, we 
instead allow all non-timer instigated reloads to proceed.
+ *
+ * On a cache refresh in which the cache entry for a denylist entry expires we 
do not continue with the reload of the
+ * cache value and instead use the expired value, warning to the user that 
their cache state is likely stale.
+ *
+ * Notably, in the current design it's possible for a table *cache expiration 
instigated* reload to end up violating the
+ * contract on total denylisted keys allowed in the case where it initially 
loads with a value less than the DBD
+ * allowable max per table limit due to global constraint enforcement on 
initial load. Our load and reload function
+ * simply enforce the *per table* limit without consideration to what that 
entails at the global key level. While we
+ * could track the constrained state and count in DenylistEntry, for now the 
complexity doesn't seem to justify the
+ * protection against that edge case. The enforcement should take place on a 
user-instigated full reload as well as
+ * error messaging about count violations, so this only applies to situations 
in which someone adds a key and doesn't
+ * actively tell the cache to fully reload to take that key into 
consideration, which one could reasonably expect to be
+ * an antipattern.
+ */
+public class PartitionDenylist
+{
+    private static final Logger logger = 
LoggerFactory.getLogger(PartitionDenylist.class);
+    private static final NoSpamLogger AVAILABILITY_LOGGER = 
NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
+    private final ExecutorService executor = 
executorFactory().pooled("DenylistCache", 2);
+
+    /** We effectively don't use our initial empty cache to denylist until the 
{@link #load()} call which will replace it */
+    private volatile LoadingCache<TableId, DenylistEntry> denylist = 
buildEmptyCache();
+
+    /** Denylist entry is never mutated once constructed, only replaced with a 
new entry when the cache is refreshed */
+    private static class DenylistEntry
+    {
+        public final ImmutableSet<ByteBuffer> keys;
+        public final ImmutableSortedSet<Token> tokens;
+
+        public DenylistEntry()
+        {
+            keys = ImmutableSet.of();
+            tokens = ImmutableSortedSet.of();
+        }
+
+        public DenylistEntry(final ImmutableSet<ByteBuffer> keys, final 
ImmutableSortedSet<Token> tokens)
+        {
+            this.keys = keys;
+            this.tokens = tokens;
+        }
+    }
+
+    // synchronized on this
+    private int loadAttempts = 0;
+    private int loadSuccesses = 0;
+
+    public synchronized int getLoadAttempts()
+    {
+        return loadAttempts;
+    }
+    public synchronized int getLoadSuccesses()
+    {
+        return loadSuccesses;
+    }
+
+    /**
+     * Performs initial load of the partition denylist. Should be called at 
startup and only loads if the operation
+     * is expected to succeed.  If it is not possible to load at call time, a 
timer is set to retry.
+     */
+    public void initialLoad()
+    {
+        if (!DatabaseDescriptor.getEnablePartitionDenylist())
+            return;
+
+        synchronized (this)
+        {
+            loadAttempts++;
+        }
+
+        // Check if there are sufficient nodes to attempt reading all the 
denylist partitions before issuing the query.

Review comment:
       @azotcsit I spoke with some of the operators that use this feature and 
the initial implementation apparently loaded the denylist regardless of other 
node availability which led to alert storming on range slice queries. Given you 
have to have two nodes down in an RF=3 scenario and operator priority is on 
getting another node back up or bootstrapped due to proximity to data loss, the 
frequency of this being a problem is low enough and the priority of operators 
mitigating it that it's not expected to be a long-running problem.
   
   Further, the JMX load call will load regardless of CL availability which 
gives operators the ability to make a choice, either the tighter consistency 
level requirements in the .yaml (node start load + timer refresh) or the looser 
CL.ONE of using the JMX.load() call.
   
   fwiw, I don't really like this. I'd prefer we have a consistent application 
of CL across all denylist load attempts to provide a simpler API to the end 
user and something easier to reason about, but I get how it landed here.




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