Repository: cassandra Updated Branches: refs/heads/trunk 75f78729d -> 428806165
Allow simultaneous bootstrapping with strict consistency when no vnodes are used patch by Sankalp Kohli; reviewed by tjake for CASSANDRA-11005 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/abe0c677 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/abe0c677 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/abe0c677 Branch: refs/heads/trunk Commit: abe0c6779c287c4f55a48a08c83f9cda7d6616f5 Parents: 03e6f45 Author: T Jake Luciani <j...@apache.org> Authored: Fri Jan 15 09:31:22 2016 -0500 Committer: T Jake Luciani <j...@apache.org> Committed: Fri Jan 15 09:31:22 2016 -0500 ---------------------------------------------------------------------- CHANGES.txt | 1 + src/java/org/apache/cassandra/dht/RangeStreamer.java | 1 + src/java/org/apache/cassandra/service/StorageService.java | 7 ++++++- 3 files changed, 8 insertions(+), 1 deletion(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/abe0c677/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index 636c15d..4b87ed0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.1.13 + * Allow simultaneous bootstrapping with strict consistency when no vnodes are used (CASSANDRA-11005) * Log a message when major compaction does not result in a single file (CASSANDRA-10847) * (cqlsh) fix cqlsh_copy_tests when vnodes are disabled (CASSANDRA-10997) * (cqlsh) fix formatting bytearray values (CASSANDRA-10839) http://git-wip-us.apache.org/repos/asf/cassandra/blob/abe0c677/src/java/org/apache/cassandra/dht/RangeStreamer.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/dht/RangeStreamer.java b/src/java/org/apache/cassandra/dht/RangeStreamer.java index 3b095b1..f8e29b6 100644 --- a/src/java/org/apache/cassandra/dht/RangeStreamer.java +++ b/src/java/org/apache/cassandra/dht/RangeStreamer.java @@ -49,6 +49,7 @@ public class RangeStreamer { private static final Logger logger = LoggerFactory.getLogger(RangeStreamer.class); public static final boolean useStrictConsistency = Boolean.valueOf(System.getProperty("cassandra.consistent.rangemovement","true")); + public static final boolean allowSimultaneousMoves = Boolean.valueOf(System.getProperty("cassandra.consistent.simultaneousmoves.allow","false")); private final Collection<Token> tokens; private final TokenMetadata metadata; private final InetAddress address; http://git-wip-us.apache.org/repos/asf/cassandra/blob/abe0c677/src/java/org/apache/cassandra/service/StorageService.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index f134e8a..606670c 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -545,7 +545,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE "Use cassandra.replace_address if you want to replace this node.", FBUtilities.getBroadcastAddress())); } - if (RangeStreamer.useStrictConsistency) + if (RangeStreamer.useStrictConsistency && !allowSimultaneousMoves()) { for (Map.Entry<InetAddress, EndpointState> entry : Gossiper.instance.getEndpointStates()) { @@ -562,6 +562,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE Gossiper.instance.resetEndpointStateMap(); } + private boolean allowSimultaneousMoves() + { + return RangeStreamer.allowSimultaneousMoves && DatabaseDescriptor.getNumTokens() == 1; + } + public synchronized void initClient() throws ConfigurationException { // We don't wait, because we're going to actually try to work on