You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sm...@apache.org on 2022/12/02 11:31:20 UTC

[cassandra] branch trunk updated: Implement a guardrail for not having zero default_time_to_live on tables with TWCS

This is an automated email from the ASF dual-hosted git repository.

smiklosovic pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 797b96958a Implement a guardrail for not having zero default_time_to_live on tables with TWCS
797b96958a is described below

commit 797b96958af07945b0a1799024c1ef664dfee2e2
Author: Stefan Miklosovic <sm...@apache.org>
AuthorDate: Fri Nov 18 20:45:37 2022 +0100

    Implement a guardrail for not having zero default_time_to_live on tables with TWCS
    
    patch by Stefan Miklosovic; reviewed by Andrés de la Peña, Josh McKenzie, Brandon Williams and Brad Schoening for CASSANDRA-18042
    
    Co-authored-by: Andrés de la Peña <a....@gmail.com>
---
 CHANGES.txt                                        |   1 +
 conf/cassandra.yaml                                |  13 +++
 src/java/org/apache/cassandra/config/Config.java   |   2 +
 .../apache/cassandra/config/GuardrailsOptions.java |  30 +++++++
 .../statements/schema/AlterSchemaStatement.java    |  10 +++
 .../statements/schema/AlterTableStatement.java     |   2 +
 .../statements/schema/CreateTableStatement.java    |   2 +
 .../apache/cassandra/db/guardrails/EnableFlag.java |  36 +++++++-
 .../apache/cassandra/db/guardrails/Guardrails.java |  39 ++++++++
 .../cassandra/db/guardrails/GuardrailsConfig.java  |  31 +++++++
 .../cassandra/db/guardrails/GuardrailsMBean.java   |  31 +++++++
 .../GuardrailZeroDefaultTTLOnTWCSTest.java         | 100 +++++++++++++++++++++
 .../cassandra/db/guardrails/GuardrailsTest.java    |  18 ++++
 13 files changed, 314 insertions(+), 1 deletion(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 69c57eeb9a..34754e59d0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.2
+ * Implement a guardrail for not having zero default_time_to_live on tables with TWCS (CASSANDRA-18042)
  * Add CQL scalar functions for collection aggregation (CASSANDRA-18060)
  * Make cassandra.replayList property for CommitLogReplayer possible to react on keyspaces only (CASSANDRA-18044)
  * Add Mathematical functions (CASSANDRA-17221)
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index cdbbc252bc..530ae8cac6 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -1862,6 +1862,19 @@ drop_compact_storage_enabled: false
 # maximum_replication_factor_warn_threshold: -1
 # maximum_replication_factor_fail_threshold: -1
 
+# Guardrail to enable a CREATE or ALTER TABLE statement when default_time_to_live is set to 0
+# and the table is using TimeWindowCompactionStrategy compaction or a subclass of it.
+# It is suspicious to use default_time_to_live set to 0 with such compaction strategy.
+# Please keep in mind that data will not start to automatically expire after they are older than
+# a respective compaction window unit of a certain size. Please set TTL for your INSERT or UPDATE
+# statements if you expect data to be expired as table settings will not do it.
+# Defaults to true. If set to false, such statements fail and zero_ttl_on_twcs_warned flag is irrelevant.
+#zero_ttl_on_twcs_enabled: true
+# Guardrail to warn a user upon executing CREATE or ALTER TABLE statement when default_time_to_live is set to 0
+# and the table is using TimeWindowCompactionStrategy compaction or a subclass of it. Defaults to true.
+# if zero_ttl_on_twcs_enabled is set to false, this property is irrelevant as such statements will fail.
+#zero_ttl_on_twcs_warned: true
+
 # Startup Checks are executed as part of Cassandra startup process, not all of them
 # are configurable (so you can disable them) but these which are enumerated bellow.
 # Uncomment the startup checks and configure them appropriately to cover your needs.
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 79da6e5f1d..43780c2ffa 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -869,6 +869,8 @@ public class Config
     public volatile int minimum_replication_factor_fail_threshold = -1;
     public volatile int maximum_replication_factor_warn_threshold = -1;
     public volatile int maximum_replication_factor_fail_threshold = -1;
+    public volatile boolean zero_ttl_on_twcs_warned = true;
+    public volatile boolean zero_ttl_on_twcs_enabled = true;
 
     public volatile DurationSpec.LongNanosecondsBound streaming_state_expires = new DurationSpec.LongNanosecondsBound("3d");
     public volatile DataStorageSpec.LongBytesBound streaming_state_size = new DataStorageSpec.LongBytesBound("40MiB");
diff --git a/src/java/org/apache/cassandra/config/GuardrailsOptions.java b/src/java/org/apache/cassandra/config/GuardrailsOptions.java
index 9c7bd55cfd..434c4deb4c 100644
--- a/src/java/org/apache/cassandra/config/GuardrailsOptions.java
+++ b/src/java/org/apache/cassandra/config/GuardrailsOptions.java
@@ -730,6 +730,36 @@ public class GuardrailsOptions implements GuardrailsConfig
                                   x -> config.maximum_replication_factor_fail_threshold = x);
     }
 
+    @Override
+    public boolean getZeroTTLOnTWCSWarned()
+    {
+        return config.zero_ttl_on_twcs_warned;
+    }
+
+    @Override
+    public void setZeroTTLOnTWCSWarned(boolean value)
+    {
+        updatePropertyWithLogging("zero_ttl_on_twcs_warned",
+                                  value,
+                                  () -> config.zero_ttl_on_twcs_warned,
+                                  x -> config.zero_ttl_on_twcs_warned = x);
+    }
+
+    @Override
+    public boolean getZeroTTLOnTWCSEnabled()
+    {
+        return config.zero_ttl_on_twcs_enabled;
+    }
+
+    @Override
+    public void setZeroTTLOnTWCSEnabled(boolean value)
+    {
+        updatePropertyWithLogging("zero_ttl_on_twcs_enabled",
+                                  value,
+                                  () -> config.zero_ttl_on_twcs_enabled,
+                                  x -> config.zero_ttl_on_twcs_enabled = x);
+    }
+
     private static <T> void updatePropertyWithLogging(String propertyName, T newValue, Supplier<T> getter, Consumer<T> setter)
     {
         T oldValue = getter.get();
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java
index fdc49216c8..a539ea74e8 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java
@@ -26,6 +26,8 @@ import org.apache.cassandra.auth.IResource;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.db.compaction.TimeWindowCompactionStrategy;
+import org.apache.cassandra.db.guardrails.Guardrails;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
@@ -140,6 +142,14 @@ abstract public class AlterSchemaStatement implements CQLStatement.SingleKeyspac
         }
     }
 
+    protected void validateDefaultTimeToLive(TableParams params)
+    {
+        if (params.defaultTimeToLive == 0
+            && !SchemaConstants.isSystemKeyspace(keyspaceName)
+            && TimeWindowCompactionStrategy.class.isAssignableFrom(params.compaction.klass()))
+            Guardrails.zeroTTLOnTWCSEnabled.ensureEnabled(state);
+    }
+
     private void grantPermissionsOnResource(IResource resource, AuthenticatedUser user)
     {
         try
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
index ab6a631b99..ad0358c2cb 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
@@ -443,6 +443,8 @@ public abstract class AlterTableStatement extends AlterSchemaStatement
             super.validate(state);
 
             Guardrails.tableProperties.guard(attrs.updatedProperties(), attrs::removeProperty, state);
+
+            validateDefaultTimeToLive(attrs.asNewTableParams());
         }
 
         public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java
index 1a625380cb..3b1ea36b4e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java
@@ -149,6 +149,8 @@ public final class CreateTableStatement extends AlterSchemaStatement
         // Guardrail to check whether creation of new COMPACT STORAGE tables is allowed
         if (useCompactStorage)
             Guardrails.compactTablesEnabled.ensureEnabled(state);
+
+        validateDefaultTimeToLive(attrs.asNewTableParams());
     }
 
     SchemaChange schemaChangeEvent(KeyspacesDiff diff)
diff --git a/src/java/org/apache/cassandra/db/guardrails/EnableFlag.java b/src/java/org/apache/cassandra/db/guardrails/EnableFlag.java
index 7b5a63eb99..dde09be8ff 100644
--- a/src/java/org/apache/cassandra/db/guardrails/EnableFlag.java
+++ b/src/java/org/apache/cassandra/db/guardrails/EnableFlag.java
@@ -32,6 +32,7 @@ import org.apache.cassandra.service.ClientState;
  */
 public class EnableFlag extends Guardrail
 {
+    private final Predicate<ClientState> warned;
     private final Predicate<ClientState> enabled;
     private final String featureName;
 
@@ -46,8 +47,32 @@ public class EnableFlag extends Guardrail
      *                    EnableFlag#ensureEnabled(String, ClientState)} can specify a different {@code featureName}.
      */
     public EnableFlag(String name, @Nullable String reason, Predicate<ClientState> enabled, String featureName)
+    {
+        this(name, reason, (state) -> false, enabled, featureName);
+    }
+
+    /**
+     * Creates a new {@link EnableFlag} guardrail.
+     *
+     * @param name        the identifying name of the guardrail
+     * @param reason      the optional description of the reason for guarding the operation
+     * @param warned      a {@link ClientState}-based supplier of boolean indicating whether warning should be
+     *                    emitted even guardrail as such has passed. If guardrail fails, the warning will not be
+     *                    emitted. This might be used for cases when we want to warn a user regardless of successful
+     *                    guardrail execution.
+     * @param enabled     a {@link ClientState}-based supplier of boolean indicating whether the feature guarded by this
+     *                    guardrail is enabled.
+     * @param featureName The feature that is guarded by this guardrail (for reporting in error messages), {@link
+     *                    EnableFlag#ensureEnabled(String, ClientState)} can specify a different {@code featureName}.
+     */
+    public EnableFlag(String name,
+                      @Nullable String reason,
+                      Predicate<ClientState> warned,
+                      Predicate<ClientState> enabled,
+                      String featureName)
     {
         super(name, reason);
+        this.warned = warned;
         this.enabled = enabled;
         this.featureName = featureName;
     }
@@ -93,7 +118,16 @@ public class EnableFlag extends Guardrail
      */
     public void ensureEnabled(String featureName, @Nullable ClientState state)
     {
-        if (!isEnabled(state))
+        if (!enabled(state))
+            return;
+
+        if (!enabled.test(state))
+        {
             fail(featureName + " is not allowed", state);
+            return;
+        }
+
+        if (warned.test(state))
+            warn(featureName + " is not recommended");
     }
 }
diff --git a/src/java/org/apache/cassandra/db/guardrails/Guardrails.java b/src/java/org/apache/cassandra/db/guardrails/Guardrails.java
index 4ea6312b4f..a4e0cd9d83 100644
--- a/src/java/org/apache/cassandra/db/guardrails/Guardrails.java
+++ b/src/java/org/apache/cassandra/db/guardrails/Guardrails.java
@@ -33,6 +33,7 @@ import org.apache.cassandra.config.DataStorageSpec;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.GuardrailsOptions;
 import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.compaction.TimeWindowCompactionStrategy;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.service.disk.usage.DiskUsageBroadcaster;
 import org.apache.cassandra.utils.MBeanWrapper;
@@ -199,6 +200,20 @@ public final class Guardrails implements GuardrailsMBean
                    state -> CONFIG_PROVIDER.getOrCreate(state).getCompactTablesEnabled(),
                    "Creation of new COMPACT STORAGE tables");
 
+    /**
+     * Guardrail to warn or fail a CREATE or ALTER TABLE statement when default_time_to_live is set to 0 and
+     * the table is using TimeWindowCompactionStrategy compaction or a subclass of it.
+     */
+    public static final EnableFlag zeroTTLOnTWCSEnabled =
+    new EnableFlag("zero_ttl_on_twcs",
+                   "It is suspicious to use default_time_to_live set to 0 with such compaction strategy. " +
+                   "Please keep in mind that data will not start to automatically expire after they are older " +
+                   "than a respective compaction window unit of a certain size. Please set TTL for your INSERT or UPDATE " +
+                   "statements if you expect data to be expired as table settings will not do it. ",
+                   state -> CONFIG_PROVIDER.getOrCreate(state).getZeroTTLOnTWCSWarned(),
+                   state -> CONFIG_PROVIDER.getOrCreate(state).getZeroTTLOnTWCSEnabled(),
+                   "0 default_time_to_live on a table with " + TimeWindowCompactionStrategy.class.getSimpleName() + " compaction strategy");
+
     /**
      * Guardrail on the number of elements returned within page.
      */
@@ -1013,6 +1028,30 @@ public final class Guardrails implements GuardrailsMBean
         DEFAULT_CONFIG.setMinimumReplicationFactorThreshold(warn, fail);
     }
 
+    @Override
+    public boolean getZeroTTLOnTWCSEnabled()
+    {
+        return DEFAULT_CONFIG.getZeroTTLOnTWCSEnabled();
+    }
+
+    @Override
+    public void setZeroTTLOnTWCSEnabled(boolean value)
+    {
+        DEFAULT_CONFIG.setZeroTTLOnTWCSEnabled(value);
+    }
+
+    @Override
+    public boolean getZeroTTLOnTWCSWarned()
+    {
+        return DEFAULT_CONFIG.getZeroTTLOnTWCSWarned();
+    }
+
+    @Override
+    public void setZeroTTLOnTWCSWarned(boolean value)
+    {
+        DEFAULT_CONFIG.setZeroTTLOnTWCSWarned(value);
+    }
+
     private static String toCSV(Set<String> values)
     {
         return values == null || values.isEmpty() ? "" : String.join(",", values);
diff --git a/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java b/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java
index 9cf481e27b..9d990f20be 100644
--- a/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java
+++ b/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java
@@ -319,4 +319,35 @@ public interface GuardrailsConfig
      * @return The threshold to fail when replication factor is greater than threshold.
      */
     int getMaximumReplicationFactorFailThreshold();
+
+    /**
+     * Returns whether warnings will be emitted when usage of 0 default TTL on a
+     * table with TimeWindowCompactionStrategy is detected.
+     *
+     * @return {@code true} if warnings will be emitted, {@code false} otherwise.
+     */
+    boolean getZeroTTLOnTWCSWarned();
+
+    /**
+     * Sets whether warnings will be emitted when usage of 0 default TTL on a
+     * table with TimeWindowCompactionStrategy is detected.
+     *
+     * @param value {@code true} if warning will be emitted, {@code false} otherwise.
+     */
+    void setZeroTTLOnTWCSWarned(boolean value);
+
+    /**
+     * Returns whether it is allowed to create or alter table to use 0 default TTL with TimeWindowCompactionStrategy.
+     * If it is not, such query will fail.
+     *
+     * @return {@code true} if 0 default TTL is allowed on TWCS table, {@code false} otherwise.
+     */
+    boolean getZeroTTLOnTWCSEnabled();
+
+    /**
+     * Sets whether users can use 0 default TTL on a table with TimeWindowCompactionStrategy.
+     *
+     * @param value {@code true} if 0 default TTL on TWCS tables is allowed, {@code false} otherwise.
+     */
+    void setZeroTTLOnTWCSEnabled(boolean value);
 }
diff --git a/src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java b/src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java
index 48a01dad6f..d738aa3dbf 100644
--- a/src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java
+++ b/src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java
@@ -620,4 +620,35 @@ public interface GuardrailsMBean
      *             -1 means disabled.
      */
     void setMaximumReplicationFactorThreshold (int warn, int fail);
+
+    /**
+     * Returns whether warnings will be emitted when usage of 0 default TTL on a
+     * table with TimeWindowCompactionStrategy is detected.
+     *
+     * @return {@code true} if warnings will be emitted, {@code false} otherwise.
+     */
+    boolean getZeroTTLOnTWCSWarned();
+
+    /**
+     * Sets whether warnings will be emitted when usage of 0 default TTL on a
+     * table with TimeWindowCompactionStrategy is detected.
+     *
+     * @param value {@code true} if warning will be emitted, {@code false} otherwise.
+     */
+    void setZeroTTLOnTWCSWarned(boolean value);
+
+    /**
+     * Returns whether it is allowed to create or alter table to use 0 default TTL with TimeWindowCompactionStrategy.
+     * If it is not, such query will fail.
+     *
+     * @return {@code true} if 0 default TTL is allowed on TWCS table, {@code false} otherwise.
+     */
+    boolean getZeroTTLOnTWCSEnabled();
+
+    /**
+     * Sets whether users can use 0 default TTL on a table with TimeWindowCompactionStrategy.
+     *
+     * @param value {@code true} if 0 default TTL on TWCS tables is allowed, {@code false} otherwise.
+     */
+    void setZeroTTLOnTWCSEnabled(boolean value);
 }
diff --git a/test/unit/org/apache/cassandra/db/guardrails/GuardrailZeroDefaultTTLOnTWCSTest.java b/test/unit/org/apache/cassandra/db/guardrails/GuardrailZeroDefaultTTLOnTWCSTest.java
new file mode 100644
index 0000000000..afd00e6ee0
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/guardrails/GuardrailZeroDefaultTTLOnTWCSTest.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 org.junit.Test;
+
+import org.apache.cassandra.db.compaction.TimeWindowCompactionStrategy;
+
+public class GuardrailZeroDefaultTTLOnTWCSTest extends GuardrailTester
+{
+    private static final String QUERY = "CREATE TABLE IF NOT EXISTS tb1 (k int PRIMARY KEY, a int, b int) " +
+                                        "WITH default_time_to_live = 0 " +
+                                        "AND compaction = {'class': 'TimeWindowCompactionStrategy', 'enabled': true }";
+
+    private static final String VALID_QUERY_1 = "CREATE TABLE IF NOT EXISTS tb2 (k int PRIMARY KEY, a int, b int) " +
+                                                "WITH default_time_to_live = 1 " +
+                                                "AND compaction = {'class': 'TimeWindowCompactionStrategy', 'enabled': true }";
+
+    private static final String VALID_QUERY_2 = "CREATE TABLE IF NOT EXISTS tb3 (k int PRIMARY KEY, a int, b int) " +
+                                                "WITH default_time_to_live = 0";
+
+    public GuardrailZeroDefaultTTLOnTWCSTest()
+    {
+        super(Guardrails.zeroTTLOnTWCSEnabled);
+    }
+
+    @Test
+    public void testGuardrailDisabled() throws Throwable
+    {
+        prepareTest(false, true);
+        assertFails(QUERY, "0 default_time_to_live on a table with " +
+                           TimeWindowCompactionStrategy.class.getSimpleName() +
+                           " compaction strategy is not allowed");
+    }
+
+    @Test
+    public void testGuardrailEnabledWarnEnabled() throws Throwable
+    {
+        prepareTest(true, true);
+        assertWarns(QUERY, "0 default_time_to_live on a table with " +
+                           TimeWindowCompactionStrategy.class.getSimpleName() +
+                           " compaction strategy is not recommended");
+    }
+
+    @Test
+    public void testGuardrailEnabledWarnDisabled() throws Throwable
+    {
+        prepareTest(true, false);
+        assertValid(QUERY);
+    }
+
+    @Test
+    public void testGuardrailNotTriggered() throws Throwable
+    {
+        prepareTest(true, true);
+        assertValid(VALID_QUERY_1);
+        assertValid(VALID_QUERY_2);
+
+        prepareTest(false, true);
+        assertValid(VALID_QUERY_1);
+        assertValid(VALID_QUERY_2);
+    }
+
+    @Test
+    public void testExcludedUsers() throws Throwable
+    {
+        for (boolean enabled : new boolean[] { false, true })
+        {
+            for (boolean warned : new boolean[]{ false, true })
+            {
+                prepareTest(enabled, warned);
+                testExcludedUsers(() -> QUERY,
+                                  () -> VALID_QUERY_1,
+                                  () -> VALID_QUERY_2);
+            }
+        }
+    }
+
+    private void prepareTest(boolean enabled, boolean warned)
+    {
+        guardrails().setZeroTTLOnTWCSEnabled(enabled);
+        guardrails().setZeroTTLOnTWCSWarned(warned);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/guardrails/GuardrailsTest.java b/test/unit/org/apache/cassandra/db/guardrails/GuardrailsTest.java
index 9cea89b4b5..f143120d56 100644
--- a/test/unit/org/apache/cassandra/db/guardrails/GuardrailsTest.java
+++ b/test/unit/org/apache/cassandra/db/guardrails/GuardrailsTest.java
@@ -39,6 +39,7 @@ public class GuardrailsTest extends GuardrailTester
 {
     public static final int DISABLED = -1;
     public static final String REASON = "Testing";
+    public static final String FEATURE = "Feature name";
 
     private void testDisabledThreshold(Threshold guard) throws Throwable
     {
@@ -283,6 +284,23 @@ public class GuardrailsTest extends GuardrailTester
         assertValid(() -> disabled.ensureEnabled(superClientState));
     }
 
+    @Test
+    public void testEnableFlagWarn() throws Throwable
+    {
+        EnableFlag disabledGuard = new EnableFlag("x", null, state -> true, state -> false, FEATURE);
+
+        assertFails(() -> disabledGuard.ensureEnabled(null), false, FEATURE + " is not allowed");
+        assertFails(() -> disabledGuard.ensureEnabled(userClientState), FEATURE + " is not allowed");
+        assertValid(() -> disabledGuard.ensureEnabled(systemClientState));
+        assertValid(() -> disabledGuard.ensureEnabled(superClientState));
+
+        EnableFlag enabledGuard = new EnableFlag("x", null, state -> true, state -> true, FEATURE);
+        assertWarns(() -> enabledGuard.ensureEnabled(null), FEATURE + " is not recommended");
+        assertWarns(() -> enabledGuard.ensureEnabled(userClientState), FEATURE + " is not recommended");
+        assertValid(() -> enabledGuard.ensureEnabled(systemClientState));
+        assertValid(() -> enabledGuard.ensureEnabled(superClientState));
+    }
+
     @Test
     public void testValuesWarned() throws Throwable
     {


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