You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2021/12/02 02:23:15 UTC

[GitHub] [cassandra] dcapwell commented on a change in pull request #1321: CASSANDRA-17147 trunk: Guardrails prototype

dcapwell commented on a change in pull request #1321:
URL: https://github.com/apache/cassandra/pull/1321#discussion_r760696410



##########
File path: src/java/org/apache/cassandra/config/GuardrailsOptions.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.config;
+
+import java.util.Collections;
+import java.util.Set;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.Sets;
+
+import org.apache.cassandra.cql3.statements.schema.TableAttributes;
+import org.apache.cassandra.db.guardrails.Guardrails;
+import org.apache.cassandra.db.guardrails.GuardrailsConfig;
+import org.apache.cassandra.db.guardrails.Values;
+
+import static java.lang.String.format;
+import static java.util.stream.Collectors.toSet;
+
+/**
+ * Configuration settings for guardrails populated from the Yaml file.
+ *
+ * <p>Note that the settings here must only be used to build the {@link GuardrailsConfig} class and not directly by the
+ * code checking each guarded constraint. That code should use the higher level abstractions defined in
+ * {@link Guardrails}).
+ *
+ * <p>This contains a main setting, {@code enabled}, controlling if guardrails are globally active or not, and
+ * individual settings to control each guardrail.
+ *
+ * <p>We have 2 variants of guardrails, soft (warn) and hard (abort) limits, each guardrail having either one of the
+ * variants or both. Note in particular that hard limits only make sense for guardrails triggering during query
+ * execution. For other guardrails, say one triggering during compaction, aborting that compaction does not make sense.
+ *
+ * <p>Additionally, each individual setting should have a specific value (typically -1 for numeric settings),
+ * that allows to disable the corresponding guardrail.
+ */
+public class GuardrailsOptions implements GuardrailsConfig
+{
+    public volatile boolean enabled = false;
+    public final IntThreshold tables = new IntThreshold();
+    public final IntThreshold columns_per_table = new IntThreshold();
+    public final IntThreshold secondary_indexes_per_table = new IntThreshold();
+    public final IntThreshold materialized_views_per_table = new IntThreshold();
+    public final TableProperties table_properties = new TableProperties();
+    public volatile boolean user_timestamps_enabled = true;
+
+    public void validate()
+    {
+        tables.validate("guardrails.tables");
+        columns_per_table.validate("guardrails.columns_per_table");
+        secondary_indexes_per_table.validate("guardrails.secondary_indexes_per_table");
+        materialized_views_per_table.validate("guardrails.materialized_views_per_table");
+        table_properties.validate("guardrails.table_properties");
+    }
+
+    @Override
+    public boolean getEnabled()
+    {
+        return enabled;
+    }
+
+    /**
+     * Enable/disable guardrails.
+     *
+     * @param enabled {@code true} for enabling guardrails, {@code false} for disabling them.
+     */
+    public void setEnabled(boolean enabled)
+    {
+        validateNotNull(enabled, "guardrails.enabled");
+        this.enabled = enabled;
+    }
+
+    @Override
+    public IntThreshold getTables()
+    {
+        return tables;
+    }
+
+    @Override
+    public IntThreshold getColumnsPerTable()
+    {
+        return columns_per_table;
+    }
+
+    @Override
+    public IntThreshold getSecondaryIndexesPerTable()
+    {
+        return secondary_indexes_per_table;
+    }
+
+    @Override
+    public IntThreshold getMaterializedViewsPerTable()
+    {
+        return materialized_views_per_table;
+    }
+
+    @Override
+    public TableProperties getTableProperties()
+    {
+        return table_properties;
+    }
+
+    @Override
+    public boolean getUserTimestampsEnabled()
+    {
+        return user_timestamps_enabled;
+    }
+
+    public void setUserTimestampsEnabled(boolean enabled)
+    {
+        validateNotNull(enabled, "guardrails.user_timestamps_enabled");
+        user_timestamps_enabled = enabled;
+    }
+
+    private static <T> void validateNotNull(T value, String name)
+    {
+        if (value == null)
+            throw new IllegalArgumentException(format("Invalid value for %s: null is not allowed", name));
+    }
+
+    public static abstract class Threshold implements org.apache.cassandra.db.guardrails.Threshold.Config
+    {
+        protected void validatePositiveNumeric(long value, long maxValue, boolean allowZero, String name)
+        {
+            validateNotNull(value, name);
+
+            if (value > maxValue)
+                throw new IllegalArgumentException(format("Invalid value %d for %s: maximum allowed value is %d",
+                                                          value, name, maxValue));
+
+            if (value == 0 && !allowZero)
+                throw new IllegalArgumentException(format("Invalid value for %s: 0 is not allowed", name));

Review comment:
       `-1` is the magic number to disable, maybe provide that hint? `"Invalid value for %s: 0 is not allowed; if attempting to disable use -1"`

##########
File path: src/java/org/apache/cassandra/config/GuardrailsOptions.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.config;
+
+import java.util.Collections;
+import java.util.Set;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.Sets;
+
+import org.apache.cassandra.cql3.statements.schema.TableAttributes;
+import org.apache.cassandra.db.guardrails.Guardrails;
+import org.apache.cassandra.db.guardrails.GuardrailsConfig;
+import org.apache.cassandra.db.guardrails.Values;
+
+import static java.lang.String.format;
+import static java.util.stream.Collectors.toSet;
+
+/**
+ * Configuration settings for guardrails populated from the Yaml file.
+ *
+ * <p>Note that the settings here must only be used to build the {@link GuardrailsConfig} class and not directly by the
+ * code checking each guarded constraint. That code should use the higher level abstractions defined in
+ * {@link Guardrails}).
+ *
+ * <p>This contains a main setting, {@code enabled}, controlling if guardrails are globally active or not, and
+ * individual settings to control each guardrail.
+ *
+ * <p>We have 2 variants of guardrails, soft (warn) and hard (abort) limits, each guardrail having either one of the
+ * variants or both. Note in particular that hard limits only make sense for guardrails triggering during query
+ * execution. For other guardrails, say one triggering during compaction, aborting that compaction does not make sense.
+ *
+ * <p>Additionally, each individual setting should have a specific value (typically -1 for numeric settings),
+ * that allows to disable the corresponding guardrail.
+ */
+public class GuardrailsOptions implements GuardrailsConfig
+{
+    public volatile boolean enabled = false;
+    public final IntThreshold tables = new IntThreshold();
+    public final IntThreshold columns_per_table = new IntThreshold();
+    public final IntThreshold secondary_indexes_per_table = new IntThreshold();
+    public final IntThreshold materialized_views_per_table = new IntThreshold();
+    public final TableProperties table_properties = new TableProperties();
+    public volatile boolean user_timestamps_enabled = true;
+
+    public void validate()
+    {
+        tables.validate("guardrails.tables");
+        columns_per_table.validate("guardrails.columns_per_table");
+        secondary_indexes_per_table.validate("guardrails.secondary_indexes_per_table");
+        materialized_views_per_table.validate("guardrails.materialized_views_per_table");
+        table_properties.validate("guardrails.table_properties");
+    }
+
+    @Override
+    public boolean getEnabled()
+    {
+        return enabled;
+    }
+
+    /**
+     * Enable/disable guardrails.
+     *
+     * @param enabled {@code true} for enabling guardrails, {@code false} for disabling them.
+     */
+    public void setEnabled(boolean enabled)
+    {
+        validateNotNull(enabled, "guardrails.enabled");
+        this.enabled = enabled;
+    }
+
+    @Override
+    public IntThreshold getTables()
+    {
+        return tables;
+    }
+
+    @Override
+    public IntThreshold getColumnsPerTable()
+    {
+        return columns_per_table;
+    }
+
+    @Override
+    public IntThreshold getSecondaryIndexesPerTable()
+    {
+        return secondary_indexes_per_table;
+    }
+
+    @Override
+    public IntThreshold getMaterializedViewsPerTable()
+    {
+        return materialized_views_per_table;
+    }
+
+    @Override
+    public TableProperties getTableProperties()
+    {
+        return table_properties;
+    }
+
+    @Override
+    public boolean getUserTimestampsEnabled()
+    {
+        return user_timestamps_enabled;
+    }
+
+    public void setUserTimestampsEnabled(boolean enabled)
+    {
+        validateNotNull(enabled, "guardrails.user_timestamps_enabled");

Review comment:
       not possible to be null; the type is `boolean` not `Boolean`

##########
File path: src/java/org/apache/cassandra/config/GuardrailsOptions.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.config;
+
+import java.util.Collections;
+import java.util.Set;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.Sets;
+
+import org.apache.cassandra.cql3.statements.schema.TableAttributes;
+import org.apache.cassandra.db.guardrails.Guardrails;
+import org.apache.cassandra.db.guardrails.GuardrailsConfig;
+import org.apache.cassandra.db.guardrails.Values;
+
+import static java.lang.String.format;
+import static java.util.stream.Collectors.toSet;
+
+/**
+ * Configuration settings for guardrails populated from the Yaml file.
+ *
+ * <p>Note that the settings here must only be used to build the {@link GuardrailsConfig} class and not directly by the
+ * code checking each guarded constraint. That code should use the higher level abstractions defined in
+ * {@link Guardrails}).
+ *
+ * <p>This contains a main setting, {@code enabled}, controlling if guardrails are globally active or not, and
+ * individual settings to control each guardrail.
+ *
+ * <p>We have 2 variants of guardrails, soft (warn) and hard (abort) limits, each guardrail having either one of the
+ * variants or both. Note in particular that hard limits only make sense for guardrails triggering during query
+ * execution. For other guardrails, say one triggering during compaction, aborting that compaction does not make sense.
+ *
+ * <p>Additionally, each individual setting should have a specific value (typically -1 for numeric settings),
+ * that allows to disable the corresponding guardrail.
+ */
+public class GuardrailsOptions implements GuardrailsConfig
+{
+    public volatile boolean enabled = false;
+    public final IntThreshold tables = new IntThreshold();
+    public final IntThreshold columns_per_table = new IntThreshold();
+    public final IntThreshold secondary_indexes_per_table = new IntThreshold();
+    public final IntThreshold materialized_views_per_table = new IntThreshold();
+    public final TableProperties table_properties = new TableProperties();
+    public volatile boolean user_timestamps_enabled = true;
+
+    public void validate()
+    {
+        tables.validate("guardrails.tables");
+        columns_per_table.validate("guardrails.columns_per_table");
+        secondary_indexes_per_table.validate("guardrails.secondary_indexes_per_table");
+        materialized_views_per_table.validate("guardrails.materialized_views_per_table");
+        table_properties.validate("guardrails.table_properties");
+    }
+
+    @Override
+    public boolean getEnabled()
+    {
+        return enabled;
+    }
+
+    /**
+     * Enable/disable guardrails.
+     *
+     * @param enabled {@code true} for enabling guardrails, {@code false} for disabling them.
+     */
+    public void setEnabled(boolean enabled)
+    {
+        validateNotNull(enabled, "guardrails.enabled");
+        this.enabled = enabled;
+    }
+
+    @Override
+    public IntThreshold getTables()
+    {
+        return tables;
+    }
+
+    @Override
+    public IntThreshold getColumnsPerTable()
+    {
+        return columns_per_table;
+    }
+
+    @Override
+    public IntThreshold getSecondaryIndexesPerTable()
+    {
+        return secondary_indexes_per_table;
+    }
+
+    @Override
+    public IntThreshold getMaterializedViewsPerTable()
+    {
+        return materialized_views_per_table;
+    }
+
+    @Override
+    public TableProperties getTableProperties()
+    {
+        return table_properties;
+    }
+
+    @Override
+    public boolean getUserTimestampsEnabled()
+    {
+        return user_timestamps_enabled;
+    }
+
+    public void setUserTimestampsEnabled(boolean enabled)
+    {
+        validateNotNull(enabled, "guardrails.user_timestamps_enabled");
+        user_timestamps_enabled = enabled;
+    }
+
+    private static <T> void validateNotNull(T value, String name)
+    {
+        if (value == null)
+            throw new IllegalArgumentException(format("Invalid value for %s: null is not allowed", name));
+    }
+
+    public static abstract class Threshold implements org.apache.cassandra.db.guardrails.Threshold.Config
+    {
+        protected void validatePositiveNumeric(long value, long maxValue, boolean allowZero, String name)
+        {
+            validateNotNull(value, name);
+
+            if (value > maxValue)
+                throw new IllegalArgumentException(format("Invalid value %d for %s: maximum allowed value is %d",
+                                                          value, name, maxValue));
+
+            if (value == 0 && !allowZero)
+                throw new IllegalArgumentException(format("Invalid value for %s: 0 is not allowed", name));
+
+            // We allow -1 as a general "disabling" flag. But reject anything lower to avoid mistakes.
+            if (value < -1L)
+                throw new IllegalArgumentException(format("Invalid value %d for %s: negative values are not allowed, " +
+                                                          "outside of -1 which disables the guardrail",
+                                                          value, name));
+        }
+
+        protected void validateWarnLowerThanAbort(long warnValue, long abortValue, @Nullable String name)
+        {
+            if (warnValue == -1 || abortValue == -1)
+                return;
+
+            if (abortValue < warnValue)
+                throw new IllegalArgumentException(format("The warn threshold %d%s should be lower than the abort " +
+                                                          "threshold %d",
+                                                          warnValue,
+                                                          name == null ? "" : " for " + name,
+                                                          abortValue));
+        }
+    }
+
+    public static class IntThreshold extends Threshold
+    {
+        public volatile int warn_threshold = -1;

Review comment:
       should move away from -1 as a magic number in favor of a constant.

##########
File path: src/java/org/apache/cassandra/config/GuardrailsOptions.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.config;
+
+import java.util.Collections;
+import java.util.Set;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.Sets;
+
+import org.apache.cassandra.cql3.statements.schema.TableAttributes;
+import org.apache.cassandra.db.guardrails.Guardrails;
+import org.apache.cassandra.db.guardrails.GuardrailsConfig;
+import org.apache.cassandra.db.guardrails.Values;
+
+import static java.lang.String.format;
+import static java.util.stream.Collectors.toSet;
+
+/**
+ * Configuration settings for guardrails populated from the Yaml file.
+ *
+ * <p>Note that the settings here must only be used to build the {@link GuardrailsConfig} class and not directly by the
+ * code checking each guarded constraint. That code should use the higher level abstractions defined in
+ * {@link Guardrails}).
+ *
+ * <p>This contains a main setting, {@code enabled}, controlling if guardrails are globally active or not, and
+ * individual settings to control each guardrail.
+ *
+ * <p>We have 2 variants of guardrails, soft (warn) and hard (abort) limits, each guardrail having either one of the
+ * variants or both. Note in particular that hard limits only make sense for guardrails triggering during query
+ * execution. For other guardrails, say one triggering during compaction, aborting that compaction does not make sense.
+ *
+ * <p>Additionally, each individual setting should have a specific value (typically -1 for numeric settings),
+ * that allows to disable the corresponding guardrail.
+ */
+public class GuardrailsOptions implements GuardrailsConfig
+{
+    public volatile boolean enabled = false;
+    public final IntThreshold tables = new IntThreshold();
+    public final IntThreshold columns_per_table = new IntThreshold();
+    public final IntThreshold secondary_indexes_per_table = new IntThreshold();
+    public final IntThreshold materialized_views_per_table = new IntThreshold();
+    public final TableProperties table_properties = new TableProperties();
+    public volatile boolean user_timestamps_enabled = true;
+
+    public void validate()
+    {
+        tables.validate("guardrails.tables");
+        columns_per_table.validate("guardrails.columns_per_table");
+        secondary_indexes_per_table.validate("guardrails.secondary_indexes_per_table");
+        materialized_views_per_table.validate("guardrails.materialized_views_per_table");
+        table_properties.validate("guardrails.table_properties");
+    }
+
+    @Override
+    public boolean getEnabled()
+    {
+        return enabled;
+    }
+
+    /**
+     * Enable/disable guardrails.
+     *
+     * @param enabled {@code true} for enabling guardrails, {@code false} for disabling them.
+     */
+    public void setEnabled(boolean enabled)
+    {
+        validateNotNull(enabled, "guardrails.enabled");
+        this.enabled = enabled;
+    }
+
+    @Override
+    public IntThreshold getTables()
+    {
+        return tables;
+    }
+
+    @Override
+    public IntThreshold getColumnsPerTable()
+    {
+        return columns_per_table;
+    }
+
+    @Override
+    public IntThreshold getSecondaryIndexesPerTable()
+    {
+        return secondary_indexes_per_table;
+    }
+
+    @Override
+    public IntThreshold getMaterializedViewsPerTable()
+    {
+        return materialized_views_per_table;
+    }
+
+    @Override
+    public TableProperties getTableProperties()
+    {
+        return table_properties;
+    }
+
+    @Override
+    public boolean getUserTimestampsEnabled()
+    {
+        return user_timestamps_enabled;
+    }
+
+    public void setUserTimestampsEnabled(boolean enabled)
+    {
+        validateNotNull(enabled, "guardrails.user_timestamps_enabled");
+        user_timestamps_enabled = enabled;
+    }
+
+    private static <T> void validateNotNull(T value, String name)
+    {
+        if (value == null)
+            throw new IllegalArgumentException(format("Invalid value for %s: null is not allowed", name));
+    }
+
+    public static abstract class Threshold implements org.apache.cassandra.db.guardrails.Threshold.Config
+    {
+        protected void validatePositiveNumeric(long value, long maxValue, boolean allowZero, String name)
+        {
+            validateNotNull(value, name);

Review comment:
       not possible to be null; `long` is used

##########
File path: src/java/org/apache/cassandra/db/guardrails/Guardrails.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.db.guardrails;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.service.ClientState;
+
+import static java.lang.String.format;
+
+/**
+ * Entry point for Guardrails, storing the defined guardrails and providing a few global methods over them.
+ */
+public abstract class Guardrails
+{
+    public static final GuardrailsConfigProvider configProvider = GuardrailsConfigProvider.instance;
+    public static final GuardrailsConfigService configService = new GuardrailsConfigService();

Review comment:
       should be CONFIG_PROVIDER or PROVIDER given its static.  Some examples in code don't do this but its more common than not (and its the java recommended style); first example that came to mind was `TableMetrics`

##########
File path: src/java/org/apache/cassandra/db/guardrails/Guardrail.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.db.guardrails;
+
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.NoSpamLogger;
+
+/**
+ * General class defining a given guardrail that guards against some particular usage/condition.
+ * <p>
+ * Some guardrails only emit warnings when triggered, while others abort the query that triggers them. Some may do one
+ * or the other based on specific threshold. The queries are aborted with an {@link InvalidRequestException}, unless
+ * {@link #setNoExceptionOnAbort()} is used to prevent aborting background processes such as compaction.
+ * <p>
+ * Note that all the defined classes support live updates, which is why each guardrail class constructor takes
+ * suppliers of the condition the guardrail acts on rather than the condition itself. This implies that said suppliers
+ * should be fast and non-blocking to avoid surprises.
+ */
+public abstract class Guardrail
+{
+    protected static final NoSpamLogger logger = NoSpamLogger.getLogger(LoggerFactory.getLogger(Guardrail.class),
+                                                                        10, TimeUnit.MINUTES);
+
+    /** Whether to throw {@link InvalidRequestException} on {@link this#abort(String)}. */
+    protected boolean throwOnAbort = true;

Review comment:
       deadcode, also not thread safe if mutated; can we remove?

##########
File path: src/java/org/apache/cassandra/db/guardrails/GuardrailsConfigProvider.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.db.guardrails;
+
+import javax.annotation.Nullable;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * Provider of {@link GuardrailsConfig}s for a {@link ClientState}.
+ * <p>
+ * The {@link Default} implementation always retuns the {@link GuardrailsConfig} parsed from {@code cassandra.yaml},
+ * but different implementations can return different configurations based on the specified {@link ClientState}.
+ * <p>
+ * Custom implementations can be specified at runtime with the system property {@link #CUSTOM_IMPLEMENTATION_PROPERTY}.
+ * These configurations can be used to read the guardrails configuration from some other source, and provide different
+ * configurations depending on the {@link ClientState} or some other factors. However, this mechanism for pluggability
+ * and the related {@link GuardrailsConfig} interface are not officially supported and may change in a minor release.
+ */
+public interface GuardrailsConfigProvider
+{
+    public static final String CUSTOM_IMPLEMENTATION_PROPERTY = "cassandra.custom_guardrails_config_provider_class";
+
+    public static final GuardrailsConfigProvider instance = System.getProperty(CUSTOM_IMPLEMENTATION_PROPERTY) == null

Review comment:
       nit: drop `public`, not needed (see https://docs.oracle.com/javase/specs/jls/se7/html/jls-6.html#jls-6.6.1)

##########
File path: conf/cassandra.yaml
##########
@@ -1571,3 +1571,41 @@ enable_drop_compact_storage: false
 #     row_index_size:
 #         warn_threshold_kb: 0
 #         abort_threshold_kb: 0
+
+# Guardrails settings.
+guardrails:
+
+    # Whether guardrails are enabled or not. Guardrails are disabled by default.
+    enabled: false
+
+    # Guardrail to warn or abort when creating more user tables than threshold.
+    # The two thresholds default to -1 to disable.
+    tables:
+        warn_threshold: -1
+        abort_threshold: -1
+
+    # Guardrail to warn or abort when creating/altering a table with more columns per table than threshold.
+    # The two thresholds default to -1 to disable.
+    columns_per_table:
+        warn_threshold: -1
+        abort_threshold: -1
+
+    # Guardrail to warn or abort when creating more secondary indexes per table than threshold.
+    # The two thresholds default to -1 to disable.
+    secondary_indexes_per_table:
+        warn_threshold: -1
+        abort_threshold: -1
+
+    # Guardrail to warn or abort when creating more materialized views per table than threshold.
+    # The two thresholds default to -1 to disable.
+    materialized_views_per_table:
+        warn_threshold: -1
+        abort_threshold: -1
+
+    # Guardrail to ignore or reject properties when creating tables. By default all properties are allowed.
+    table_properties:
+        ignored: []
+        disallowed: []
+
+    # Guardrail to allow/disallow user-provided timestamps. Defaults to true.
+    user_timestamps_enabled: true

Review comment:
       I am good with the YAML; can you comment out?  We leave defaults commented out; mostly for documentation

##########
File path: src/java/org/apache/cassandra/db/guardrails/Threshold.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.db.guardrails;
+
+import java.util.function.Function;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+
+import org.apache.cassandra.service.ClientState;
+
+/**
+ * A guardrail based on numeric threshold(s).
+ *
+ * <p>A {@link Threshold} guardrail defines (up to) 2 thresholds, one at which a warning is issued, and a higher one
+ * at which the operation is aborted with an exception. Only one of those thresholds can be activated if desired.
+ *
+ * <p>This guardrail only handles guarding positive values.
+ */
+public class Threshold extends Guardrail
+{
+    /**
+     * A {@link Threshold} with both warning and abort thresholds disabled, so that it cannot ever be triggered.
+     */
+    public static final Threshold NEVER_TRIGGERED = new Threshold(state -> Config.DEFAULT, null);

Review comment:
       deadcode (transitively, only user is dead code); can we remove?

##########
File path: src/java/org/apache/cassandra/db/guardrails/GuardrailsConfigService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.db.guardrails;
+
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.GuardrailsOptions;
+import org.apache.cassandra.utils.MBeanWrapper;
+
+/**
+ * Implementation of {@link GuardrailsConfigServiceMBean}.
+ */
+public class GuardrailsConfigService implements GuardrailsConfigServiceMBean
+{
+    public static final String MBEAN_NAME = "org.apache.cassandra.db:type=Guardrails";
+
+    private final GuardrailsOptions config = DatabaseDescriptor.getGuardrailsConfig();
+
+    GuardrailsConfigService()
+    {
+        MBeanWrapper.instance.registerMBean(this, MBEAN_NAME);
+    }
+
+    @Override
+    public boolean getEnabled()
+    {
+        return config.getEnabled();
+    }
+
+    @Override
+    public void setEnabled(boolean enabled)
+    {
+        config.setEnabled(enabled);
+    }
+
+    @Override
+    public int getTablesWarnThreshold()
+    {
+        return (int) config.getTables().getWarnThreshold();
+    }
+
+    @Override
+    public void setTablesWarnThreshold(int threshold)
+    {
+        config.getTables().setWarnThreshold(threshold);
+    }
+
+    @Override
+    public int getTablesAbortThreshold()
+    {
+        return (int) config.getTables().getAbortThreshold();
+    }
+
+    @Override
+    public void setTablesAbortThreshold(int threshold)
+    {
+        config.getTables().setAbortThreshold(threshold);
+    }
+
+    @Override
+    public int getColumnsPerTableWarnThreshold()
+    {
+        return (int) config.getColumnsPerTable().getWarnThreshold();
+    }
+
+    @Override
+    public void setColumnsPerTableWarnThreshold(int threshold)
+    {
+        config.getColumnsPerTable().setWarnThreshold(threshold);
+    }
+
+    @Override
+    public int getColumnsPerTableAbortThreshold()
+    {
+        return (int) config.getColumnsPerTable().getAbortThreshold();
+    }
+
+    @Override
+    public void setColumnsPerTableAbortThreshold(int threshold)
+    {
+        config.getColumnsPerTable().setAbortThreshold(threshold);
+    }
+
+    @Override
+    public int getSecondaryIndexesPerTableWarnThreshold()
+    {
+        return (int) config.getSecondaryIndexesPerTable().getWarnThreshold();
+    }
+
+    @Override
+    public void setSecondaryIndexesPerTableWarnThreshold(int threshold)
+    {
+        config.getSecondaryIndexesPerTable().setWarnThreshold(threshold);
+    }
+
+    @Override
+    public int getSecondaryIndexesPerTableAbortThreshold()
+    {
+        return (int) config.getSecondaryIndexesPerTable().getAbortThreshold();
+    }
+
+    @Override
+    public void setSecondaryIndexesPerTableAbortThreshold(int threshold)
+    {
+        config.getSecondaryIndexesPerTable().setAbortThreshold(threshold);
+    }
+
+    @Override
+    public int getMaterializedViewsPerTableWarnThreshold()
+    {
+        return (int) config.getMaterializedViewsPerTable().getWarnThreshold();
+    }
+
+    @Override
+    public void setMaterializedViewsPerTableWarnThreshold(int threshold)
+    {
+        config.getMaterializedViewsPerTable().setWarnThreshold(threshold);
+    }
+
+    @Override
+    public int getMaterializedViewsPerTableAbortThreshold()
+    {
+        return (int) config.getMaterializedViewsPerTable().getAbortThreshold();
+    }
+
+    @Override
+    public void setMaterializedViewsPerTableAbortThreshold(int threshold)
+    {
+        config.getMaterializedViewsPerTable().setAbortThreshold(threshold);
+    }
+
+    @Override
+    public String getTablePropertiesDisallowed()
+    {
+        return toCSV(config.getTableProperties().getDisallowed());
+    }
+
+    @Override
+    public void setTablePropertiesDisallowed(String properties)
+    {
+        setTablePropertiesDisallowed(fromCSV(properties));
+    }
+
+    public void setTablePropertiesDisallowed(String... properties)
+    {
+        setTablePropertiesDisallowed(ImmutableSet.copyOf(properties));
+    }
+
+    public void setTablePropertiesDisallowed(Set<String> properties)
+    {
+        config.getTableProperties().setDisallowedValues(properties);
+    }
+
+    @Override
+    public String getTablePropertiesIgnored()
+    {
+        return toCSV(config.getTableProperties().getIgnored());
+    }
+
+    @Override
+    public void setTablePropertiesIgnored(String properties)
+    {
+        setTablePropertiesIgnored(fromCSV(properties));
+    }
+
+    public void setTablePropertiesIgnored(String... properties)
+    {
+        setTablePropertiesIgnored(ImmutableSet.copyOf(properties));
+    }
+
+    public void setTablePropertiesIgnored(Set<String> properties)
+    {
+        config.getTableProperties().setIgnoredValues(properties);
+    }

Review comment:
       you could expose via the mbean, many tools do support this.

##########
File path: src/java/org/apache/cassandra/db/guardrails/Threshold.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.db.guardrails;
+
+import java.util.function.Function;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+
+import org.apache.cassandra.service.ClientState;
+
+/**
+ * A guardrail based on numeric threshold(s).
+ *
+ * <p>A {@link Threshold} guardrail defines (up to) 2 thresholds, one at which a warning is issued, and a higher one
+ * at which the operation is aborted with an exception. Only one of those thresholds can be activated if desired.
+ *
+ * <p>This guardrail only handles guarding positive values.
+ */
+public class Threshold extends Guardrail
+{
+    /**
+     * A {@link Threshold} with both warning and abort thresholds disabled, so that it cannot ever be triggered.
+     */
+    public static final Threshold NEVER_TRIGGERED = new Threshold(state -> Config.DEFAULT, null);
+
+    private final Function<ClientState, Config> configProvider;
+    private final ErrorMessageProvider messageProvider;
+
+    /**
+     * Creates a new threshold guardrail.
+     *
+     * @param configProvider  a {@link ClientState}-based provider of {@link Config}s.
+     * @param messageProvider a function to generate the warning or error message if the guardrail is triggered
+     */
+    public Threshold(Function<ClientState, Config> configProvider, ErrorMessageProvider messageProvider)
+    {
+        this.configProvider = configProvider;
+        this.messageProvider = messageProvider;
+    }
+
+    private String errMsg(boolean isWarning, String what, long value, long thresholdValue)
+    {
+        return messageProvider.createMessage(isWarning,
+                                             what,
+                                             value,
+                                             thresholdValue);
+    }
+
+    private long abortValue(Config config)
+    {
+        long abortValue = config.getAbortThreshold();
+        return abortValue < 0 ? Long.MAX_VALUE : abortValue;
+    }
+
+    private long warnValue(Config config)
+    {
+        long warnValue = config.getWarnThreshold();
+        return warnValue < 0 ? Long.MAX_VALUE : warnValue;
+    }
+
+    @Override
+    public boolean enabled(@Nullable ClientState state)
+    {
+        if (!super.enabled(state))
+            return false;
+
+        Config config = configProvider.apply(state);
+        return config.getAbortThreshold() >= 0 || config.getWarnThreshold() >= 0;
+    }
+
+    /**
+     * Checks whether the provided value would trigger a warning or abort the operation if passed to {@link #guard}.
+     *
+     * <p>This method is optional (does not have to be called) but can be used in the case where the "what"
+     * argument to {@link #guard} is expensive to build to save doing so in the common case (of the guardrail
+     * not being triggered).
+     *
+     * @param value The value to test.
+     * @param state The client state, used to skip the check if the query is internal or is done by a superuser.
+     *              A {@code null} value means that the check should be done regardless of the query.
+     * @return {@code true} if {@code value} is above the warning or abort thresholds of this guardrail,
+     * {@code false otherwise}.
+     */
+    public boolean triggersOn(long value, @Nullable ClientState state)
+    {
+        if (!enabled(state))
+            return false;
+
+        Config config = configProvider.apply(state);
+        return value > Math.min(abortValue(config), warnValue(config));
+    }
+
+    /**
+     * Apply the guardrail to the provided value, warning or aborting if appropriate.
+     *
+     * @param value The value to check.
+     * @param what  A string describing what {@code value} is a value of. This is used in the error message if the
+     *              guardrail is triggered. For instance, say the guardrail guards the size of column values, then this
+     *              argument must describe which column of which row is triggering the guardrail for convenience. Note
+     *              that this is only used if the guardrail triggers, so if it is expensive to build, you can put the
+     *              call to this method behind a {@link #triggersOn} call.
+     * @param state The client state, used to skip the check if the query is internal or is done by a superuser.
+     *              A {@code null} value means that the check should be done regardless of the query.
+     */
+    public void guard(long value, String what, @Nullable ClientState state)
+    {
+        if (!enabled(state))
+            return;
+
+        Config config = configProvider.apply(state);
+
+        long abortValue = abortValue(config);
+        if (value > abortValue)
+        {
+            triggerAbort(value, abortValue, what);
+            return;
+        }
+
+        long warnValue = warnValue(config);
+        if (value > warnValue)
+            triggerWarn(value, warnValue, what);
+    }
+
+    private void triggerAbort(long value, long abortValue, String what)
+    {
+        String message = errMsg(false, what, value, abortValue);
+        abort(message);
+    }
+
+    private void triggerWarn(long value, long warnValue, String what)
+    {
+        String message = errMsg(true, what, value, warnValue);
+        warn(message);
+    }
+
+    /**
+     * Creates a new {@link GuardedCounter} guarded by this threshold guardrail.
+     *
+     * @param whatFct A function called when either a warning or abort is triggered by the created counter to
+     *                describe the value. This is equivalent to the {@code what} argument of {@link #guard}, but it is a
+     *                function to allow the output string to be computed lazily (only if a warn/abort ends up being
+     *                triggered).
+     * @param state   The client state, used to skip the check if the query is internal or is done by a superuser.
+     *                A {@code null} value means that the check should be done regardless of the query.
+     * @return the newly created guarded counter
+     */
+    public GuardedCounter newCounter(Supplier<String> whatFct, @Nullable ClientState state)
+    {
+        Threshold threshold = enabled(state) ? this : NEVER_TRIGGERED;
+        return threshold.new GuardedCounter(whatFct, configProvider.apply(state));
+    }
+
+    /**
+     * A function used to build the error message of a triggered {@link Threshold} guardrail.
+     */
+    interface ErrorMessageProvider
+    {
+        /**
+         * Called when the guardrail is triggered to build the corresponding error message.
+         *
+         * @param isWarning Whether the trigger is a warning one; otherwise it is an abort one.
+         * @param what      A string, provided by the call to the {@link #guard} method, describing what the guardrail
+         *                  has been applied to (and that has triggered it).
+         * @param value     The value that triggered the guardrail (as a string).
+         * @param threshold The threshold that was passed to trigger the guardrail (as a string).
+         */
+        String createMessage(boolean isWarning, String what, long value, long threshold);
+    }
+
+    /**
+     * A facility for when the value to be guarded is built incrementally, but we want to abort the operation as soon as
+     * the abort threshold is reached, but only trigger the warning on the final value (and so only if the abort
+     * threshold hasn't also been reached).
+     * <p>
+     * Note that instances are neither thread safe nor reusable.
+     */
+    public class GuardedCounter
+    {
+        private final long warnValue;
+        private final long abortValue;
+        private final Supplier<String> what;
+
+        private long accumulated;
+
+        private GuardedCounter(Supplier<String> what, Config config)
+        {
+            // We capture the warn and abort value at the time of the counter construction to ensure we use
+            // stable value during the counter lifetime (and reading a final field is possibly at tad faster).
+            this.warnValue = warnValue(config);
+            this.abortValue = abortValue(config);
+            this.what = what;
+        }
+
+        /**
+         * The currently accumulated value of the counter.
+         */
+        public long get()
+        {
+            return accumulated;
+        }
+
+        /**
+         * Add the provided increment to the counter, aborting the operation if the counter after this addition crosses
+         * the abort threshold.
+         *
+         * @param increment the increment to add
+         */
+        public void add(long increment)
+        {
+            accumulated += increment;
+            if (accumulated > abortValue)
+                triggerAbort(accumulated, abortValue, what.get());
+        }
+
+        /**
+         * Trigger the warn if the currently accumulated counter value crosses warning threshold and the abort threshold
+         * has not been crossed yet.
+         * <p>
+         * This is generally meant to be called when the guarded value is complete.
+         *
+         * @return {@code true} and trigger a warning if the current counter value is greater than the warning
+         * threshold and lesser than or equal to the abort threshold, {@code false} otherwise.
+         */
+        public boolean checkAndTriggerWarning()
+        {
+            if (accumulated > warnValue && accumulated <= abortValue)
+            {
+                triggerWarn(accumulated, warnValue, what.get());
+                return true;
+            }
+            return false;
+        }
+    }
+
+    /**
+     * Configuration class containing the thresholds to be used to check if the guarded value should trigger a warning
+     * or abort the operation.
+     */
+    public interface Config
+    {
+        public static Config DEFAULT = new Config()
+        {
+            @Override
+            public long getWarnThreshold()
+            {
+                return -1;
+            }
+
+            @Override
+            public long getAbortThreshold()
+            {
+                return -1;
+            }
+        };

Review comment:
       deadcode (transitively, caller of caller is not used); can we remove?

##########
File path: src/java/org/apache/cassandra/config/GuardrailsOptions.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.config;
+
+import java.util.Collections;
+import java.util.Set;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.Sets;
+
+import org.apache.cassandra.cql3.statements.schema.TableAttributes;
+import org.apache.cassandra.db.guardrails.Guardrails;
+import org.apache.cassandra.db.guardrails.GuardrailsConfig;
+import org.apache.cassandra.db.guardrails.Values;
+
+import static java.lang.String.format;
+import static java.util.stream.Collectors.toSet;
+
+/**
+ * Configuration settings for guardrails populated from the Yaml file.
+ *
+ * <p>Note that the settings here must only be used to build the {@link GuardrailsConfig} class and not directly by the
+ * code checking each guarded constraint. That code should use the higher level abstractions defined in
+ * {@link Guardrails}).
+ *
+ * <p>This contains a main setting, {@code enabled}, controlling if guardrails are globally active or not, and
+ * individual settings to control each guardrail.
+ *
+ * <p>We have 2 variants of guardrails, soft (warn) and hard (abort) limits, each guardrail having either one of the
+ * variants or both. Note in particular that hard limits only make sense for guardrails triggering during query
+ * execution. For other guardrails, say one triggering during compaction, aborting that compaction does not make sense.
+ *
+ * <p>Additionally, each individual setting should have a specific value (typically -1 for numeric settings),
+ * that allows to disable the corresponding guardrail.
+ */
+public class GuardrailsOptions implements GuardrailsConfig
+{
+    public volatile boolean enabled = false;
+    public final IntThreshold tables = new IntThreshold();
+    public final IntThreshold columns_per_table = new IntThreshold();
+    public final IntThreshold secondary_indexes_per_table = new IntThreshold();
+    public final IntThreshold materialized_views_per_table = new IntThreshold();
+    public final TableProperties table_properties = new TableProperties();
+    public volatile boolean user_timestamps_enabled = true;
+
+    public void validate()
+    {
+        tables.validate("guardrails.tables");
+        columns_per_table.validate("guardrails.columns_per_table");
+        secondary_indexes_per_table.validate("guardrails.secondary_indexes_per_table");
+        materialized_views_per_table.validate("guardrails.materialized_views_per_table");
+        table_properties.validate("guardrails.table_properties");
+    }
+
+    @Override
+    public boolean getEnabled()
+    {
+        return enabled;
+    }
+
+    /**
+     * Enable/disable guardrails.
+     *
+     * @param enabled {@code true} for enabling guardrails, {@code false} for disabling them.
+     */
+    public void setEnabled(boolean enabled)
+    {
+        validateNotNull(enabled, "guardrails.enabled");
+        this.enabled = enabled;
+    }
+
+    @Override
+    public IntThreshold getTables()
+    {
+        return tables;
+    }
+
+    @Override
+    public IntThreshold getColumnsPerTable()
+    {
+        return columns_per_table;
+    }
+
+    @Override
+    public IntThreshold getSecondaryIndexesPerTable()
+    {
+        return secondary_indexes_per_table;
+    }
+
+    @Override
+    public IntThreshold getMaterializedViewsPerTable()
+    {
+        return materialized_views_per_table;
+    }
+
+    @Override
+    public TableProperties getTableProperties()
+    {
+        return table_properties;
+    }
+
+    @Override
+    public boolean getUserTimestampsEnabled()
+    {
+        return user_timestamps_enabled;
+    }
+
+    public void setUserTimestampsEnabled(boolean enabled)
+    {
+        validateNotNull(enabled, "guardrails.user_timestamps_enabled");
+        user_timestamps_enabled = enabled;
+    }
+
+    private static <T> void validateNotNull(T value, String name)
+    {
+        if (value == null)
+            throw new IllegalArgumentException(format("Invalid value for %s: null is not allowed", name));
+    }
+
+    public static abstract class Threshold implements org.apache.cassandra.db.guardrails.Threshold.Config
+    {
+        protected void validatePositiveNumeric(long value, long maxValue, boolean allowZero, String name)
+        {
+            validateNotNull(value, name);
+
+            if (value > maxValue)
+                throw new IllegalArgumentException(format("Invalid value %d for %s: maximum allowed value is %d",
+                                                          value, name, maxValue));
+
+            if (value == 0 && !allowZero)
+                throw new IllegalArgumentException(format("Invalid value for %s: 0 is not allowed", name));
+
+            // We allow -1 as a general "disabling" flag. But reject anything lower to avoid mistakes.
+            if (value < -1L)
+                throw new IllegalArgumentException(format("Invalid value %d for %s: negative values are not allowed, " +
+                                                          "outside of -1 which disables the guardrail",
+                                                          value, name));
+        }
+
+        protected void validateWarnLowerThanAbort(long warnValue, long abortValue, @Nullable String name)

Review comment:
       JMX is single field but we check this early; if we are trying to shrink we need to be very careful about how we do that; maybe JMX should offer a `set(warn, abort)` so we don't need to check this twice and have that ordering issue?
   
   Example
   
   ```
   // yaml
   warn=100
   abort=200
   // jmx set abort
   abort=70 // currently fails
   // jmx set both
   warn=50, abort=70 // success
   ```

##########
File path: src/java/org/apache/cassandra/config/GuardrailsOptions.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.config;
+
+import java.util.Collections;
+import java.util.Set;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.Sets;
+
+import org.apache.cassandra.cql3.statements.schema.TableAttributes;
+import org.apache.cassandra.db.guardrails.Guardrails;
+import org.apache.cassandra.db.guardrails.GuardrailsConfig;
+import org.apache.cassandra.db.guardrails.Values;
+
+import static java.lang.String.format;
+import static java.util.stream.Collectors.toSet;
+
+/**
+ * Configuration settings for guardrails populated from the Yaml file.
+ *
+ * <p>Note that the settings here must only be used to build the {@link GuardrailsConfig} class and not directly by the
+ * code checking each guarded constraint. That code should use the higher level abstractions defined in
+ * {@link Guardrails}).
+ *
+ * <p>This contains a main setting, {@code enabled}, controlling if guardrails are globally active or not, and
+ * individual settings to control each guardrail.
+ *
+ * <p>We have 2 variants of guardrails, soft (warn) and hard (abort) limits, each guardrail having either one of the
+ * variants or both. Note in particular that hard limits only make sense for guardrails triggering during query
+ * execution. For other guardrails, say one triggering during compaction, aborting that compaction does not make sense.
+ *
+ * <p>Additionally, each individual setting should have a specific value (typically -1 for numeric settings),
+ * that allows to disable the corresponding guardrail.
+ */
+public class GuardrailsOptions implements GuardrailsConfig
+{
+    public volatile boolean enabled = false;
+    public final IntThreshold tables = new IntThreshold();
+    public final IntThreshold columns_per_table = new IntThreshold();
+    public final IntThreshold secondary_indexes_per_table = new IntThreshold();
+    public final IntThreshold materialized_views_per_table = new IntThreshold();
+    public final TableProperties table_properties = new TableProperties();
+    public volatile boolean user_timestamps_enabled = true;
+
+    public void validate()
+    {
+        tables.validate("guardrails.tables");
+        columns_per_table.validate("guardrails.columns_per_table");
+        secondary_indexes_per_table.validate("guardrails.secondary_indexes_per_table");
+        materialized_views_per_table.validate("guardrails.materialized_views_per_table");
+        table_properties.validate("guardrails.table_properties");
+    }
+
+    @Override
+    public boolean getEnabled()
+    {
+        return enabled;
+    }
+
+    /**
+     * Enable/disable guardrails.
+     *
+     * @param enabled {@code true} for enabling guardrails, {@code false} for disabling them.
+     */
+    public void setEnabled(boolean enabled)
+    {
+        validateNotNull(enabled, "guardrails.enabled");

Review comment:
       not possible

##########
File path: src/java/org/apache/cassandra/db/guardrails/Guardrails.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.db.guardrails;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.service.ClientState;
+
+import static java.lang.String.format;
+
+/**
+ * Entry point for Guardrails, storing the defined guardrails and providing a few global methods over them.
+ */
+public abstract class Guardrails

Review comment:
       nothing extends this; if you want a util class that nothing can init, you can do the following
   
   ```
   public final class Guardrails
   {
     private Guardrails()
     {
     }
   ...
   ```

##########
File path: src/java/org/apache/cassandra/config/GuardrailsOptions.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.config;
+
+import java.util.Collections;
+import java.util.Set;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.Sets;
+
+import org.apache.cassandra.cql3.statements.schema.TableAttributes;
+import org.apache.cassandra.db.guardrails.Guardrails;
+import org.apache.cassandra.db.guardrails.GuardrailsConfig;
+import org.apache.cassandra.db.guardrails.Values;
+
+import static java.lang.String.format;
+import static java.util.stream.Collectors.toSet;
+
+/**
+ * Configuration settings for guardrails populated from the Yaml file.
+ *
+ * <p>Note that the settings here must only be used to build the {@link GuardrailsConfig} class and not directly by the
+ * code checking each guarded constraint. That code should use the higher level abstractions defined in
+ * {@link Guardrails}).
+ *
+ * <p>This contains a main setting, {@code enabled}, controlling if guardrails are globally active or not, and
+ * individual settings to control each guardrail.
+ *
+ * <p>We have 2 variants of guardrails, soft (warn) and hard (abort) limits, each guardrail having either one of the
+ * variants or both. Note in particular that hard limits only make sense for guardrails triggering during query
+ * execution. For other guardrails, say one triggering during compaction, aborting that compaction does not make sense.
+ *
+ * <p>Additionally, each individual setting should have a specific value (typically -1 for numeric settings),
+ * that allows to disable the corresponding guardrail.
+ */
+public class GuardrailsOptions implements GuardrailsConfig
+{
+    public volatile boolean enabled = false;
+    public final IntThreshold tables = new IntThreshold();
+    public final IntThreshold columns_per_table = new IntThreshold();
+    public final IntThreshold secondary_indexes_per_table = new IntThreshold();
+    public final IntThreshold materialized_views_per_table = new IntThreshold();
+    public final TableProperties table_properties = new TableProperties();
+    public volatile boolean user_timestamps_enabled = true;
+
+    public void validate()
+    {
+        tables.validate("guardrails.tables");
+        columns_per_table.validate("guardrails.columns_per_table");
+        secondary_indexes_per_table.validate("guardrails.secondary_indexes_per_table");
+        materialized_views_per_table.validate("guardrails.materialized_views_per_table");
+        table_properties.validate("guardrails.table_properties");
+    }
+
+    @Override
+    public boolean getEnabled()
+    {
+        return enabled;
+    }
+
+    /**
+     * Enable/disable guardrails.
+     *
+     * @param enabled {@code true} for enabling guardrails, {@code false} for disabling them.
+     */
+    public void setEnabled(boolean enabled)
+    {
+        validateNotNull(enabled, "guardrails.enabled");
+        this.enabled = enabled;
+    }
+
+    @Override
+    public IntThreshold getTables()
+    {
+        return tables;
+    }
+
+    @Override
+    public IntThreshold getColumnsPerTable()
+    {
+        return columns_per_table;
+    }
+
+    @Override
+    public IntThreshold getSecondaryIndexesPerTable()
+    {
+        return secondary_indexes_per_table;
+    }
+
+    @Override
+    public IntThreshold getMaterializedViewsPerTable()
+    {
+        return materialized_views_per_table;
+    }
+
+    @Override
+    public TableProperties getTableProperties()
+    {
+        return table_properties;
+    }
+
+    @Override
+    public boolean getUserTimestampsEnabled()
+    {
+        return user_timestamps_enabled;
+    }
+
+    public void setUserTimestampsEnabled(boolean enabled)
+    {
+        validateNotNull(enabled, "guardrails.user_timestamps_enabled");
+        user_timestamps_enabled = enabled;
+    }
+
+    private static <T> void validateNotNull(T value, String name)

Review comment:
       the only valid user is `org.apache.cassandra.config.GuardrailsOptions.TableProperties#validateTableProperties`

##########
File path: src/java/org/apache/cassandra/db/guardrails/Predicates.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.db.guardrails;
+
+import java.util.function.Function;
+import java.util.function.Predicate;
+import javax.annotation.Nullable;
+
+import org.apache.cassandra.service.ClientState;
+
+/**
+ * A guardrail based on two warn/abort predicates applied to the guarded value.
+ *
+ * @param <T> The type of the values to be tested.
+ */
+public class Predicates<T> extends Guardrail

Review comment:
       deadcode, can we drop?

##########
File path: src/java/org/apache/cassandra/db/guardrails/GuardrailsConfigService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.db.guardrails;
+
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.GuardrailsOptions;
+import org.apache.cassandra.utils.MBeanWrapper;
+
+/**
+ * Implementation of {@link GuardrailsConfigServiceMBean}.
+ */
+public class GuardrailsConfigService implements GuardrailsConfigServiceMBean
+{
+    public static final String MBEAN_NAME = "org.apache.cassandra.db:type=Guardrails";
+
+    private final GuardrailsOptions config = DatabaseDescriptor.getGuardrailsConfig();
+
+    GuardrailsConfigService()
+    {
+        MBeanWrapper.instance.registerMBean(this, MBEAN_NAME);
+    }
+
+    @Override
+    public boolean getEnabled()
+    {
+        return config.getEnabled();
+    }
+
+    @Override
+    public void setEnabled(boolean enabled)
+    {
+        config.setEnabled(enabled);
+    }
+
+    @Override
+    public int getTablesWarnThreshold()
+    {
+        return (int) config.getTables().getWarnThreshold();
+    }
+
+    @Override
+    public void setTablesWarnThreshold(int threshold)
+    {
+        config.getTables().setWarnThreshold(threshold);
+    }
+
+    @Override
+    public int getTablesAbortThreshold()
+    {
+        return (int) config.getTables().getAbortThreshold();
+    }
+
+    @Override
+    public void setTablesAbortThreshold(int threshold)
+    {
+        config.getTables().setAbortThreshold(threshold);
+    }
+
+    @Override
+    public int getColumnsPerTableWarnThreshold()
+    {
+        return (int) config.getColumnsPerTable().getWarnThreshold();
+    }
+
+    @Override
+    public void setColumnsPerTableWarnThreshold(int threshold)
+    {
+        config.getColumnsPerTable().setWarnThreshold(threshold);
+    }
+
+    @Override
+    public int getColumnsPerTableAbortThreshold()
+    {
+        return (int) config.getColumnsPerTable().getAbortThreshold();
+    }
+
+    @Override
+    public void setColumnsPerTableAbortThreshold(int threshold)
+    {
+        config.getColumnsPerTable().setAbortThreshold(threshold);
+    }
+
+    @Override
+    public int getSecondaryIndexesPerTableWarnThreshold()
+    {
+        return (int) config.getSecondaryIndexesPerTable().getWarnThreshold();
+    }
+
+    @Override
+    public void setSecondaryIndexesPerTableWarnThreshold(int threshold)
+    {
+        config.getSecondaryIndexesPerTable().setWarnThreshold(threshold);
+    }
+
+    @Override
+    public int getSecondaryIndexesPerTableAbortThreshold()
+    {
+        return (int) config.getSecondaryIndexesPerTable().getAbortThreshold();
+    }
+
+    @Override
+    public void setSecondaryIndexesPerTableAbortThreshold(int threshold)
+    {
+        config.getSecondaryIndexesPerTable().setAbortThreshold(threshold);
+    }
+
+    @Override
+    public int getMaterializedViewsPerTableWarnThreshold()
+    {
+        return (int) config.getMaterializedViewsPerTable().getWarnThreshold();
+    }
+
+    @Override
+    public void setMaterializedViewsPerTableWarnThreshold(int threshold)
+    {
+        config.getMaterializedViewsPerTable().setWarnThreshold(threshold);
+    }
+
+    @Override
+    public int getMaterializedViewsPerTableAbortThreshold()
+    {
+        return (int) config.getMaterializedViewsPerTable().getAbortThreshold();
+    }
+
+    @Override
+    public void setMaterializedViewsPerTableAbortThreshold(int threshold)
+    {
+        config.getMaterializedViewsPerTable().setAbortThreshold(threshold);
+    }
+
+    @Override
+    public String getTablePropertiesDisallowed()
+    {
+        return toCSV(config.getTableProperties().getDisallowed());
+    }
+
+    @Override
+    public void setTablePropertiesDisallowed(String properties)
+    {
+        setTablePropertiesDisallowed(fromCSV(properties));
+    }
+
+    public void setTablePropertiesDisallowed(String... properties)
+    {
+        setTablePropertiesDisallowed(ImmutableSet.copyOf(properties));
+    }
+
+    public void setTablePropertiesDisallowed(Set<String> properties)
+    {
+        config.getTableProperties().setDisallowedValues(properties);
+    }

Review comment:
       you could expose via the mbean, many tools do support this.

##########
File path: src/java/org/apache/cassandra/db/guardrails/GuardrailsConfigService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.db.guardrails;
+
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.GuardrailsOptions;
+import org.apache.cassandra.utils.MBeanWrapper;
+
+/**
+ * Implementation of {@link GuardrailsConfigServiceMBean}.
+ */
+public class GuardrailsConfigService implements GuardrailsConfigServiceMBean
+{
+    public static final String MBEAN_NAME = "org.apache.cassandra.db:type=Guardrails";
+
+    private final GuardrailsOptions config = DatabaseDescriptor.getGuardrailsConfig();
+
+    GuardrailsConfigService()
+    {
+        MBeanWrapper.instance.registerMBean(this, MBEAN_NAME);
+    }
+
+    @Override
+    public boolean getEnabled()
+    {
+        return config.getEnabled();
+    }
+
+    @Override
+    public void setEnabled(boolean enabled)
+    {
+        config.setEnabled(enabled);
+    }
+
+    @Override
+    public int getTablesWarnThreshold()
+    {
+        return (int) config.getTables().getWarnThreshold();

Review comment:
       nit: In TrackWarnings we just relied on a different type for int and long; that way casting isn't needed and there is no confusion about min/max values

##########
File path: src/java/org/apache/cassandra/db/guardrails/Guardrails.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.db.guardrails;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.service.ClientState;
+
+import static java.lang.String.format;
+
+/**
+ * Entry point for Guardrails, storing the defined guardrails and providing a few global methods over them.
+ */
+public abstract class Guardrails

Review comment:
       Why don't you merge this class with `GuardrailsConfigService`?  Seems simpler and solves the "who allocates `GuardrailsConfigService` for the mbean" problem. 

##########
File path: src/java/org/apache/cassandra/db/guardrails/Threshold.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.db.guardrails;
+
+import java.util.function.Function;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+
+import org.apache.cassandra.service.ClientState;
+
+/**
+ * A guardrail based on numeric threshold(s).
+ *
+ * <p>A {@link Threshold} guardrail defines (up to) 2 thresholds, one at which a warning is issued, and a higher one
+ * at which the operation is aborted with an exception. Only one of those thresholds can be activated if desired.
+ *
+ * <p>This guardrail only handles guarding positive values.
+ */
+public class Threshold extends Guardrail
+{
+    /**
+     * A {@link Threshold} with both warning and abort thresholds disabled, so that it cannot ever be triggered.
+     */
+    public static final Threshold NEVER_TRIGGERED = new Threshold(state -> Config.DEFAULT, null);
+
+    private final Function<ClientState, Config> configProvider;
+    private final ErrorMessageProvider messageProvider;
+
+    /**
+     * Creates a new threshold guardrail.
+     *
+     * @param configProvider  a {@link ClientState}-based provider of {@link Config}s.
+     * @param messageProvider a function to generate the warning or error message if the guardrail is triggered
+     */
+    public Threshold(Function<ClientState, Config> configProvider, ErrorMessageProvider messageProvider)
+    {
+        this.configProvider = configProvider;
+        this.messageProvider = messageProvider;
+    }
+
+    private String errMsg(boolean isWarning, String what, long value, long thresholdValue)
+    {
+        return messageProvider.createMessage(isWarning,
+                                             what,
+                                             value,
+                                             thresholdValue);
+    }
+
+    private long abortValue(Config config)
+    {
+        long abortValue = config.getAbortThreshold();
+        return abortValue < 0 ? Long.MAX_VALUE : abortValue;
+    }
+
+    private long warnValue(Config config)
+    {
+        long warnValue = config.getWarnThreshold();
+        return warnValue < 0 ? Long.MAX_VALUE : warnValue;
+    }
+
+    @Override
+    public boolean enabled(@Nullable ClientState state)
+    {
+        if (!super.enabled(state))
+            return false;
+
+        Config config = configProvider.apply(state);
+        return config.getAbortThreshold() >= 0 || config.getWarnThreshold() >= 0;
+    }
+
+    /**
+     * Checks whether the provided value would trigger a warning or abort the operation if passed to {@link #guard}.
+     *
+     * <p>This method is optional (does not have to be called) but can be used in the case where the "what"
+     * argument to {@link #guard} is expensive to build to save doing so in the common case (of the guardrail
+     * not being triggered).
+     *
+     * @param value The value to test.
+     * @param state The client state, used to skip the check if the query is internal or is done by a superuser.
+     *              A {@code null} value means that the check should be done regardless of the query.
+     * @return {@code true} if {@code value} is above the warning or abort thresholds of this guardrail,
+     * {@code false otherwise}.
+     */
+    public boolean triggersOn(long value, @Nullable ClientState state)
+    {
+        if (!enabled(state))
+            return false;
+
+        Config config = configProvider.apply(state);
+        return value > Math.min(abortValue(config), warnValue(config));
+    }
+
+    /**
+     * Apply the guardrail to the provided value, warning or aborting if appropriate.
+     *
+     * @param value The value to check.
+     * @param what  A string describing what {@code value} is a value of. This is used in the error message if the
+     *              guardrail is triggered. For instance, say the guardrail guards the size of column values, then this
+     *              argument must describe which column of which row is triggering the guardrail for convenience. Note
+     *              that this is only used if the guardrail triggers, so if it is expensive to build, you can put the
+     *              call to this method behind a {@link #triggersOn} call.
+     * @param state The client state, used to skip the check if the query is internal or is done by a superuser.
+     *              A {@code null} value means that the check should be done regardless of the query.
+     */
+    public void guard(long value, String what, @Nullable ClientState state)
+    {
+        if (!enabled(state))
+            return;
+
+        Config config = configProvider.apply(state);
+
+        long abortValue = abortValue(config);
+        if (value > abortValue)
+        {
+            triggerAbort(value, abortValue, what);
+            return;
+        }
+
+        long warnValue = warnValue(config);
+        if (value > warnValue)
+            triggerWarn(value, warnValue, what);
+    }
+
+    private void triggerAbort(long value, long abortValue, String what)
+    {
+        String message = errMsg(false, what, value, abortValue);
+        abort(message);
+    }
+
+    private void triggerWarn(long value, long warnValue, String what)
+    {
+        String message = errMsg(true, what, value, warnValue);
+        warn(message);
+    }
+
+    /**
+     * Creates a new {@link GuardedCounter} guarded by this threshold guardrail.
+     *
+     * @param whatFct A function called when either a warning or abort is triggered by the created counter to
+     *                describe the value. This is equivalent to the {@code what} argument of {@link #guard}, but it is a
+     *                function to allow the output string to be computed lazily (only if a warn/abort ends up being
+     *                triggered).
+     * @param state   The client state, used to skip the check if the query is internal or is done by a superuser.
+     *                A {@code null} value means that the check should be done regardless of the query.
+     * @return the newly created guarded counter
+     */
+    public GuardedCounter newCounter(Supplier<String> whatFct, @Nullable ClientState state)
+    {
+        Threshold threshold = enabled(state) ? this : NEVER_TRIGGERED;
+        return threshold.new GuardedCounter(whatFct, configProvider.apply(state));
+    }

Review comment:
       deadcode; can we remove?

##########
File path: src/java/org/apache/cassandra/db/guardrails/Values.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.db.guardrails;
+
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.Sets;
+
+import org.apache.cassandra.service.ClientState;
+
+import static java.lang.String.format;
+
+/**
+ * A guardrail that warns about but ignores some specific values, and rejects the use of some other values.
+ *
+ * @param <T> The type of the values of which certain are disallowed.
+ */
+public class Values<T> extends Guardrail
+{
+    private final Function<ClientState, Config<T>> configProvider;
+    private final String what;
+
+    /**
+     * Creates a new values guardrail.
+     *
+     * @param configProvider a {@link ClientState}-based provider of {@link Config}s.
+     * @param what           The feature that is guarded by this guardrail (for reporting in error messages).
+     */
+    public Values(Function<ClientState, Config<T>> configProvider, String what)
+    {
+        this.configProvider = configProvider;
+        this.what = what;
+    }
+
+    /**
+     * Checks whether the provided value would trigger this guardrail.
+     *
+     * <p>This method is optional (does not have to be called) but can be used in the case some of the arguments
+     * to the actual guardrail method are expensive to build to save doing so in the common case of the guardrail not
+     * being triggered.
+     *
+     * @param value The value to test.
+     * @param state The client state, used to skip the check if the query is internal or is done by a superuser.
+     * @return {@code true} if {@code value} is not allowed by this guardrail,
+     * {@code false otherwise}.
+     */
+    public boolean triggersOn(T value, @Nullable ClientState state)
+    {
+        if (!enabled(state))
+            return false;
+
+        Config<T> config = configProvider.apply(state);
+        return config.getIgnored().contains(value) || config.getDisallowed().contains(value);
+    }
+
+    /**
+     * Triggers a warning if the provided value is disallowed by this guardrail and triggers an action to ignore it.
+     * If the value is disallowed it will abort the operation.
+     *
+     * @param value        The value to check.
+     * @param ignoreAction An action called on the subset of {@code values} that should be ignored. This action
+     *                     should do whatever is necessary to make sure the value is ignored.
+     * @param state        The client state, used to skip the check if the query is internal or is done by a superuser.
+     *                     A {@code null} value means that the check should be done regardless of the query.
+     */
+    public void guard(T value, Consumer<T> ignoreAction, @Nullable ClientState state)
+    {
+        if (!enabled(state))
+            return;
+
+        Config<T> config = configProvider.apply(state);
+
+        Set<T> ignored = config.getIgnored();
+        if (ignored.contains(value))
+        {
+            warn(format("Ignoring provided value %s as it is not supported for %s (ignored values are: %s)",
+                        value, what, ignored.toString()));
+            ignoreAction.accept(value);
+        }
+
+        Set<T> disallowed = config.getDisallowed();
+        if (disallowed.contains(value))
+            abort(format("Provided value %s is not allowed for %s (disallowed values are: %s)",
+                         value, what, disallowed.toString()));
+    }
+
+    /**
+     * Triggers a warning for each of the provided values that are disallowed by this guardrail and triggers an action
+     * to ignore them. If the values are disallowed it will abort the operation.
+     *
+     * @param values       The values to check.
+     * @param ignoreAction An action called on the subset of {@code values} that should be ignored. This action
+     *                     should do whatever is necessary to make sure the value is ignored.
+     * @param state        The client state, used to skip the check if the query is internal or is done by a superuser.
+     *                     A {@code null} value means that the check should be done regardless of the query.
+     */
+    public void guard(Set<T> values, Consumer<T> ignoreAction, @Nullable ClientState state)
+    {
+        if (!enabled(state))
+            return;
+
+        Config<T> config = configProvider.apply(state);
+
+        Set<T> ignored = config.getIgnored();
+        Set<T> toIgnore = Sets.intersection(values, ignored);
+        if (!toIgnore.isEmpty())
+        {
+            warn(format("Ignoring provided values %s as they are not supported for %s (ignored values are: %s)",
+                        toIgnore.stream().sorted().collect(Collectors.toList()), what, ignored.toString()));
+            toIgnore.forEach(ignoreAction);
+        }
+
+        Set<T> disallowed = config.getDisallowed();
+        Set<T> toDisallow = Sets.intersection(values, disallowed);
+        if (!toDisallow.isEmpty())
+            abort(format("Provided values %s are not allowed for %s (disallowed values are: %s)",
+                         toDisallow.stream().sorted().collect(Collectors.toList()), what, disallowed.toString()));

Review comment:
       can we do disallow before ignore?  ignore mutates, so it could be a problem in the future if we mutate then abort

##########
File path: src/java/org/apache/cassandra/db/guardrails/Threshold.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.db.guardrails;
+
+import java.util.function.Function;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+
+import org.apache.cassandra.service.ClientState;
+
+/**
+ * A guardrail based on numeric threshold(s).
+ *
+ * <p>A {@link Threshold} guardrail defines (up to) 2 thresholds, one at which a warning is issued, and a higher one
+ * at which the operation is aborted with an exception. Only one of those thresholds can be activated if desired.
+ *
+ * <p>This guardrail only handles guarding positive values.
+ */
+public class Threshold extends Guardrail
+{
+    /**
+     * A {@link Threshold} with both warning and abort thresholds disabled, so that it cannot ever be triggered.
+     */
+    public static final Threshold NEVER_TRIGGERED = new Threshold(state -> Config.DEFAULT, null);
+
+    private final Function<ClientState, Config> configProvider;
+    private final ErrorMessageProvider messageProvider;
+
+    /**
+     * Creates a new threshold guardrail.
+     *
+     * @param configProvider  a {@link ClientState}-based provider of {@link Config}s.
+     * @param messageProvider a function to generate the warning or error message if the guardrail is triggered
+     */
+    public Threshold(Function<ClientState, Config> configProvider, ErrorMessageProvider messageProvider)
+    {
+        this.configProvider = configProvider;
+        this.messageProvider = messageProvider;
+    }
+
+    private String errMsg(boolean isWarning, String what, long value, long thresholdValue)
+    {
+        return messageProvider.createMessage(isWarning,
+                                             what,
+                                             value,
+                                             thresholdValue);
+    }
+
+    private long abortValue(Config config)
+    {
+        long abortValue = config.getAbortThreshold();
+        return abortValue < 0 ? Long.MAX_VALUE : abortValue;
+    }
+
+    private long warnValue(Config config)
+    {
+        long warnValue = config.getWarnThreshold();
+        return warnValue < 0 ? Long.MAX_VALUE : warnValue;
+    }
+
+    @Override
+    public boolean enabled(@Nullable ClientState state)
+    {
+        if (!super.enabled(state))
+            return false;
+
+        Config config = configProvider.apply(state);
+        return config.getAbortThreshold() >= 0 || config.getWarnThreshold() >= 0;
+    }
+
+    /**
+     * Checks whether the provided value would trigger a warning or abort the operation if passed to {@link #guard}.
+     *
+     * <p>This method is optional (does not have to be called) but can be used in the case where the "what"
+     * argument to {@link #guard} is expensive to build to save doing so in the common case (of the guardrail
+     * not being triggered).
+     *
+     * @param value The value to test.
+     * @param state The client state, used to skip the check if the query is internal or is done by a superuser.
+     *              A {@code null} value means that the check should be done regardless of the query.
+     * @return {@code true} if {@code value} is above the warning or abort thresholds of this guardrail,
+     * {@code false otherwise}.
+     */
+    public boolean triggersOn(long value, @Nullable ClientState state)
+    {
+        if (!enabled(state))
+            return false;
+
+        Config config = configProvider.apply(state);
+        return value > Math.min(abortValue(config), warnValue(config));
+    }
+
+    /**
+     * Apply the guardrail to the provided value, warning or aborting if appropriate.
+     *
+     * @param value The value to check.
+     * @param what  A string describing what {@code value} is a value of. This is used in the error message if the
+     *              guardrail is triggered. For instance, say the guardrail guards the size of column values, then this
+     *              argument must describe which column of which row is triggering the guardrail for convenience. Note
+     *              that this is only used if the guardrail triggers, so if it is expensive to build, you can put the
+     *              call to this method behind a {@link #triggersOn} call.
+     * @param state The client state, used to skip the check if the query is internal or is done by a superuser.
+     *              A {@code null} value means that the check should be done regardless of the query.
+     */
+    public void guard(long value, String what, @Nullable ClientState state)
+    {
+        if (!enabled(state))
+            return;
+
+        Config config = configProvider.apply(state);
+
+        long abortValue = abortValue(config);
+        if (value > abortValue)
+        {
+            triggerAbort(value, abortValue, what);
+            return;
+        }
+
+        long warnValue = warnValue(config);
+        if (value > warnValue)
+            triggerWarn(value, warnValue, what);
+    }
+
+    private void triggerAbort(long value, long abortValue, String what)
+    {
+        String message = errMsg(false, what, value, abortValue);
+        abort(message);
+    }
+
+    private void triggerWarn(long value, long warnValue, String what)
+    {
+        String message = errMsg(true, what, value, warnValue);
+        warn(message);
+    }

Review comment:
       nit: can remove the tmp variable and directly pass to the function.

##########
File path: src/java/org/apache/cassandra/db/guardrails/Values.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.db.guardrails;
+
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.Sets;
+
+import org.apache.cassandra.service.ClientState;
+
+import static java.lang.String.format;
+
+/**
+ * A guardrail that warns about but ignores some specific values, and rejects the use of some other values.
+ *
+ * @param <T> The type of the values of which certain are disallowed.
+ */
+public class Values<T> extends Guardrail
+{
+    private final Function<ClientState, Config<T>> configProvider;
+    private final String what;
+
+    /**
+     * Creates a new values guardrail.
+     *
+     * @param configProvider a {@link ClientState}-based provider of {@link Config}s.
+     * @param what           The feature that is guarded by this guardrail (for reporting in error messages).
+     */
+    public Values(Function<ClientState, Config<T>> configProvider, String what)
+    {
+        this.configProvider = configProvider;
+        this.what = what;
+    }
+
+    /**
+     * Checks whether the provided value would trigger this guardrail.
+     *
+     * <p>This method is optional (does not have to be called) but can be used in the case some of the arguments
+     * to the actual guardrail method are expensive to build to save doing so in the common case of the guardrail not
+     * being triggered.
+     *
+     * @param value The value to test.
+     * @param state The client state, used to skip the check if the query is internal or is done by a superuser.
+     * @return {@code true} if {@code value} is not allowed by this guardrail,
+     * {@code false otherwise}.
+     */
+    public boolean triggersOn(T value, @Nullable ClientState state)
+    {
+        if (!enabled(state))
+            return false;
+
+        Config<T> config = configProvider.apply(state);
+        return config.getIgnored().contains(value) || config.getDisallowed().contains(value);
+    }

Review comment:
       deadcode; can we remove?

##########
File path: src/java/org/apache/cassandra/db/guardrails/Threshold.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.db.guardrails;
+
+import java.util.function.Function;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+
+import org.apache.cassandra.service.ClientState;
+
+/**
+ * A guardrail based on numeric threshold(s).
+ *
+ * <p>A {@link Threshold} guardrail defines (up to) 2 thresholds, one at which a warning is issued, and a higher one
+ * at which the operation is aborted with an exception. Only one of those thresholds can be activated if desired.
+ *
+ * <p>This guardrail only handles guarding positive values.
+ */
+public class Threshold extends Guardrail
+{
+    /**
+     * A {@link Threshold} with both warning and abort thresholds disabled, so that it cannot ever be triggered.
+     */
+    public static final Threshold NEVER_TRIGGERED = new Threshold(state -> Config.DEFAULT, null);
+
+    private final Function<ClientState, Config> configProvider;
+    private final ErrorMessageProvider messageProvider;
+
+    /**
+     * Creates a new threshold guardrail.
+     *
+     * @param configProvider  a {@link ClientState}-based provider of {@link Config}s.
+     * @param messageProvider a function to generate the warning or error message if the guardrail is triggered
+     */
+    public Threshold(Function<ClientState, Config> configProvider, ErrorMessageProvider messageProvider)
+    {
+        this.configProvider = configProvider;
+        this.messageProvider = messageProvider;
+    }
+
+    private String errMsg(boolean isWarning, String what, long value, long thresholdValue)
+    {
+        return messageProvider.createMessage(isWarning,
+                                             what,
+                                             value,
+                                             thresholdValue);
+    }
+
+    private long abortValue(Config config)
+    {
+        long abortValue = config.getAbortThreshold();
+        return abortValue < 0 ? Long.MAX_VALUE : abortValue;
+    }
+
+    private long warnValue(Config config)
+    {
+        long warnValue = config.getWarnThreshold();
+        return warnValue < 0 ? Long.MAX_VALUE : warnValue;
+    }
+
+    @Override
+    public boolean enabled(@Nullable ClientState state)
+    {
+        if (!super.enabled(state))
+            return false;
+
+        Config config = configProvider.apply(state);
+        return config.getAbortThreshold() >= 0 || config.getWarnThreshold() >= 0;
+    }
+
+    /**
+     * Checks whether the provided value would trigger a warning or abort the operation if passed to {@link #guard}.
+     *
+     * <p>This method is optional (does not have to be called) but can be used in the case where the "what"
+     * argument to {@link #guard} is expensive to build to save doing so in the common case (of the guardrail
+     * not being triggered).
+     *
+     * @param value The value to test.
+     * @param state The client state, used to skip the check if the query is internal or is done by a superuser.
+     *              A {@code null} value means that the check should be done regardless of the query.
+     * @return {@code true} if {@code value} is above the warning or abort thresholds of this guardrail,
+     * {@code false otherwise}.
+     */
+    public boolean triggersOn(long value, @Nullable ClientState state)
+    {
+        if (!enabled(state))
+            return false;
+
+        Config config = configProvider.apply(state);
+        return value > Math.min(abortValue(config), warnValue(config));
+    }

Review comment:
       deadcode; can we remove?

##########
File path: src/java/org/apache/cassandra/db/guardrails/GuardrailsConfigService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.db.guardrails;
+
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.GuardrailsOptions;
+import org.apache.cassandra.utils.MBeanWrapper;
+
+/**
+ * Implementation of {@link GuardrailsConfigServiceMBean}.
+ */
+public class GuardrailsConfigService implements GuardrailsConfigServiceMBean
+{
+    public static final String MBEAN_NAME = "org.apache.cassandra.db:type=Guardrails";
+
+    private final GuardrailsOptions config = DatabaseDescriptor.getGuardrailsConfig();

Review comment:
       this works for default but feels meaningless for the plugin case as `GuardrailsConfigProvider` isn't used.
   
   Now, we talked about the plugin case, so I am cool with this, but just calling out as I know that is a thing for your team

##########
File path: src/java/org/apache/cassandra/db/guardrails/Values.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.db.guardrails;
+
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.Sets;
+
+import org.apache.cassandra.service.ClientState;
+
+import static java.lang.String.format;
+
+/**
+ * A guardrail that warns about but ignores some specific values, and rejects the use of some other values.
+ *
+ * @param <T> The type of the values of which certain are disallowed.
+ */
+public class Values<T> extends Guardrail
+{
+    private final Function<ClientState, Config<T>> configProvider;
+    private final String what;
+
+    /**
+     * Creates a new values guardrail.
+     *
+     * @param configProvider a {@link ClientState}-based provider of {@link Config}s.
+     * @param what           The feature that is guarded by this guardrail (for reporting in error messages).
+     */
+    public Values(Function<ClientState, Config<T>> configProvider, String what)
+    {
+        this.configProvider = configProvider;
+        this.what = what;
+    }
+
+    /**
+     * Checks whether the provided value would trigger this guardrail.
+     *
+     * <p>This method is optional (does not have to be called) but can be used in the case some of the arguments
+     * to the actual guardrail method are expensive to build to save doing so in the common case of the guardrail not
+     * being triggered.
+     *
+     * @param value The value to test.
+     * @param state The client state, used to skip the check if the query is internal or is done by a superuser.
+     * @return {@code true} if {@code value} is not allowed by this guardrail,
+     * {@code false otherwise}.
+     */
+    public boolean triggersOn(T value, @Nullable ClientState state)
+    {
+        if (!enabled(state))
+            return false;
+
+        Config<T> config = configProvider.apply(state);
+        return config.getIgnored().contains(value) || config.getDisallowed().contains(value);
+    }
+
+    /**
+     * Triggers a warning if the provided value is disallowed by this guardrail and triggers an action to ignore it.
+     * If the value is disallowed it will abort the operation.
+     *
+     * @param value        The value to check.
+     * @param ignoreAction An action called on the subset of {@code values} that should be ignored. This action
+     *                     should do whatever is necessary to make sure the value is ignored.
+     * @param state        The client state, used to skip the check if the query is internal or is done by a superuser.
+     *                     A {@code null} value means that the check should be done regardless of the query.
+     */
+    public void guard(T value, Consumer<T> ignoreAction, @Nullable ClientState state)
+    {
+        if (!enabled(state))
+            return;
+
+        Config<T> config = configProvider.apply(state);
+
+        Set<T> ignored = config.getIgnored();
+        if (ignored.contains(value))
+        {
+            warn(format("Ignoring provided value %s as it is not supported for %s (ignored values are: %s)",
+                        value, what, ignored.toString()));
+            ignoreAction.accept(value);
+        }
+
+        Set<T> disallowed = config.getDisallowed();
+        if (disallowed.contains(value))
+            abort(format("Provided value %s is not allowed for %s (disallowed values are: %s)",
+                         value, what, disallowed.toString()));
+    }

Review comment:
       deadcode; can we remove?

##########
File path: src/java/org/apache/cassandra/db/guardrails/GuardrailsConfigService.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.db.guardrails;
+
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.utils.MBeanWrapper;
+
+/**
+ * JMX entrypoint for updating the default guardrails configuration parsed from {@code cassandra.yaml}.
+ *
+ * This is different to just exposing {@link GuardrailsConfig} in that the methods here should be JMX-friendly.
+ */
+public class GuardrailsConfigService implements GuardrailsConfigServiceMBean
+{
+    public static final String MBEAN_NAME = "org.apache.cassandra.db:type=Guardrails";
+
+    private final GuardrailsConfig config = DatabaseDescriptor.getGuardrailsConfig();
+
+    GuardrailsConfigService()
+    {
+        MBeanWrapper.instance.registerMBean(this, MBEAN_NAME);
+    }
+
+    @Override
+    public boolean getEnabled()
+    {
+        return config.getEnabled();
+    }
+
+    @Override
+    public void setEnabled(boolean enabled)
+    {
+        config.setEnabled(enabled);
+    }
+
+    @Override
+    public int getTablesWarnThreshold()
+    {
+        return config.getTablesWarnThreshold();
+    }
+
+    @Override
+    public void setTablesWarnThreshold(int threshold)
+    {
+        config.setTablesWarnThreshold(threshold);
+    }
+
+    @Override
+    public int getTablesFailThreshold()
+    {
+        return config.getTablesFailThreshold();
+    }
+
+    @Override
+    public void setTablesFailThreshold(int threshold)
+    {
+        config.setTablesFailThreshold(threshold);
+    }
+
+    @Override
+    public int getColumnsPerTableFailThreshold()
+    {
+        return config.getColumnsPerTableFailThreshold();
+    }
+
+    @Override
+    public void setColumnsPerTableFailThreshold(int threshold)
+    {
+        config.setColumnsPerTableFailThreshold(threshold);
+    }
+
+    @Override
+    public int getSecondaryIndexesPerTableFailThreshold()
+    {
+        return config.getSecondaryIndexesPerTableFailThreshold();
+    }
+
+    @Override
+    public void setSecondaryIndexesPerTableFailThreshold(int threshold)
+    {
+        config.setSecondaryIndexesPerTableFailThreshold(threshold);
+    }
+
+    @Override
+    public int getMaterializedViewsPerTableFailThreshold()
+    {
+        return config.getMaterializedViewsPerTableFailThreshold();
+    }
+
+    @Override
+    public void setMaterializedViewsPerTableFailThreshold(int threshold)
+    {
+        config.setMaterializedViewsPerTableFailThreshold(threshold);
+    }
+
+    @Override
+    public String getTablePropertiesDisallowed()
+    {
+        return toCSV(config.getTablePropertiesDisallowed());
+    }
+
+    @Override
+    public void setTablePropertiesDisallowed(String properties)
+    {
+        config.setTablePropertiesDisallowed(fromCSV(properties));
+    }
+
+    public void setTablePropertiesDisallowed(String... properties)
+    {
+        config.setTablePropertiesDisallowed(properties);
+    }
+
+    public void setTablePropertiesDisallowed(Set<String> properties)
+    {
+        config.setTablePropertiesDisallowed(properties);
+    }
+
+    @Override
+    public String getTablePropertiesIgnored()
+    {
+        return toCSV(config.getTablePropertiesIgnored());
+    }
+
+    @Override
+    public void setTablePropertiesIgnored(String properties)
+    {
+        config.setTablePropertiesIgnored(fromCSV(properties));
+    }
+
+    public void setTablePropertiesIgnored(String... properties)
+    {
+        config.setTablePropertiesIgnored(properties);
+    }
+
+    public void setTablePropertiesIgnored(Set<String> properties)
+    {
+        config.setTablePropertiesIgnored(properties);
+    }
+
+    @Override
+    public boolean getUserTimestampsEnabled()
+    {
+        return config.getUserTimestampsEnabled();
+    }
+
+    @Override
+    public void setUserTimestampsEnabled(boolean enabled)
+    {
+        config.setUserTimestampsEnabled(enabled);
+    }
+
+    private static String toCSV(Set<String> values)
+    {
+        return values == null ? "" : String.join(",", values);
+    }
+
+    private static Set<String> fromCSV(String csv)
+    {
+        return csv == null ? null : ImmutableSet.copyOf(csv.split(","));
+    }

Review comment:
       I am cool with string versions, just think you should also provide collection as well.

##########
File path: src/java/org/apache/cassandra/db/guardrails/Values.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.db.guardrails;
+
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.Sets;
+
+import org.apache.cassandra.service.ClientState;
+
+import static java.lang.String.format;
+
+/**
+ * A guardrail that warns about but ignores some specific values, and rejects the use of some other values.

Review comment:
       not sure how common this pattern is, kinda think we should merge into TableProperties and remove this class; will leave as your call though.




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