You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by mc...@apache.org on 2020/11/17 21:55:13 UTC
[cassandra] branch cassandra-3.0 updated: Improved check of
num_tokens against initial_token in the cassandra.yaml
This is an automated email from the ASF dual-hosted git repository.
mck pushed a commit to branch cassandra-3.0
in repository https://gitbox.apache.org/repos/asf/cassandra.git
The following commit(s) were added to refs/heads/cassandra-3.0 by this push:
new 8ef5a88 Improved check of num_tokens against initial_token in the cassandra.yaml
8ef5a88 is described below
commit 8ef5a886312e20f09cd4b0358c71018908341796
Author: Stefan Miklosovic <st...@instaclustr.com>
AuthorDate: Thu Oct 29 17:14:22 2020 +0100
Improved check of num_tokens against initial_token in the cassandra.yaml
patch by Stefan Miklosovic; reviewed by Mick Semb Wever for CASSANDRA-14477
---
CHANGES.txt | 1 +
NEWS.txt | 7 +-
src/java/org/apache/cassandra/config/Config.java | 2 +-
.../cassandra/config/DatabaseDescriptor.java | 47 +++++---
.../cassandra/config/DatabaseDescriptorTest.java | 134 ++++++++++++++++++++-
5 files changed, 171 insertions(+), 20 deletions(-)
diff --git a/CHANGES.txt b/CHANGES.txt
index 41daa6a..7aecefa 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
3.0.24:
+ * Improved check of num_tokens against the length of initial_token (CASSANDRA-14477)
* Fix a race condition on ColumnFamilyStore and TableMetrics (CASSANDRA-16228)
* Remove the SEPExecutor blocking behavior (CASSANDRA-16186)
* Wait for schema agreement when bootstrapping (CASSANDRA-15158)
diff --git a/NEWS.txt b/NEWS.txt
index 5a2ef51..42fbf63 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -42,13 +42,14 @@ restore snapshots created with the previous major version using the
'sstableloader' tool. You can upgrade the file format of your snapshots
using the provided 'sstableupgrade' tool.
-3.0.21
+3.0.24
======
Upgrading
---------
- - Nothing specific to this release, but please see previous upgrading sections,
- especially if you are upgrading from 2.2.
+ - In cassandra.yaml, num_tokens must be defined if initial_token is defined.
+ If it is not defined, or not equal to the numbers of tokens defined in initial_tokens,
+ the node will not start. See CASSANDRA-14477 for details.
3.0.20
======
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 2218ee2..277a68a 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -79,7 +79,7 @@ public class Config
/* initial token in the ring */
public String initial_token;
- public Integer num_tokens = 1;
+ public Integer num_tokens;
/** Triggers automatic allocation of tokens if set, using the replication strategy of the referenced keyspace */
public String allocate_tokens_for_keyspace = null;
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 9369229..04293fb 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -289,6 +289,37 @@ public class DatabaseDescriptor
}
}
+ @VisibleForTesting
+ static void applyTokensConfig(Config config) throws ConfigurationException
+ {
+ if (config.num_tokens != null && config.num_tokens > MAX_NUM_TOKENS)
+ throw new ConfigurationException(String.format("A maximum number of %d tokens per node is supported", MAX_NUM_TOKENS), false);
+
+ if (config.initial_token != null)
+ {
+ Collection<String> tokens = tokensFromString(config.initial_token);
+ if (config.num_tokens == null)
+ {
+ throw new ConfigurationException("initial_token was set but num_tokens is not!", false);
+ }
+
+ if (tokens.size() != config.num_tokens)
+ {
+ throw new ConfigurationException(String.format("The number of initial tokens (by initial_token) specified (%s) is different from num_tokens value (%s)",
+ tokens.size(),
+ config.num_tokens),
+ false);
+ }
+
+ for (String token : tokens)
+ partitioner.getTokenFactory().validate(token);
+ }
+ else if (config.num_tokens == null)
+ {
+ config.num_tokens = 1;
+ }
+ }
+
public static void applyConfig(Config config) throws ConfigurationException
{
conf = config;
@@ -655,21 +686,7 @@ public class DatabaseDescriptor
if (conf.concurrent_compactors <= 0)
throw new ConfigurationException("concurrent_compactors should be strictly greater than 0, but was " + conf.concurrent_compactors, false);
- if (conf.num_tokens == null)
- conf.num_tokens = 1;
- else if (conf.num_tokens > MAX_NUM_TOKENS)
- throw new ConfigurationException(String.format("A maximum number of %d tokens per node is supported", MAX_NUM_TOKENS), false);
-
- if (conf.initial_token != null)
- {
- Collection<String> tokens = tokensFromString(conf.initial_token);
- if (tokens.size() != conf.num_tokens)
- throw new ConfigurationException("The number of initial tokens (by initial_token) specified is different from num_tokens value", false);
-
- for (String token : tokens)
- partitioner.getTokenFactory().validate(token);
- }
-
+ applyTokensConfig(config);
try
{
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 4a43388..7614e02 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -19,12 +19,17 @@
package org.apache.cassandra.config;
import java.io.IOException;
+import java.lang.management.ManagementFactory;
import java.net.Inet4Address;
import java.net.Inet6Address;
import java.net.InetAddress;
import java.net.NetworkInterface;
import java.util.Enumeration;
+import javax.management.InstanceNotFoundException;
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -308,4 +313,131 @@ public class DatabaseDescriptorTest
DatabaseDescriptor.setRepairSessionMaxTreeDepth(previousDepth);
}
}
-}
+
+ @Test
+ public void testApplyInitialTokensInitialTokensSetNumTokensSetAndDoesMatch() throws Exception
+ {
+ Config config = DatabaseDescriptor.loadConfig();
+ config.initial_token = "0,256,1024";
+ config.num_tokens = 3;
+
+ unregisterSnitchesForTokenConfigTest();
+
+ try
+ {
+ DatabaseDescriptor.applyTokensConfig(config);
+
+ Assert.assertEquals(Integer.valueOf(3), config.num_tokens);
+ Assert.assertEquals(3, DatabaseDescriptor.tokensFromString(config.initial_token).size());
+ }
+ finally
+ {
+ unregisterSnitchesForTokenConfigTest();
+ }
+ }
+
+ @Test
+ public void testApplyInitialTokensInitialTokensSetNumTokensSetAndDoesntMatch() throws Exception
+ {
+ Config config = DatabaseDescriptor.loadConfig();
+ config.initial_token = "0,256,1024";
+ config.num_tokens = 10;
+
+ unregisterSnitchesForTokenConfigTest();
+
+ try
+ {
+ DatabaseDescriptor.applyTokensConfig(config);
+
+ Assert.fail("initial_token = 0,256,1024 and num_tokens = 10 but applyInitialTokens() did not fail!");
+ }
+ catch (ConfigurationException ex)
+ {
+ Assert.assertEquals("The number of initial tokens (by initial_token) specified (3) is different from num_tokens value (10)",
+ ex.getMessage());
+ }
+ finally
+ {
+ unregisterSnitchesForTokenConfigTest();
+ }
+ }
+
+ @Test
+ public void testApplyInitialTokensInitialTokensSetNumTokensNotSet() throws Exception
+ {
+ Config config = DatabaseDescriptor.loadConfig();
+
+ unregisterSnitchesForTokenConfigTest();
+
+ try
+ {
+ config.initial_token = "0,256,1024";
+ config.num_tokens = null;
+ DatabaseDescriptor.applyTokensConfig(config);
+ Assert.fail("setting initial_token and not setting num_tokens is invalid");
+ }
+ catch (ConfigurationException ex)
+ {
+ Assert.assertEquals("initial_token was set but num_tokens is not!", ex.getMessage());
+ }
+ finally
+ {
+ unregisterSnitchesForTokenConfigTest();
+ }
+ }
+
+ @Test
+ public void testApplyInitialTokensInitialTokensNotSetNumTokensSet() throws Exception
+ {
+ Config config = DatabaseDescriptor.loadConfig();
+ config.num_tokens = 3;
+
+ unregisterSnitchesForTokenConfigTest();
+
+ try
+ {
+ DatabaseDescriptor.applyTokensConfig(config);
+ }
+ finally
+ {
+ unregisterSnitchesForTokenConfigTest();
+ }
+
+ Assert.assertEquals(Integer.valueOf(3), config.num_tokens);
+ Assert.assertTrue(DatabaseDescriptor.tokensFromString(config.initial_token).isEmpty());
+ }
+
+ @Test
+ public void testApplyInitialTokensInitialTokensNotSetNumTokensNotSet() throws Exception
+ {
+ Config config = DatabaseDescriptor.loadConfig();
+
+ unregisterSnitchesForTokenConfigTest();
+
+ try
+ {
+ DatabaseDescriptor.applyTokensConfig(config);
+ }
+ finally
+ {
+ unregisterSnitchesForTokenConfigTest();
+ }
+
+ Assert.assertEquals(Integer.valueOf(1), config.num_tokens);
+ Assert.assertTrue(DatabaseDescriptor.tokensFromString(config.initial_token).isEmpty());
+ }
+
+ private void unregisterSnitchesForTokenConfigTest() throws Exception
+ {
+ try
+ {
+ MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+ mbs.unregisterMBean(new ObjectName("org.apache.cassandra.db:type=DynamicEndpointSnitch"));
+ mbs.unregisterMBean(new ObjectName("org.apache.cassandra.db:type=EndpointSnitchInfo"));
+ }
+ catch (InstanceNotFoundException ex)
+ {
+ // ok
+ }
+ }
+}
\ No newline at end of file
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org