azotcsit commented on a change in pull request #1213:
URL: https://github.com/apache/cassandra/pull/1213#discussion_r722979087
##########
File path: src/java/org/apache/cassandra/config/DatabaseDescriptor.java
##########
@@ -3447,6 +3447,105 @@ public static void
setConsecutiveMessageErrorsThreshold(int value)
conf.consecutive_message_errors_threshold = value;
}
+ public static boolean getPartitionDenylistEnabled()
Review comment:
I believe we've already discuss smth here, but naming still seems a bit
confusing, I believe we need to have `getEnablePartitionDenylist` to be
coherent to others.
##########
File path: conf/cassandra.yaml
##########
@@ -992,6 +992,34 @@ slow_query_log_timeout_in_ms: 500
# bound (for example a few nodes with big files).
# streaming_connections_per_host: 1
+# Allows denying configurable access (rw/rr) to operations on configured ks,
table, and partitions, intended for use by
+# operators to manage cluster health vs application access. See
CASSANDRA-12106 and CEP-13 for more details.
+# enable_partition_denylist = false;
+
+# enable_denylist_writes = true;
+# enable_denylist_reads = true;
+# enable_denylist_range_reads = true;
+
+# The interval at which keys in the cache for denylisting will "expire" and
async refresh from the backing DB.
+# denylist_refresh_seconds = 86400;
+
+# In the event of errors on attempting to load the denylist cache, retry on
this interval.
+# denylist_initial_load_retry_seconds = 5;
+
+# We cap the number of denylisted keys allowed per table to keep things from
growing unbounded. Nodes will warn above
+# this limit while allowing new denylisted keys to be inserted. Denied keys
are loaded in natural query / clustering
+# ordering by partition key in case of overflow.
+# max_denylist_keys_per_table = 1000;
Review comment:
Looks like you renamed `max_denylist_keys_per_table` and
`max_denylist_keys_total` in `Config`, but you forgot to rename them in
`casandra.yaml`.
##########
File path: src/java/org/apache/cassandra/config/DatabaseDescriptor.java
##########
@@ -3447,6 +3447,105 @@ public static void
setConsecutiveMessageErrorsThreshold(int value)
conf.consecutive_message_errors_threshold = value;
}
+ public static boolean getPartitionDenylistEnabled()
+ {
+ return conf.enable_partition_denylist;
+ }
+
+ public static void setEnablePartitionDenylist(boolean enabled)
+ {
+ conf.enable_partition_denylist = enabled;
+ }
+
+ public static boolean getEnableDenylistWrites()
+ {
+ return conf.enable_denylist_writes;
+ }
+
+ public static void setEnableDenylistWrites(boolean enabled)
+ {
+ conf.enable_denylist_writes = enabled;
+ }
+
+ public static boolean getEnableDenylistReads()
+ {
+ return conf.enable_denylist_reads;
+ }
+
+ public static void setEnableDenylistReads(boolean enabled)
+ {
+ conf.enable_denylist_reads = enabled;
+ }
+
+ public static boolean getEnableDenylistRangeReads()
+ {
+ return conf.enable_denylist_range_reads;
+ }
+
+ public static void setEnableDenylistRangeReads(boolean enabled)
+ {
+ conf.enable_denylist_range_reads = enabled;
+ }
+
+ public static int getDenylistRefreshSeconds()
+ {
+ return conf.denylist_refresh_seconds;
+ }
+
+ public static void setDenylistRefreshSeconds(int seconds)
+ {
+ if (seconds <= 0)
+ throw new IllegalArgumentException("denylist_refresh_seconds must
be non-negative.");
+ conf.denylist_refresh_seconds = seconds;
+ }
+
+ public static int getDenylistInitialLoadRetrySeconds()
+ {
+ return conf.denylist_initial_load_retry_seconds;
+ }
+
+ public static void setDenylistInitialLoadRetrySeconds(int seconds)
+ {
+ if (seconds <= 0)
+ throw new
IllegalArgumentException("denylist_initial_load_retry_seconds must be
non-negative.");
+ conf.denylist_initial_load_retry_seconds = seconds;
+ }
+
+
+ public static ConsistencyLevel getDenylistConsistencyLevel()
+ {
+ return conf.denylist_consistency_level;
+ }
+
+ public static void setDenylistConsistencyLevel(ConsistencyLevel cl)
+ {
+ conf.denylist_consistency_level = cl;
+ }
+
+ public static int getDenylistKeysPerTableMax()
+ {
+ return conf.denylist_keys_per_table_max;
+ }
+
+ public static void setDenylistKeysPerTableMax(int value)
+ {
+ if (value <= 0)
+ throw new IllegalArgumentException("max_denylist_keys_per_table
must be non-negative.");
+ conf.denylist_keys_per_table_max = value;
+ }
+
+ public static int getDenylistKeysTotalMax()
+ {
+ return conf.denylist_keys_total_max;
+ }
+
+ public static void setDenylistKeysTotalMax(int value)
+ {
+ if (value <= 0)
+ throw new IllegalArgumentException("Expected an integer >= 0 for
max_denylist_keys_total. Got: " + value);
Review comment:
nit: `max_denylist_keys_total` was renamed, I feel we need to address it
in the message,
##########
File path: src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java
##########
@@ -68,6 +62,8 @@ private SystemDistributedKeyspace()
{
}
+ public static final String NAME = "system_distributed";
Review comment:
is this ever used? I feel it is a duplicate of
`SchemaConstants.DISTRIBUTED_KEYSPACE_NAME`
##########
File path: src/java/org/apache/cassandra/config/Config.java
##########
@@ -515,6 +515,43 @@
*/
public volatile int validation_preview_purge_head_start_in_sec = 60 * 60;
+ /** This feature allows denying access to operations on certain key
partitions, intended for use by operators to
+ * provide another tool to manage cluster health vs application access.
See CASSANDRA-12106 and CEP-13 for more details.
+ */
+ public volatile Boolean enable_partition_denylist = false;
+
+ /** Indicates whether denylisting (i.e. blocking operation) on configured
keys for writes is enabled. */
+ public volatile Boolean enable_denylist_writes = true;
+
+ /** Indicates whether denylisting (i.e. blocking operation) on configured
keys for reads is enabled. */
+ public volatile Boolean enable_denylist_reads = true;
+
+ /** Indicates whether denylisting (i.e. blocking operations) on configured
keys for range reads is enabled. */
+ public volatile Boolean enable_denylist_range_reads = false;
+
+ /** The interval at which keys in the cache for denylisting will "expire"
and async refresh from the backing DB. */
+ public int denylist_refresh_seconds = 86400;
+
+ /** In the event we run into errors on attempting to load the denylist
cache, we'll retry on this interval. */
+ public int denylist_initial_load_retry_seconds = 5;
+
+ /** We cap the number of denylisted keys allowed per table to keep things
from growing unbounded. Operators will
+ * receive warnings and only max_denylist_keys_per_table in natural query
ordering will be processed on overflow.
Review comment:
`max_denylist_keys_per_table` has been renamed, please, rename it
everywhere, I might miss some places to comment.
##########
File path: src/java/org/apache/cassandra/config/DatabaseDescriptor.java
##########
@@ -3447,6 +3447,105 @@ public static void
setConsecutiveMessageErrorsThreshold(int value)
conf.consecutive_message_errors_threshold = value;
}
+ public static boolean getPartitionDenylistEnabled()
+ {
+ return conf.enable_partition_denylist;
+ }
+
+ public static void setEnablePartitionDenylist(boolean enabled)
+ {
+ conf.enable_partition_denylist = enabled;
+ }
+
+ public static boolean getEnableDenylistWrites()
+ {
+ return conf.enable_denylist_writes;
+ }
+
+ public static void setEnableDenylistWrites(boolean enabled)
+ {
+ conf.enable_denylist_writes = enabled;
+ }
+
+ public static boolean getEnableDenylistReads()
+ {
+ return conf.enable_denylist_reads;
+ }
+
+ public static void setEnableDenylistReads(boolean enabled)
+ {
+ conf.enable_denylist_reads = enabled;
+ }
+
+ public static boolean getEnableDenylistRangeReads()
+ {
+ return conf.enable_denylist_range_reads;
+ }
+
+ public static void setEnableDenylistRangeReads(boolean enabled)
+ {
+ conf.enable_denylist_range_reads = enabled;
+ }
+
+ public static int getDenylistRefreshSeconds()
+ {
+ return conf.denylist_refresh_seconds;
+ }
+
+ public static void setDenylistRefreshSeconds(int seconds)
+ {
+ if (seconds <= 0)
+ throw new IllegalArgumentException("denylist_refresh_seconds must
be non-negative.");
+ conf.denylist_refresh_seconds = seconds;
+ }
+
+ public static int getDenylistInitialLoadRetrySeconds()
+ {
+ return conf.denylist_initial_load_retry_seconds;
+ }
+
+ public static void setDenylistInitialLoadRetrySeconds(int seconds)
+ {
+ if (seconds <= 0)
+ throw new
IllegalArgumentException("denylist_initial_load_retry_seconds must be
non-negative.");
+ conf.denylist_initial_load_retry_seconds = seconds;
+ }
+
+
+ public static ConsistencyLevel getDenylistConsistencyLevel()
+ {
+ return conf.denylist_consistency_level;
+ }
+
+ public static void setDenylistConsistencyLevel(ConsistencyLevel cl)
+ {
+ conf.denylist_consistency_level = cl;
+ }
+
+ public static int getDenylistKeysPerTableMax()
+ {
+ return conf.denylist_keys_per_table_max;
+ }
+
+ public static void setDenylistKeysPerTableMax(int value)
+ {
+ if (value <= 0)
+ throw new IllegalArgumentException("max_denylist_keys_per_table
must be non-negative.");
Review comment:
nit: `max_denylist_keys_per_table` was renamed, I feel we need to
address it in the message,
##########
File path: src/java/org/apache/cassandra/schema/PartitionDenylist.java
##########
@@ -0,0 +1,462 @@
+/*
+ * 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.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.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.ImmutableSortedSet;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListenableFutureTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.Hex;
+import org.apache.cassandra.utils.Pair;
+
+import static org.apache.cassandra.cql3.QueryProcessor.process;
+
+/**
+ * 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.
+ */
+public class PartitionDenylist
+{
+ private static final Logger logger =
LoggerFactory.getLogger(PartitionDenylist.class);
+ public static final String PARTITION_DENYLIST_TABLE = "partition_denylist";
+
+ private final ExecutorService executor = new ThreadPoolExecutor(2, 2,
Long.MAX_VALUE, TimeUnit.SECONDS, new LinkedBlockingQueue<>());
+
+ /** 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.enablePartitionDenylist())
+ return;
+
+ synchronized (this)
+ {
+ loadAttempts++;
+ }
+
+ // Check if there are sufficient nodes to attempt reading all the
denylist partitions before issuing the query.
+ // The pre-check prevents definite range-slice unavailables being
marked and triggering an alert. Nodes may still change
+ // state between the check and the query, but it should significantly
reduce the alert volume.
+ String retryReason = "Insufficient nodes";
+ try
+ {
+ if (readAllHasSufficientNodes() && load())
+ {
+ return;
+ }
+ }
+ catch (Throwable tr)
+ {
+ logger.error("Failed to load partition denylist", tr);
+ retryReason = "Exception";
+ }
+
+ // This path will also be taken on other failures other than
UnavailableException,
+ // but seems like a good idea to retry anyway.
+ int retryInSeconds =
DatabaseDescriptor.getDenylistInitialLoadRetrySeconds();
+ logger.info(retryReason + " while loading partition denylist cache.
Scheduled retry in {} seconds.",
+ retryInSeconds);
+ ScheduledExecutors.optionalTasks.schedule(this::initialLoad,
retryInSeconds, TimeUnit.SECONDS);
+ }
+
+ private boolean readAllHasSufficientNodes()
+ {
+ return
RangeCommands.sufficientLiveNodesForSelectStar(SystemDistributedKeyspace.PartitionDenylistTable,
+
DatabaseDescriptor.getDenylistConsistencyLevel());
+ }
+
+ /** Helper method as we need to both build cache on initial init but also
on reload of cache contents and params */
+ private LoadingCache<TableId, DenylistEntry> buildEmptyCache()
+ {
+ return CacheBuilder.newBuilder()
+
.refreshAfterWrite(DatabaseDescriptor.getDenylistRefreshSeconds(),
TimeUnit.SECONDS)
+ .build(new CacheLoader<TableId, DenylistEntry>()
+ {
+ @Override
+ public DenylistEntry load(final TableId tid)
throws Exception
+ {
+ return getDenylistForTable(tid);
+ }
+
+ @Override
+ public ListenableFuture<DenylistEntry>
reload(final TableId tid, final DenylistEntry oldValue)
+ {
+ ListenableFutureTask<DenylistEntry> task =
ListenableFutureTask.create(new Callable<DenylistEntry>()
+ {
+ @Override
+ public DenylistEntry call()
+ {
+ final DenylistEntry newEntry =
getDenylistForTable(tid);
+ if (newEntry != null)
+ return newEntry;
+ if (oldValue != null)
+ return oldValue;
+ return new DenylistEntry();
+ }
+ });
+ executor.execute(task);
+ return task;
+ }
+ });
+ }
+
+ /**
+ * We need to fully rebuild a new cache to accommodate deleting items from
the denylist and potentially shrinking
+ * the max allowable size in the list. We do not serve queries out of this
denylist until it is populated
+ * so as not to introduce a window of having a partially filled cache
allow denylisted entries.
+ */
+ public boolean load()
+ {
+ final long start = System.currentTimeMillis();
Review comment:
There should be another comment with a little bit more details. But the
general idea is that it is impossible to mock `System.currentTimeMillis` which
a problem for Simulator (see CEP-15) for details. So the bunch of CEP-15
changes went to trunk and it prohibits usage of `System.currentTimeMillis`. As
an alternative it proposes to use a custom class called `Clock`.
--
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]