cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From slebre...@apache.org
Subject [2/3] cassandra git commit: Validate that num_tokens and initial_token are consistent with one another
Date Tue, 05 Apr 2016 14:32:26 GMT
Validate that num_tokens and initial_token are consistent with one another

patch by pogribnyi; reviewed by slebresne for CASSANDRA-10120


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ac6ca9c5
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ac6ca9c5
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ac6ca9c5

Branch: refs/heads/trunk
Commit: ac6ca9c501975ee9fc3d8982d651c9109a51644b
Parents: 45f1db0
Author: Sylvain Lebresne <sylvain@datastax.com>
Authored: Tue Apr 5 16:28:05 2016 +0200
Committer: Sylvain Lebresne <sylvain@datastax.com>
Committed: Tue Apr 5 16:28:05 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                        |  2 +-
 .../cassandra/config/DatabaseDescriptor.java       | 17 ++++++++++++-----
 2 files changed, 13 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ac6ca9c5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d03705a..4baaa30 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,8 +1,8 @@
 3.0.6
+ * Validate that num_tokens and initial_token are consistent with one another (CASSANDRA-10120)
 Merged from 2.2:
  * IncomingStreamingConnection version check message wrong (CASSANDRA-11462)
 
-
 3.0.5
  * Fix rare NPE on schema upgrade from 2.x to 3.x (CASSANDRA-10943)
  * Improve backoff policy for cqlsh COPY FROM (CASSANDRA-11320)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ac6ca9c5/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index c505004..67f7883 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -610,15 +610,22 @@ 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.initial_token != null)
-            for (String token : tokensFromString(conf.initial_token))
-                partitioner.getTokenFactory().validate(token);
-
         if (conf.num_tokens == null)
-        	conf.num_tokens = 1;
+            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);
+        }
+
+
         try
         {
             // if key_cache_size_in_mb option was set to "auto" then size of the cache should
be "min(5% of Heap (in MB), 100MB)


Mime
View raw message