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 2022/04/26 18:21:07 UTC

[GitHub] [cassandra] dcapwell commented on a diff in pull request #1583: CASSANDRA-17212 : Migrate threshold for minimum keyspace replication factor to guardrails

dcapwell commented on code in PR #1583:
URL: https://github.com/apache/cassandra/pull/1583#discussion_r859009806


##########
src/java/org/apache/cassandra/cql3/statements/schema/AlterKeyspaceStatement.java:
##########
@@ -99,6 +102,15 @@ public void authorize(ClientState client)
         client.ensureKeyspacePermission(keyspaceName, Permission.ALTER);
     }
 
+    @Override
+    public void validate(ClientState state)
+    {
+        super.validate(state);

Review Comment:
   if you rebase I believe this was done by Josh, so you don't need to store the state



##########
src/java/org/apache/cassandra/db/guardrails/Guardrails.java:
##########
@@ -51,11 +51,11 @@ public final class Guardrails implements GuardrailsMBean
     /**
      * Guardrail on the total number of user keyspaces.
      */
-    public static final Threshold keyspaces =
-    new Threshold("keyspaces",
+    public static final MaxThreshold keyspaces =

Review Comment:
   I like types as its clear at usage, but agree that 90% of the code should be in a shared base type of Threshold



##########
test/unit/org/apache/cassandra/tools/nodetool/GetMinimumKeyspaceRFTest.java:
##########
@@ -1,87 +0,0 @@
-/*
- * 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.tools.nodetool;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.CQLTester;
-import org.apache.cassandra.tools.ToolRunner;
-import org.assertj.core.api.Assertions;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-public class GetMinimumKeyspaceRFTest extends CQLTester

Review Comment:
   @adelapena I don't know about getting rid of them, would need to see original JIRA more closely to see the full intent.  I would be concerned getting rid of a new feature when we can port it to the new configs without much prob



##########
src/java/org/apache/cassandra/db/guardrails/MinThreshold.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.ToLongFunction;
+import javax.annotation.Nullable;
+
+import org.apache.cassandra.service.ClientState;
+
+/**
+ * A guardrail based on numeric threshold(s).
+ *
+ * <p>A {@link MinThreshold} guardrail defines (up to) 2 thresholds, one at which a warning is issued, and a lower one
+ * at which the operation is aborted with an exception. Only one of those thresholds can be activated if desired.
+ * A {@Link MinThreshold} is defined for minimum guardrails, the value is checked to see if
+ * it is less than the warn and fail thresholds.
+ *
+ * <p>This guardrail only handles guarding positive values.
+ */
+public class MinThreshold extends Threshold
+{
+    private final ToLongFunction<ClientState> warnThreshold;
+    private final ToLongFunction<ClientState> failThreshold;
+
+    /**
+     * Creates a new minimum threshold guardrail.
+     *
+     * @param name            the identifying name of the guardrail
+     * @param warnThreshold   a {@link ClientState}-based provider of the value above which a warning should be triggered.
+     * @param failThreshold   a {@link ClientState}-based provider of the value above which the operation should be aborted.
+     * @param messageProvider a function to generate the warning or error message if the guardrail is triggered
+     */
+    public MinThreshold(String name,
+                        ToLongFunction<ClientState> warnThreshold,
+                        ToLongFunction<ClientState> failThreshold,
+                        Threshold.ErrorMessageProvider messageProvider)
+    {
+        super(name, warnThreshold, failThreshold, messageProvider);
+        this.warnThreshold = warnThreshold;
+        this.failThreshold = failThreshold;
+    }
+
+    private long failValue(ClientState state)
+    {
+        long failValue = failThreshold.applyAsLong(state);
+        return failValue <= 0 ? Long.MIN_VALUE : failValue;
+    }
+
+    private long warnValue(ClientState state)
+    {
+        long warnValue = warnThreshold.applyAsLong(state);
+        return warnValue <= 0 ? Long.MIN_VALUE : warnValue;
+    }
+
+    /**
+     * Checks whether the provided value would trigger a warning or failure 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 failure thresholds of this guardrail,
+     * {@code false otherwise}.
+     */
+    public boolean triggersOn(long value, @Nullable ClientState state)
+    {
+        return enabled(state) && (value < Math.max(failValue(state), warnValue(state)));
+    }
+
+    /**
+     * Apply the guardrail to the provided value, warning or failing 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.
+     * @param containsUserData whether the {@code what} contains user data that should be redacted on external systems.
+     * @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, boolean containsUserData, @Nullable ClientState state)
+    {
+        if (!enabled(state))
+            return;
+
+        long failValue = failValue(state);
+        if (value < failValue)
+        {
+            triggerFail(value, failValue, what, containsUserData, state);
+            return;
+        }
+
+        long warnValue = warnValue(state);
+        if (value < warnValue)
+            triggerWarn(value, warnValue, what, containsUserData);
+    }

Review Comment:
   This is all copy/paste, can we refactor so the operator is pluggable?  Either have an abstract compare method
   
   ```
   protected boolean compare(long a, long b)
   ```
   
   or make the comparison an enum like
   
   ```
   enum Operator {
     GREATER_THAN: { public boolean compare(long a, long b) { return a > b; }}
   ...
   ```



##########
src/java/org/apache/cassandra/tools/nodetool/GetMinimumKeyspaceRF.java:
##########
@@ -26,8 +26,11 @@
 @Command(name = "getminimumrf", description = "Gets minimum keyspace replication factor.")
 public class GetMinimumKeyspaceRF extends NodeTool.NodeToolCmd
 {
+    @Arguments(title = "<getWarn>", usage = "<getWarn>", description = "warn flag", required = true)
+    private Boolean getWarn = false;

Review Comment:
   use `boolean`



##########
src/java/org/apache/cassandra/locator/LocalStrategy.java:
##########
@@ -70,7 +71,7 @@ public void validateOptions() throws ConfigurationException
     }
 
     @Override
-    public void maybeWarnOnOptions()
+    public void maybeWarnOnOptions(ClientState state)

Review Comment:
   please revert, you do not have a body so you should take in the default implementations



##########
src/java/org/apache/cassandra/tools/nodetool/GetMinimumKeyspaceRF.java:
##########
@@ -26,8 +26,11 @@
 @Command(name = "getminimumrf", description = "Gets minimum keyspace replication factor.")
 public class GetMinimumKeyspaceRF extends NodeTool.NodeToolCmd
 {
+    @Arguments(title = "<getWarn>", usage = "<getWarn>", description = "warn flag", required = true)

Review Comment:
   why is it required when this is optional?  if you added tests you should see this didn't work.
   
   Also, `<getWarn>` doesn't make sense, if you added a test you should see how this impacts the CLI



##########
src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java:
##########
@@ -422,10 +434,7 @@ protected void validateReplicationFactor(String s) throws ConfigurationException
         {
             ReplicationFactor rf = ReplicationFactor.fromString(s);
 
-            if (rf.fullReplicas < DatabaseDescriptor.getMinimumKeyspaceRF())
-            {
-                throw new ConfigurationException(String.format("Replication factor cannot be less than minimum_keyspace_rf (%d), found %d", DatabaseDescriptor.getMinimumKeyspaceRF(), rf.fullReplicas));
-            }
+            // mininum replication factor checked in #maybeWarnOnOptions

Review Comment:
   please remove



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