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/05/04 15:34:26 UTC

[GitHub] [cassandra] adelapena commented on a diff in pull request #1582: CASSANDRA-17500 Create Maximum Keyspace Replication Factor Guardrail

adelapena commented on code in PR #1582:
URL: https://github.com/apache/cassandra/pull/1582#discussion_r864978813


##########
src/java/org/apache/cassandra/db/guardrails/Guardrails.java:
##########
@@ -333,6 +333,19 @@ public final class Guardrails implements GuardrailsMBean
                                : format("The keyspace %s has a replication factor of %s, below the failure threshold of %s.",
                                         what, value, threshold));
 
+    /**
+     * Guardrail on the maximum replication factor.
+     */
+    public static final MaxThreshold maximumReplicationFactor =
+    new MaxThreshold("maximum_replication_factor",
+                  state -> CONFIG_PROVIDER.getOrCreate(state).getMaximumReplicationFactorWarnThreshold(),
+                  state -> CONFIG_PROVIDER.getOrCreate(state).getMaximumReplicationFactorFailThreshold(),
+                  (isWarning, what, value, threshold) ->
+                  isWarning ? format("The keyspace %s has a replication factor of %s, above the warning threshold of %s.",
+                                     what, value, threshold)
+                            :format("The keyspace %s has a replication factor of %s, above the failure threshold of %s.",
+                                    what, value, threshold));

Review Comment:
   Nit: misaligned
   ```suggestion
                               : format("The keyspace %s has a replication factor of %s, above the failure threshold of %s.",
                                        what, value, threshold));
   ```



##########
test/unit/org/apache/cassandra/db/guardrails/GuardrailMaximumReplicationFactorTest.java:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import org.junit.After;
+import org.junit.Test;
+
+import org.apache.cassandra.ServerTestUtils;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.StorageService;
+
+import static java.lang.String.format;
+
+public class GuardrailMaximumReplicationFactorTest extends ThresholdTester
+{
+    private static final int MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD = 2;
+    private static final int MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD = 4;
+    private static final int DISABLED_GUARDRAIL = -1;
+    private static final String WHAT = "maximum_replication_factor";
+    private static final String KS = "ks";
+
+    public GuardrailMaximumReplicationFactorTest()
+    {
+        super(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD,
+              MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD,
+              Guardrails.maximumReplicationFactor,
+              Guardrails::setMaximumReplicationFactorThreshold,
+              Guardrails::getMaximumReplicationFactorWarnThreshold,
+              Guardrails::getMaximumReplicationFactorFailThreshold);
+    }
+
+    @After
+    public void cleanupTest() throws Throwable
+    {
+        execute("DROP KEYSPACE IF EXISTS ks");
+    }
+
+    @Override
+    protected long currentValue()
+    {
+        return Long.parseLong((Keyspace.open("ks").getReplicationStrategy()).configOptions.get("datacenter1"));
+    }
+
+    @Override
+    protected List<String> getWarnings()
+    {
+        List<String> warnings = ClientWarn.instance.getWarnings();
+
+        return warnings == null
+               ? Collections.emptyList()
+               : warnings.stream()
+                         .filter(w -> !w.contains("keyspace ks is higher than the number of nodes 1 for datacenter1") &&
+                                      !w.contains("When increasing replication factor you need to run a full (-full) repair to distribute the data") &&
+                                      !w.contains("keyspace ks is higher than the number of nodes") &&
+                                      !w.contains("Your replication factor 3 for keyspace ks is higher than the number of nodes 2 for datacenter datacenter2"))
+                         .collect(Collectors.toList());
+    }
+
+    @Test
+    public void testMaxKeyspaceRFDisabled() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(DISABLED_GUARDRAIL, DISABLED_GUARDRAIL);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 6}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 10}");
+    }
+
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertWarns("CREATE KEYSPACE ks WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': 3}",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.",
+                           KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+        assertFails("ALTER KEYSPACE ks WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': 5}",
+                    format("The keyspace %s has a replication factor of 5, above the failure threshold of %s.",
+                           KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+    }
+
+    @Test
+    public void testMultipleDatacenter() throws Throwable
+    {
+        IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
+        DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
+        {
+            public static final String RACK1 = ServerTestUtils.RACK1;
+
+            @Override
+            public String getRack(InetAddressAndPort endpoint) { return RACK1; }
+
+            @Override
+            public String getDatacenter(InetAddressAndPort endpoint) { return "datacenter2"; }
+
+            @Override
+            public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) { return 0; }
+        });
+
+        List<String> twoWarnings = Arrays.asList(format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD),
+                                                 format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+
+        StorageService.instance.getTokenMetadata().updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.0.0.255"));
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertValid("CREATE KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 2 };");
+
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 3 };",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 3, 'datacenter2' : 3 };", twoWarnings);
+
+        assertFails("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 5 };",
+                    format("The keyspace %s has a replication factor of 5, above the failure threshold of %s", KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+
+        assertFails("CREATE KEYSPACE ks1 WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 5, 'datacenter2' : 5 };",
+                    format("The keyspace ks1 has a replication factor of 5, above the failure threshold of %s", MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+
+        DatabaseDescriptor.setEndpointSnitch(snitch);
+        execute("DROP KEYSPACE IF EXISTS ks1");
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyWarnBelow() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, DISABLED_GUARDRAIL);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 2}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 1}");
+
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyWarnAbove() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, DISABLED_GUARDRAIL);
+        assertWarns("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 3}",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 4}",
+                    format("The keyspace %s has a replication factor of 4, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyFailBelow() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(DISABLED_GUARDRAIL, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 2}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 3}");
+

Review Comment:
   Nit: unneeded blank line.



##########
test/unit/org/apache/cassandra/db/guardrails/GuardrailMaximumReplicationFactorTest.java:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import org.junit.After;
+import org.junit.Test;
+
+import org.apache.cassandra.ServerTestUtils;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.StorageService;
+
+import static java.lang.String.format;
+
+public class GuardrailMaximumReplicationFactorTest extends ThresholdTester
+{
+    private static final int MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD = 2;
+    private static final int MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD = 4;
+    private static final int DISABLED_GUARDRAIL = -1;
+    private static final String WHAT = "maximum_replication_factor";
+    private static final String KS = "ks";
+
+    public GuardrailMaximumReplicationFactorTest()
+    {
+        super(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD,
+              MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD,
+              Guardrails.maximumReplicationFactor,
+              Guardrails::setMaximumReplicationFactorThreshold,
+              Guardrails::getMaximumReplicationFactorWarnThreshold,
+              Guardrails::getMaximumReplicationFactorFailThreshold);
+    }
+
+    @After
+    public void cleanupTest() throws Throwable
+    {
+        execute("DROP KEYSPACE IF EXISTS ks");
+    }
+
+    @Override
+    protected long currentValue()
+    {
+        return Long.parseLong((Keyspace.open("ks").getReplicationStrategy()).configOptions.get("datacenter1"));
+    }
+
+    @Override
+    protected List<String> getWarnings()
+    {
+        List<String> warnings = ClientWarn.instance.getWarnings();
+
+        return warnings == null
+               ? Collections.emptyList()
+               : warnings.stream()
+                         .filter(w -> !w.contains("keyspace ks is higher than the number of nodes 1 for datacenter1") &&
+                                      !w.contains("When increasing replication factor you need to run a full (-full) repair to distribute the data") &&
+                                      !w.contains("keyspace ks is higher than the number of nodes") &&
+                                      !w.contains("Your replication factor 3 for keyspace ks is higher than the number of nodes 2 for datacenter datacenter2"))
+                         .collect(Collectors.toList());
+    }
+
+    @Test
+    public void testMaxKeyspaceRFDisabled() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(DISABLED_GUARDRAIL, DISABLED_GUARDRAIL);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 6}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 10}");
+    }
+
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertWarns("CREATE KEYSPACE ks WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': 3}",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.",
+                           KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+        assertFails("ALTER KEYSPACE ks WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': 5}",
+                    format("The keyspace %s has a replication factor of 5, above the failure threshold of %s.",
+                           KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+    }
+
+    @Test
+    public void testMultipleDatacenter() throws Throwable
+    {
+        IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
+        DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
+        {
+            public static final String RACK1 = ServerTestUtils.RACK1;
+
+            @Override
+            public String getRack(InetAddressAndPort endpoint) { return RACK1; }
+
+            @Override
+            public String getDatacenter(InetAddressAndPort endpoint) { return "datacenter2"; }
+
+            @Override
+            public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) { return 0; }
+        });
+
+        List<String> twoWarnings = Arrays.asList(format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD),
+                                                 format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+
+        StorageService.instance.getTokenMetadata().updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.0.0.255"));
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertValid("CREATE KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 2 };");
+
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 3 };",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 3, 'datacenter2' : 3 };", twoWarnings);
+
+        assertFails("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 5 };",
+                    format("The keyspace %s has a replication factor of 5, above the failure threshold of %s", KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+
+        assertFails("CREATE KEYSPACE ks1 WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 5, 'datacenter2' : 5 };",
+                    format("The keyspace ks1 has a replication factor of 5, above the failure threshold of %s", MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+
+        DatabaseDescriptor.setEndpointSnitch(snitch);
+        execute("DROP KEYSPACE IF EXISTS ks1");
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyWarnBelow() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, DISABLED_GUARDRAIL);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 2}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 1}");
+
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyWarnAbove() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, DISABLED_GUARDRAIL);
+        assertWarns("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 3}",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 4}",
+                    format("The keyspace %s has a replication factor of 4, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+

Review Comment:
   Nit: unneeded blank line.



##########
test/unit/org/apache/cassandra/db/guardrails/GuardrailMaximumReplicationFactorTest.java:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import org.junit.After;
+import org.junit.Test;
+
+import org.apache.cassandra.ServerTestUtils;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.StorageService;
+
+import static java.lang.String.format;
+
+public class GuardrailMaximumReplicationFactorTest extends ThresholdTester
+{
+    private static final int MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD = 2;
+    private static final int MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD = 4;
+    private static final int DISABLED_GUARDRAIL = -1;
+    private static final String WHAT = "maximum_replication_factor";

Review Comment:
   Unused, probably because, differently from `GuardrailMinimumReplicationFactorTest`, there isn't a `testConfigValidation` test method.



##########
test/unit/org/apache/cassandra/db/guardrails/GuardrailMaximumReplicationFactorTest.java:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import org.junit.After;
+import org.junit.Test;
+
+import org.apache.cassandra.ServerTestUtils;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.StorageService;
+
+import static java.lang.String.format;
+
+public class GuardrailMaximumReplicationFactorTest extends ThresholdTester
+{
+    private static final int MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD = 2;
+    private static final int MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD = 4;
+    private static final int DISABLED_GUARDRAIL = -1;
+    private static final String WHAT = "maximum_replication_factor";
+    private static final String KS = "ks";
+
+    public GuardrailMaximumReplicationFactorTest()
+    {
+        super(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD,
+              MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD,
+              Guardrails.maximumReplicationFactor,
+              Guardrails::setMaximumReplicationFactorThreshold,
+              Guardrails::getMaximumReplicationFactorWarnThreshold,
+              Guardrails::getMaximumReplicationFactorFailThreshold);
+    }
+
+    @After
+    public void cleanupTest() throws Throwable
+    {
+        execute("DROP KEYSPACE IF EXISTS ks");
+    }
+
+    @Override
+    protected long currentValue()
+    {
+        return Long.parseLong((Keyspace.open("ks").getReplicationStrategy()).configOptions.get("datacenter1"));
+    }
+
+    @Override
+    protected List<String> getWarnings()
+    {
+        List<String> warnings = ClientWarn.instance.getWarnings();
+
+        return warnings == null
+               ? Collections.emptyList()
+               : warnings.stream()
+                         .filter(w -> !w.contains("keyspace ks is higher than the number of nodes 1 for datacenter1") &&
+                                      !w.contains("When increasing replication factor you need to run a full (-full) repair to distribute the data") &&
+                                      !w.contains("keyspace ks is higher than the number of nodes") &&
+                                      !w.contains("Your replication factor 3 for keyspace ks is higher than the number of nodes 2 for datacenter datacenter2"))
+                         .collect(Collectors.toList());
+    }
+
+    @Test
+    public void testMaxKeyspaceRFDisabled() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(DISABLED_GUARDRAIL, DISABLED_GUARDRAIL);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 6}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 10}");
+    }
+
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertWarns("CREATE KEYSPACE ks WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': 3}",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.",
+                           KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+        assertFails("ALTER KEYSPACE ks WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': 5}",
+                    format("The keyspace %s has a replication factor of 5, above the failure threshold of %s.",
+                           KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+    }
+
+    @Test
+    public void testMultipleDatacenter() throws Throwable
+    {
+        IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
+        DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
+        {
+            public static final String RACK1 = ServerTestUtils.RACK1;
+
+            @Override
+            public String getRack(InetAddressAndPort endpoint) { return RACK1; }
+
+            @Override
+            public String getDatacenter(InetAddressAndPort endpoint) { return "datacenter2"; }
+
+            @Override
+            public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) { return 0; }
+        });
+
+        List<String> twoWarnings = Arrays.asList(format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD),
+                                                 format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+
+        StorageService.instance.getTokenMetadata().updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.0.0.255"));
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertValid("CREATE KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 2 };");
+
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 3 };",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 3, 'datacenter2' : 3 };", twoWarnings);
+
+        assertFails("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 5 };",
+                    format("The keyspace %s has a replication factor of 5, above the failure threshold of %s", KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+
+        assertFails("CREATE KEYSPACE ks1 WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 5, 'datacenter2' : 5 };",
+                    format("The keyspace ks1 has a replication factor of 5, above the failure threshold of %s", MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+
+        DatabaseDescriptor.setEndpointSnitch(snitch);
+        execute("DROP KEYSPACE IF EXISTS ks1");
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyWarnBelow() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, DISABLED_GUARDRAIL);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 2}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 1}");
+

Review Comment:
   Nit: unneeded blank line.



##########
test/unit/org/apache/cassandra/db/guardrails/GuardrailMaximumReplicationFactorTest.java:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import org.junit.After;
+import org.junit.Test;
+
+import org.apache.cassandra.ServerTestUtils;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.StorageService;
+
+import static java.lang.String.format;
+
+public class GuardrailMaximumReplicationFactorTest extends ThresholdTester
+{
+    private static final int MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD = 2;
+    private static final int MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD = 4;
+    private static final int DISABLED_GUARDRAIL = -1;
+    private static final String WHAT = "maximum_replication_factor";
+    private static final String KS = "ks";
+
+    public GuardrailMaximumReplicationFactorTest()
+    {
+        super(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD,
+              MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD,
+              Guardrails.maximumReplicationFactor,
+              Guardrails::setMaximumReplicationFactorThreshold,
+              Guardrails::getMaximumReplicationFactorWarnThreshold,
+              Guardrails::getMaximumReplicationFactorFailThreshold);
+    }
+
+    @After
+    public void cleanupTest() throws Throwable
+    {
+        execute("DROP KEYSPACE IF EXISTS ks");
+    }
+
+    @Override
+    protected long currentValue()
+    {
+        return Long.parseLong((Keyspace.open("ks").getReplicationStrategy()).configOptions.get("datacenter1"));
+    }
+
+    @Override
+    protected List<String> getWarnings()
+    {
+        List<String> warnings = ClientWarn.instance.getWarnings();
+
+        return warnings == null
+               ? Collections.emptyList()
+               : warnings.stream()
+                         .filter(w -> !w.contains("keyspace ks is higher than the number of nodes 1 for datacenter1") &&
+                                      !w.contains("When increasing replication factor you need to run a full (-full) repair to distribute the data") &&
+                                      !w.contains("keyspace ks is higher than the number of nodes") &&
+                                      !w.contains("Your replication factor 3 for keyspace ks is higher than the number of nodes 2 for datacenter datacenter2"))
+                         .collect(Collectors.toList());
+    }
+
+    @Test
+    public void testMaxKeyspaceRFDisabled() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(DISABLED_GUARDRAIL, DISABLED_GUARDRAIL);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 6}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 10}");
+    }
+
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertWarns("CREATE KEYSPACE ks WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': 3}",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.",
+                           KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+        assertFails("ALTER KEYSPACE ks WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': 5}",
+                    format("The keyspace %s has a replication factor of 5, above the failure threshold of %s.",
+                           KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+    }
+
+    @Test
+    public void testMultipleDatacenter() throws Throwable
+    {
+        IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
+        DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
+        {
+            public static final String RACK1 = ServerTestUtils.RACK1;
+
+            @Override
+            public String getRack(InetAddressAndPort endpoint) { return RACK1; }
+
+            @Override
+            public String getDatacenter(InetAddressAndPort endpoint) { return "datacenter2"; }
+
+            @Override
+            public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) { return 0; }
+        });
+
+        List<String> twoWarnings = Arrays.asList(format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD),
+                                                 format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+
+        StorageService.instance.getTokenMetadata().updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.0.0.255"));
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertValid("CREATE KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 2 };");
+
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 3 };",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 3, 'datacenter2' : 3 };", twoWarnings);
+
+        assertFails("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 5 };",
+                    format("The keyspace %s has a replication factor of 5, above the failure threshold of %s", KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+
+        assertFails("CREATE KEYSPACE ks1 WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 5, 'datacenter2' : 5 };",
+                    format("The keyspace ks1 has a replication factor of 5, above the failure threshold of %s", MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+
+        DatabaseDescriptor.setEndpointSnitch(snitch);
+        execute("DROP KEYSPACE IF EXISTS ks1");
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyWarnBelow() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, DISABLED_GUARDRAIL);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 2}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 1}");
+
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyWarnAbove() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, DISABLED_GUARDRAIL);
+        assertWarns("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 3}",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 4}",
+                    format("The keyspace %s has a replication factor of 4, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyFailBelow() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(DISABLED_GUARDRAIL, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 2}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 3}");
+
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyFailAbove() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(DISABLED_GUARDRAIL, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertFails("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 5}",
+                    format("The keyspace %s has a replication factor of 5, above the failure threshold of %s", KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyFailAboveAlter() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(DISABLED_GUARDRAIL, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        execute("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 3}");
+        assertFails("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 6}",
+                    format("The keyspace %s has a replication factor of 6, above the failure threshold of %s", KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+    }
+
+    @Test
+    public void testMaxKeyspaceRFWarnBelow() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 1}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 2}");
+

Review Comment:
   Nit: unneeded blank line



##########
test/unit/org/apache/cassandra/db/guardrails/GuardrailMaximumReplicationFactorTest.java:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import org.junit.After;
+import org.junit.Test;
+
+import org.apache.cassandra.ServerTestUtils;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.StorageService;
+
+import static java.lang.String.format;
+
+public class GuardrailMaximumReplicationFactorTest extends ThresholdTester
+{
+    private static final int MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD = 2;
+    private static final int MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD = 4;
+    private static final int DISABLED_GUARDRAIL = -1;
+    private static final String WHAT = "maximum_replication_factor";
+    private static final String KS = "ks";
+
+    public GuardrailMaximumReplicationFactorTest()
+    {
+        super(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD,
+              MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD,
+              Guardrails.maximumReplicationFactor,
+              Guardrails::setMaximumReplicationFactorThreshold,
+              Guardrails::getMaximumReplicationFactorWarnThreshold,
+              Guardrails::getMaximumReplicationFactorFailThreshold);
+    }
+
+    @After
+    public void cleanupTest() throws Throwable
+    {
+        execute("DROP KEYSPACE IF EXISTS ks");
+    }
+
+    @Override
+    protected long currentValue()
+    {
+        return Long.parseLong((Keyspace.open("ks").getReplicationStrategy()).configOptions.get("datacenter1"));
+    }
+
+    @Override
+    protected List<String> getWarnings()
+    {
+        List<String> warnings = ClientWarn.instance.getWarnings();
+
+        return warnings == null
+               ? Collections.emptyList()
+               : warnings.stream()
+                         .filter(w -> !w.contains("keyspace ks is higher than the number of nodes 1 for datacenter1") &&
+                                      !w.contains("When increasing replication factor you need to run a full (-full) repair to distribute the data") &&
+                                      !w.contains("keyspace ks is higher than the number of nodes") &&
+                                      !w.contains("Your replication factor 3 for keyspace ks is higher than the number of nodes 2 for datacenter datacenter2"))
+                         .collect(Collectors.toList());
+    }
+
+    @Test
+    public void testMaxKeyspaceRFDisabled() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(DISABLED_GUARDRAIL, DISABLED_GUARDRAIL);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 6}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 10}");
+    }
+
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertWarns("CREATE KEYSPACE ks WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': 3}",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.",
+                           KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+        assertFails("ALTER KEYSPACE ks WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': 5}",
+                    format("The keyspace %s has a replication factor of 5, above the failure threshold of %s.",
+                           KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+    }
+
+    @Test
+    public void testMultipleDatacenter() throws Throwable
+    {
+        IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
+        DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
+        {
+            public static final String RACK1 = ServerTestUtils.RACK1;
+
+            @Override
+            public String getRack(InetAddressAndPort endpoint) { return RACK1; }
+
+            @Override
+            public String getDatacenter(InetAddressAndPort endpoint) { return "datacenter2"; }
+
+            @Override
+            public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) { return 0; }
+        });
+
+        List<String> twoWarnings = Arrays.asList(format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD),
+                                                 format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+
+        StorageService.instance.getTokenMetadata().updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.0.0.255"));
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertValid("CREATE KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 2 };");
+
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 3 };",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 3, 'datacenter2' : 3 };", twoWarnings);
+
+        assertFails("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 5 };",
+                    format("The keyspace %s has a replication factor of 5, above the failure threshold of %s", KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+
+        assertFails("CREATE KEYSPACE ks1 WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 5, 'datacenter2' : 5 };",
+                    format("The keyspace ks1 has a replication factor of 5, above the failure threshold of %s", MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+
+        DatabaseDescriptor.setEndpointSnitch(snitch);
+        execute("DROP KEYSPACE IF EXISTS ks1");
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyWarnBelow() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, DISABLED_GUARDRAIL);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 2}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 1}");
+
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyWarnAbove() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, DISABLED_GUARDRAIL);
+        assertWarns("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 3}",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 4}",
+                    format("The keyspace %s has a replication factor of 4, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyFailBelow() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(DISABLED_GUARDRAIL, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 2}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 3}");
+
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyFailAbove() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(DISABLED_GUARDRAIL, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertFails("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 5}",
+                    format("The keyspace %s has a replication factor of 5, above the failure threshold of %s", KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+    }
+
+    @Test
+    public void testMaxKeyspaceRFOnlyFailAboveAlter() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(DISABLED_GUARDRAIL, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        execute("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 3}");
+        assertFails("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 6}",
+                    format("The keyspace %s has a replication factor of 6, above the failure threshold of %s", KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+    }
+
+    @Test
+    public void testMaxKeyspaceRFWarnBelow() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertMaxThresholdValid("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 1}");
+        assertMaxThresholdValid("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 2}");
+
+    }
+
+    @Test
+    public void testMaxKeyspaceRFWarnFailBetween() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertWarns("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 3}",
+                    format("The keyspace %s has a replication factor of 3, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+        assertWarns("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 4}",
+                    format("The keyspace %s has a replication factor of 4, above the warning threshold of %s.", KS, MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+    }
+
+    @Test
+    public void testMaxKeyspaceRFFailAbove() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        assertFails("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 5}",
+                    format("he keyspace %s has a replication factor of 5, above the failure threshold of %s", KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+    }
+
+    @Test
+    public void testMaxKeyspaceRFFailAboveAlter() throws Throwable
+    {
+        guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+        execute("CREATE KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 4}");
+       assertFails("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 5}",
+                   format("he keyspace %s has a replication factor of 5, above the failure threshold of %s", KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));

Review Comment:
   ```suggestion
           assertFails("ALTER KEYSPACE ks WITH replication = { 'class': 'NetworkTopologyStrategy', 'datacenter1': 5}",
                       format("The keyspace %s has a replication factor of 5, above the failure threshold of %s", KS, MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
   ```



##########
src/java/org/apache/cassandra/config/GuardrailsOptions.java:
##########
@@ -82,6 +82,7 @@ public GuardrailsOptions(Config config)
         validatePercentageThreshold(config.data_disk_usage_percentage_warn_threshold, config.data_disk_usage_percentage_fail_threshold, "data_disk_usage_percentage");
         validateDataDiskUsageMaxDiskSize(config.data_disk_usage_max_disk_size);
         validateMinRFThreshold(config.minimum_replication_factor_warn_threshold, config.minimum_replication_factor_fail_threshold, "minimum_replication_factor");
+        validateMaxIntThreshold(config.maximum_replication_factor_warn_threshold, config.maximum_replication_factor_fail_threshold, "maximum_replication_factor");

Review Comment:
   Probably we should validate this against `default_keyspace_rf`, as we did with the guardrail for min RF.



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