This is an automated email from the ASF dual-hosted git repository.
adelapena 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 39e89fd636 Add guardrail for maximum replication factor
39e89fd636 is described below
commit 39e89fd636ee4343eb2201820da87881cbc749e2
Author: Savni Nagarkar <[email protected]>
AuthorDate: Mon Apr 18 16:50:19 2022 -0400
Add guardrail for maximum replication factor
Patch by Savni Nagarkar; reviewed by Andrés de la Peña, David Capwell and
Josh McKenzie for CASSANDRA-17500
---
CHANGES.txt | 1 +
NEWS.txt | 1 +
conf/cassandra.yaml | 5 +
src/java/org/apache/cassandra/config/Config.java | 2 +
.../cassandra/config/DatabaseDescriptor.java | 5 +
.../apache/cassandra/config/GuardrailsOptions.java | 59 +++--
.../apache/cassandra/db/guardrails/Guardrails.java | 35 ++-
.../cassandra/db/guardrails/GuardrailsConfig.java | 9 +
.../cassandra/db/guardrails/GuardrailsMBean.java | 29 ++-
.../cassandra/locator/NetworkTopologyStrategy.java | 1 +
.../apache/cassandra/locator/SimpleStrategy.java | 1 +
.../GuardrailMaximumReplicationFactorTest.java | 254 +++++++++++++++++++++
.../GuardrailMinimumReplicationFactorTest.java | 105 +++++----
13 files changed, 439 insertions(+), 68 deletions(-)
diff --git a/CHANGES.txt b/CHANGES.txt
index bac9b15dbb..8dc91eee3f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
4.2
+ * Add guardrail for maximum replication factor (CASSANDRA-17500)
* Increment CQLSH to version 6.2.0 for release 4.2 (CASSANDRA-17646)
* Adding support to perform certificate based internode authentication
(CASSANDRA-17661)
* Option to disable CDC writes of repaired data (CASSANDRA-17666)
diff --git a/NEWS.txt b/NEWS.txt
index 996113d7c7..c9edaa2a7a 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -69,6 +69,7 @@ New features
- New Guardrails added:
- Whether ALTER TABLE commands are allowed to mutate columns
- Whether SimpleStrategy is allowed on keyspace creation or alteration
+ - Maximum replication factor
Upgrading
---------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 3bab6712c8..eb1d1a97c0 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -1812,6 +1812,11 @@ drop_compact_storage_enabled: false
# Suggested value for use in production: 2 or higher
# minimum_replication_factor_warn_threshold: -1
# minimum_replication_factor_fail_threshold: -1
+#
+# Guardrail to warn or fail when the maximum replication factor is greater
than threshold.
+# This would also apply to system keyspaces.
+# maximum_replication_factor_warn_threshold: -1
+# maximum_replication_factor_fail_threshold: -1
# 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.
diff --git a/src/java/org/apache/cassandra/config/Config.java
b/src/java/org/apache/cassandra/config/Config.java
index 3d2dbb7b40..3048a9a411 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -848,6 +848,8 @@ public class Config
public volatile DataStorageSpec.LongBytesBound
data_disk_usage_max_disk_size = null;
public volatile int minimum_replication_factor_warn_threshold = -1;
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 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/DatabaseDescriptor.java
b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 8151c96871..2bd1aa8400 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -4089,6 +4089,11 @@ public class DatabaseDescriptor
throw new
IllegalArgumentException(String.format("default_keyspace_rf to be set (%d)
cannot be less than minimum_replication_factor_fail_threshold (%d)", value,
guardrails.getMinimumReplicationFactorFailThreshold()));
}
+ if (guardrails.getMaximumReplicationFactorFailThreshold() != -1 &&
value > guardrails.getMaximumReplicationFactorFailThreshold())
+ {
+ throw new
IllegalArgumentException(String.format("default_keyspace_rf to be set (%d)
cannot be greater than maximum_replication_factor_fail_threshold (%d)", value,
guardrails.getMaximumReplicationFactorFailThreshold()));
+ }
+
conf.default_keyspace_rf = value;
}
diff --git a/src/java/org/apache/cassandra/config/GuardrailsOptions.java
b/src/java/org/apache/cassandra/config/GuardrailsOptions.java
index e8d7bda77a..98d14a1d32 100644
--- a/src/java/org/apache/cassandra/config/GuardrailsOptions.java
+++ b/src/java/org/apache/cassandra/config/GuardrailsOptions.java
@@ -81,7 +81,8 @@ public class GuardrailsOptions implements GuardrailsConfig
validateMaxIntThreshold(config.fields_per_udt_warn_threshold,
config.fields_per_udt_fail_threshold, "fields_per_udt");
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");
+
validateMinRFThreshold(config.minimum_replication_factor_warn_threshold,
config.minimum_replication_factor_fail_threshold);
+
validateMaxRFThreshold(config.maximum_replication_factor_warn_threshold,
config.maximum_replication_factor_fail_threshold);
}
@Override
@@ -651,7 +652,7 @@ public class GuardrailsOptions implements GuardrailsConfig
public void setMinimumReplicationFactorThreshold(int warn, int fail)
{
- validateMinRFThreshold(warn, fail, "minimum_replication_factor");
+ validateMinRFThreshold(warn, fail);
updatePropertyWithLogging("minimum_replication_factor_warn_threshold",
warn,
() ->
config.minimum_replication_factor_warn_threshold,
@@ -662,6 +663,31 @@ public class GuardrailsOptions implements GuardrailsConfig
x ->
config.minimum_replication_factor_fail_threshold = x);
}
+ @Override
+ public int getMaximumReplicationFactorWarnThreshold()
+ {
+ return config.maximum_replication_factor_warn_threshold;
+ }
+
+ @Override
+ public int getMaximumReplicationFactorFailThreshold()
+ {
+ return config.maximum_replication_factor_fail_threshold;
+ }
+
+ public void setMaximumReplicationFactorThreshold(int warn, int fail)
+ {
+ validateMaxRFThreshold(warn, fail);
+ updatePropertyWithLogging("maximum_replication_factor_warn_threshold",
+ warn,
+ () ->
config.maximum_replication_factor_warn_threshold,
+ x ->
config.maximum_replication_factor_warn_threshold = x);
+ updatePropertyWithLogging("maximum_replication_factor_fail_threshold",
+ fail,
+ () ->
config.maximum_replication_factor_fail_threshold,
+ x ->
config.maximum_replication_factor_fail_threshold = x);
+ }
+
private static <T> void updatePropertyWithLogging(String propertyName, T
newValue, Supplier<T> getter, Consumer<T> setter)
{
T oldValue = getter.get();
@@ -717,10 +743,24 @@ public class GuardrailsOptions implements GuardrailsConfig
validateWarnGreaterThanFail(warn, fail, name);
}
- private static void validateMinRFThreshold(int warn, int fail, String name)
+ private static void validateMinRFThreshold(int warn, int fail)
{
- validateMinIntThreshold(warn, fail, name);
- validateMinRFVersusDefaultRF(fail, name);
+ validateMinIntThreshold(warn, fail, "minimum_replication_factor");
+
+ if (fail > DatabaseDescriptor.getDefaultKeyspaceRF())
+ throw new
IllegalArgumentException(format("minimum_replication_factor_fail_threshold to
be set (%d) " +
+ "cannot be greater than
default_keyspace_rf (%d)",
+ fail,
DatabaseDescriptor.getDefaultKeyspaceRF()));
+ }
+
+ private static void validateMaxRFThreshold(int warn, int fail)
+ {
+ validateMaxIntThreshold(warn, fail, "maximum_replication_factor");
+
+ if (fail != -1 && fail < DatabaseDescriptor.getDefaultKeyspaceRF())
+ throw new
IllegalArgumentException(format("maximum_replication_factor_fail_threshold to
be set (%d) " +
+ "cannot be lesser than
default_keyspace_rf (%d)",
+ fail,
DatabaseDescriptor.getDefaultKeyspaceRF()));
}
private static void validateWarnLowerThanFail(long warn, long fail, String
name)
@@ -743,15 +783,6 @@ public class GuardrailsOptions implements GuardrailsConfig
"than the fail threshold
%d", warn, name, fail));
}
- private static void validateMinRFVersusDefaultRF(int fail, String name)
throws IllegalArgumentException
- {
- if (fail > DatabaseDescriptor.getDefaultKeyspaceRF())
- {
- throw new
IllegalArgumentException(String.format("%s_fail_threshold to be set (%d) cannot
be greater than default_keyspace_rf (%d)",
- name, fail,
DatabaseDescriptor.getDefaultKeyspaceRF()));
- }
- }
-
private static void validateSize(DataStorageSpec.LongBytesBound size,
boolean allowZero, String name)
{
if (size == null)
diff --git a/src/java/org/apache/cassandra/db/guardrails/Guardrails.java
b/src/java/org/apache/cassandra/db/guardrails/Guardrails.java
index 16146fec87..36bb3d446f 100644
--- a/src/java/org/apache/cassandra/db/guardrails/Guardrails.java
+++ b/src/java/org/apache/cassandra/db/guardrails/Guardrails.java
@@ -344,10 +344,19 @@ public final class Guardrails implements GuardrailsMBean
state ->
CONFIG_PROVIDER.getOrCreate(state).getMinimumReplicationFactorWarnThreshold(),
state ->
CONFIG_PROVIDER.getOrCreate(state).getMinimumReplicationFactorFailThreshold(),
(isWarning, what, value, threshold) ->
- isWarning ? format("The keyspace %s has a replication
factor of %s, below the warning threshold of %s.",
- what, value, threshold)
- : format("The keyspace %s has a replication
factor of %s, below the failure threshold of %s.",
- what, value, threshold));
+ format("The keyspace %s has a replication factor of %s,
below the %s threshold of %s.",
+ what, value, isWarning ? "warning" : "failure",
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) ->
+ format("The keyspace %s has a replication factor of %s,
above the %s threshold of %s.",
+ what, value, isWarning ? "warning" : "failure",
threshold));
private Guardrails()
{
@@ -857,6 +866,24 @@ public final class Guardrails implements GuardrailsMBean
DEFAULT_CONFIG.setFieldsPerUDTThreshold(warn, fail);
}
+ @Override
+ public int getMaximumReplicationFactorWarnThreshold()
+ {
+ return DEFAULT_CONFIG.getMaximumReplicationFactorWarnThreshold();
+ }
+
+ @Override
+ public int getMaximumReplicationFactorFailThreshold()
+ {
+ return DEFAULT_CONFIG.getMaximumReplicationFactorFailThreshold();
+ }
+
+ @Override
+ public void setMaximumReplicationFactorThreshold (int warn, int fail)
+ {
+ DEFAULT_CONFIG.setMaximumReplicationFactorThreshold(warn, fail);
+ }
+
@Override
public int getDataDiskUsagePercentageWarnThreshold()
{
diff --git a/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java
b/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java
index 72eaaa5b48..c7067b53e0 100644
--- a/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java
+++ b/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java
@@ -291,4 +291,13 @@ public interface GuardrailsConfig
*/
int getMinimumReplicationFactorFailThreshold();
+ /**
+ * @return The threshold to warn when replication factor is greater than
threshold.
+ */
+ int getMaximumReplicationFactorWarnThreshold();
+
+ /**
+ * @return The threshold to fail when replication factor is greater than
threshold.
+ */
+ int getMaximumReplicationFactorFailThreshold();
}
diff --git a/src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java
b/src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java
index 47db91a6fa..dc3fb48e22 100644
--- a/src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java
+++ b/src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java
@@ -550,21 +550,38 @@ public interface GuardrailsMBean
void setDataDiskUsageMaxDiskSize(@Nullable String size);
/**
- * @return The threshold to warn when replication factor is lesser
threshold.
+ * @return The threshold to warn when replication factor is lesser than
threshold.
*/
int getMinimumReplicationFactorWarnThreshold();
/**
- * @return The threshold to fail when replication factor is lesser
threshold.
+ * @return The threshold to fail when replication factor is lesser than
threshold.
*/
int getMinimumReplicationFactorFailThreshold();
/**
- * @param warn the threshold to warn when the minimum replication factor
is lesser than
- * threshold -1 means disabled.
- * @param fail the threshold to fail when the minimum replication factor
is lesser than
- * threshold -1 means disabled.
+ * @param warn The threshold to warn when the minimum replication factor
is lesser than threshold.
+ * -1 means disabled.
+ * @param fail The threshold to fail when the minimum replication factor
is lesser than threshold.
+ * -1 means disabled.
*/
void setMinimumReplicationFactorThreshold (int warn, int fail);
+ /**
+ * @return The threshold to fail when replication factor is greater than
threshold.
+ */
+ int getMaximumReplicationFactorWarnThreshold();
+
+ /**
+ * @return The threshold to fail when replication factor is greater than
threshold.
+ */
+ int getMaximumReplicationFactorFailThreshold();
+
+ /**
+ * @param warn The threshold to warn when the maximum replication factor
is greater than threshold.
+ * -1 means disabled.
+ * @param fail The threshold to fail when the maximum replication factor
is greater than threshold.
+ * -1 means disabled.
+ */
+ void setMaximumReplicationFactorThreshold (int warn, int fail);
}
diff --git a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
index 9ae034121a..490c46b2f2 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -350,6 +350,7 @@ public class NetworkTopologyStrategy extends
AbstractReplicationStrategy
String dc = e.getKey();
ReplicationFactor rf = getReplicationFactor(dc);
Guardrails.minimumReplicationFactor.guard(rf.fullReplicas,
keyspaceName, false, state);
+ Guardrails.maximumReplicationFactor.guard(rf.fullReplicas,
keyspaceName, false, state);
int nodeCount = dcsNodes.get(dc).size();
// nodeCount==0 on many tests
if (rf.fullReplicas > nodeCount && nodeCount != 0)
diff --git a/src/java/org/apache/cassandra/locator/SimpleStrategy.java
b/src/java/org/apache/cassandra/locator/SimpleStrategy.java
index e5b92103b4..488b601ce7 100644
--- a/src/java/org/apache/cassandra/locator/SimpleStrategy.java
+++ b/src/java/org/apache/cassandra/locator/SimpleStrategy.java
@@ -109,6 +109,7 @@ public class SimpleStrategy extends
AbstractReplicationStrategy
int nodeCount =
StorageService.instance.getHostIdToEndpoint().size();
// nodeCount==0 on many tests
Guardrails.minimumReplicationFactor.guard(rf.fullReplicas,
keyspaceName, false, state);
+ Guardrails.maximumReplicationFactor.guard(rf.fullReplicas,
keyspaceName, false, state);
if (rf.fullReplicas > nodeCount && nodeCount != 0)
{
String msg = "Your replication factor " + rf.fullReplicas
diff --git
a/test/unit/org/apache/cassandra/db/guardrails/GuardrailMaximumReplicationFactorTest.java
b/test/unit/org/apache/cassandra/db/guardrails/GuardrailMaximumReplicationFactorTest.java
new file mode 100644
index 0000000000..865ac23c79
--- /dev/null
+++
b/test/unit/org/apache/cassandra/db/guardrails/GuardrailMaximumReplicationFactorTest.java
@@ -0,0 +1,254 @@
+/*
+ * 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 org.assertj.core.api.Assertions;
+
+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;
+
+ 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");
+ DatabaseDescriptor.setDefaultKeyspaceRF(1);
+ }
+
+ @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();
+
+ // filtering out non-guardrails produced warnings
+ 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 testSimpleStrategyCreate() 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}", 3);
+ execute("DROP KEYSPACE IF EXISTS ks");
+ assertFails("CREATE KEYSPACE ks WITH replication = { 'class':
'SimpleStrategy', 'replication_factor': 5}", 5);
+ }
+
+ @Test
+ public void testSimpleStrategyAlter() throws Throwable
+ {
+
guardrails().setMaximumReplicationFactorThreshold(MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD,
MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+ execute("CREATE KEYSPACE ks WITH replication = { 'class':
'SimpleStrategy', 'replication_factor': 2}");
+ assertWarns("ALTER KEYSPACE ks WITH replication = { 'class':
'SimpleStrategy', 'replication_factor': 3}", 3);
+ assertFails("ALTER KEYSPACE ks WITH replication = { 'class':
'SimpleStrategy', 'replication_factor': 5}", 5);
+ }
+
+ @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 ks has a
replication factor of 3, above the warning threshold of %s.",
MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD),
+ format("The keyspace ks has a
replication factor of 3, above the warning threshold of %s.",
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}");
+ execute("DROP KEYSPACE IF EXISTS ks");
+ assertWarns("CREATE KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 3}", 3);
+ execute("DROP KEYSPACE IF EXISTS ks");
+ assertWarns("CREATE KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 3, 'datacenter2' : 3}", twoWarnings);
+ execute("DROP KEYSPACE IF EXISTS ks");
+ assertFails("CREATE KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 5}", 5);
+ execute("DROP KEYSPACE IF EXISTS ks");
+ assertFails("CREATE KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 5, 'datacenter2' : 5}", 5);
+ execute("DROP KEYSPACE IF EXISTS ks");
+
+ execute("CREATE KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 1, 'datacenter2' : 1}");
+ assertValid("ALTER KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 2}");
+ assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 3}", 3);
+ 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}", 5);
+ assertFails("ALTER KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 5, 'datacenter2' : 5}", 5);
+
+ DatabaseDescriptor.setEndpointSnitch(snitch);
+ }
+
+ @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}", 3);
+ assertWarns("ALTER KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 4}", 4);
+ }
+
+ @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}", 5);
+ }
+
+ @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}", 6);
+ }
+
+ @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}", 3);
+ assertWarns("ALTER KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 4}", 4);
+ }
+
+ @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}", 5);
+ }
+
+ @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}", 5);
+ }
+
+ @Test
+ public void testMaxRFLesserThanDefaultRF()
+ {
+ DatabaseDescriptor.setDefaultKeyspaceRF(3);
+ Assertions.assertThatThrownBy(() ->
guardrails().setMaximumReplicationFactorThreshold(1, 2))
+ .isInstanceOf(IllegalArgumentException.class)
+
.hasMessageContaining("maximum_replication_factor_fail_threshold to be set (2)
cannot be lesser than default_keyspace_rf (3)");
+
+ DatabaseDescriptor.setDefaultKeyspaceRF(1);
+ guardrails().setMaximumReplicationFactorThreshold(1, 2);
+ Assertions.assertThatThrownBy(() ->
DatabaseDescriptor.setDefaultKeyspaceRF(3))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessageContaining("default_keyspace_rf to be set (3)
cannot be greater than maximum_replication_factor_fail_threshold (2)");
+ }
+
+ private void assertWarns(String query, int rf) throws Throwable
+ {
+ assertWarns(query, format("The keyspace ks has a replication factor of
%d, above the warning threshold of %s.",
+ rf,
MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+ }
+
+ private void assertFails(String query, int rf) throws Throwable
+ {
+ assertFails(query, format("The keyspace ks has a replication factor of
%d, above the failure threshold of %s.",
+ rf,
MAXIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+ }
+}
diff --git
a/test/unit/org/apache/cassandra/db/guardrails/GuardrailMinimumReplicationFactorTest.java
b/test/unit/org/apache/cassandra/db/guardrails/GuardrailMinimumReplicationFactorTest.java
index 9c984e99f5..8817f9a8c6 100644
---
a/test/unit/org/apache/cassandra/db/guardrails/GuardrailMinimumReplicationFactorTest.java
+++
b/test/unit/org/apache/cassandra/db/guardrails/GuardrailMinimumReplicationFactorTest.java
@@ -31,7 +31,6 @@ 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.exceptions.ConfigurationException;
import org.apache.cassandra.locator.AbstractEndpointSnitch;
import org.apache.cassandra.locator.IEndpointSnitch;
import org.apache.cassandra.locator.InetAddressAndPort;
@@ -90,6 +89,7 @@ public class GuardrailMinimumReplicationFactorTest extends
ThresholdTester
{
List<String> warnings = ClientWarn.instance.getWarnings();
+ // filtering out non-guardrails produced warnings
return warnings == null
? Collections.emptyList()
: warnings.stream()
@@ -133,13 +133,21 @@ public class GuardrailMinimumReplicationFactorTest
extends ThresholdTester
}
@Test
- public void testSimpleStrategy() throws Throwable
+ public void testSimpleStrategyCreate() throws Throwable
{
guardrails().setMinimumReplicationFactorThreshold(MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD,
MINIMUM_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,
below the warning threshold of %s.", KS,
MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
- assertFails("ALTER KEYSPACE ks WITH replication = { 'class':
'SimpleStrategy', 'replication_factor': 1}",
- format("The keyspace %s has a replication factor of 1,
below the failure threshold of %s.", KS,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+ assertWarns("CREATE KEYSPACE ks WITH replication = { 'class':
'SimpleStrategy', 'replication_factor': 3}", 3);
+ execute("DROP KEYSPACE IF EXISTS ks");
+ assertFails("CREATE KEYSPACE ks WITH replication = { 'class':
'SimpleStrategy', 'replication_factor': 1}", 1);
+ }
+
+ @Test
+ public void testSimpleStrategyAlter() throws Throwable
+ {
+
guardrails().setMinimumReplicationFactorThreshold(MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
+ execute("CREATE KEYSPACE ks WITH replication = { 'class':
'SimpleStrategy', 'replication_factor': 4}");
+ assertWarns("ALTER KEYSPACE ks WITH replication = { 'class':
'SimpleStrategy', 'replication_factor': 3}", 3);
+ assertFails("ALTER KEYSPACE ks WITH replication = { 'class':
'SimpleStrategy', 'replication_factor': 1}", 1);
}
@Test
@@ -162,17 +170,26 @@ public class GuardrailMinimumReplicationFactorTest
extends ThresholdTester
List<String> twoWarnings = Arrays.asList(format("The keyspace %s has a
replication factor of 2, below the warning threshold of %d.", KS,
MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD),
format("The keyspace %s has a
replication factor of 2, below the warning threshold of %d.", KS,
MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
-
+
StorageService.instance.getTokenMetadata().updateHostId(UUID.randomUUID(),
InetAddressAndPort.getByName("127.0.0.255"));
guardrails().setMinimumReplicationFactorThreshold(MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
- assertValid("CREATE KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 4, 'datacenter2' : 4 };");
- assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 4, 'datacenter2' : 2 };",
- format("The keyspace %s has a replication factor of 2,
below the warning threshold of %d.", KS,
MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
- assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 2 };",
twoWarnings);
- assertFails("ALTER KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 4, 'datacenter2' : 1 };",
- format("The keyspace %s has a replication factor of 1,
below the failure threshold of %d.", KS,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
- assertFails("CREATE KEYSPACE ks1 WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 1, 'datacenter2' : 1 };",
- format("The keyspace ks1 has a replication factor of 1,
below the failure threshold of %d.",
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+ assertValid("CREATE KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 4, 'datacenter2' : 4 }");
+ execute("DROP KEYSPACE IF EXISTS ks");
+ assertWarns("CREATE KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 4, 'datacenter2' : 2 }", 2);
+ execute("DROP KEYSPACE IF EXISTS ks");
+ assertWarns("CREATE KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 2 }", twoWarnings);
+ execute("DROP KEYSPACE IF EXISTS ks");
+ assertFails("CREATE KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 4, 'datacenter2' : 1 }", 1);
+ execute("DROP KEYSPACE IF EXISTS ks");
+ assertFails("CREATE KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 1, 'datacenter2' : 1 }", 1);
+ execute("DROP KEYSPACE IF EXISTS ks");
+
+ execute("CREATE KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 5, 'datacenter2' : 5}");
+ assertValid("ALTER KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 4, 'datacenter2' : 4 }");
+ assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 4, 'datacenter2' : 2 }", 2);
+ assertWarns("ALTER KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 2 }", twoWarnings);
+ assertFails("ALTER KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 4, 'datacenter2' : 1 }", 1);
+ assertFails("ALTER KEYSPACE ks WITH replication = { 'class' :
'NetworkTopologyStrategy', 'datacenter1': 1, 'datacenter2' : 1 }", 1);
DatabaseDescriptor.setEndpointSnitch(snitch);
execute("DROP KEYSPACE IF EXISTS ks1");
@@ -190,10 +207,8 @@ public class GuardrailMinimumReplicationFactorTest extends
ThresholdTester
public void testMinKeyspaceRFOnlyWarnBelow() throws Throwable
{
guardrails().setMinimumReplicationFactorThreshold(MINIMUM_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,
below the warning threshold of %s.", KS,
MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
- assertWarns("ALTER KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 2}",
- format("The keyspace %s has a replication factor of 2,
below the warning threshold of %s.", KS,
MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+ assertWarns("CREATE KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 3}", 3);
+ assertWarns("ALTER KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 2}", 2);
}
@Test
@@ -208,8 +223,7 @@ public class GuardrailMinimumReplicationFactorTest extends
ThresholdTester
public void testMinKeyspaceRFOnlyFailBelow() throws Throwable
{
guardrails().setMinimumReplicationFactorThreshold(DISABLED_GUARDRAIL,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
- assertFails("CREATE KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 1}",
- format("The keyspace %s has a replication factor of 1,
below the failure threshold of %s.", KS,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+ assertFails("CREATE KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 1}", 1);
}
@Test
@@ -217,8 +231,7 @@ public class GuardrailMinimumReplicationFactorTest extends
ThresholdTester
{
guardrails().setMinimumReplicationFactorThreshold(DISABLED_GUARDRAIL,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
execute("CREATE KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 3}");
- assertFails("ALTER KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 1}",
- format("The keyspace %s has a replication factor of 1,
below the failure threshold of %s.", KS,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+ assertFails("ALTER KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 1}", 1);
}
@Test
@@ -233,18 +246,15 @@ public class GuardrailMinimumReplicationFactorTest
extends ThresholdTester
public void testMinKeyspaceRFWarnFailBetween() throws Throwable
{
guardrails().setMinimumReplicationFactorThreshold(MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
- assertWarns("CREATE KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 3}",
- format("The keyspace %s has a replication factor of 3,
below the warning threshold of %s.", KS,
MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
- assertWarns("ALTER KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 2}",
- format("The keyspace %s has a replication factor of 2,
below the warning threshold of %s.", KS,
MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+ assertWarns("CREATE KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 3}", 3);
+ assertWarns("ALTER KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 2}", 2);
}
@Test
public void testMinKeyspaceRFFailBelow() throws Throwable
{
guardrails().setMinimumReplicationFactorThreshold(MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
- assertFails("CREATE KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 1}",
- format("The keyspace %s has a replication factor of 1,
below the failure threshold of %s.", KS,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+ assertFails("CREATE KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 1}", 1);
}
@Test
@@ -252,26 +262,33 @@ public class GuardrailMinimumReplicationFactorTest
extends ThresholdTester
{
guardrails().setMinimumReplicationFactorThreshold(MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
execute("CREATE KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 4}");
- assertFails("ALTER KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 1}",
- format("The keyspace %s has a replication factor of 1,
below the failure threshold of %s.", KS,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
+ assertFails("ALTER KEYSPACE ks WITH replication = { 'class':
'NetworkTopologyStrategy', 'datacenter1': 1}", 1);
}
@Test
public void testMinRFGreaterThanDefaultRF()
{
- try
- {
- DatabaseDescriptor.setDefaultKeyspaceRF(1);
-
guardrails().setMinimumReplicationFactorThreshold(MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD);
- }
- catch (IllegalArgumentException e)
- {
- String expectedMessage = "";
+ DatabaseDescriptor.setDefaultKeyspaceRF(3);
+ Assertions.assertThatThrownBy(() ->
guardrails().setMinimumReplicationFactorThreshold(5, 4))
+ .isInstanceOf(IllegalArgumentException.class)
+
.hasMessageContaining("minimum_replication_factor_fail_threshold to be set (4)
cannot be greater than default_keyspace_rf (3)");
+
+ DatabaseDescriptor.setDefaultKeyspaceRF(6);
+ guardrails().setMinimumReplicationFactorThreshold(5, 4);
+ Assertions.assertThatThrownBy(() ->
DatabaseDescriptor.setDefaultKeyspaceRF(3))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessageContaining("default_keyspace_rf to be set (3)
cannot be less than minimum_replication_factor_fail_threshold (4)");
+ }
- if(guardrails().getMinimumReplicationFactorFailThreshold() >
DatabaseDescriptor.getDefaultKeyspaceRF())
- expectedMessage = format("%s_fail_threshold to be set (%d)
cannot be greater than default_keyspace_rf (%d)",
- WHAT,
guardrails().getMinimumReplicationFactorFailThreshold(),
DatabaseDescriptor.getDefaultKeyspaceRF());
- Assertions.assertThat(e.getMessage()).contains(expectedMessage);
- }
+ private void assertWarns(String query, int rf) throws Throwable
+ {
+ assertWarns(query, format("The keyspace ks has a replication factor of
%d, below the warning threshold of %s.",
+ rf,
MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD));
+ }
+
+ private void assertFails(String query, int rf) throws Throwable
+ {
+ assertFails(query, format("The keyspace ks has a replication factor of
%d, below the failure threshold of %s.",
+ rf,
MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD));
}
}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]