Use saved tokens when setting local tokens on StorageService.joinRing() Patch by Paulo Motta; reviewed by Carl Yeksigian for CASSANDRA-12935
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/a449e8f7 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/a449e8f7 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/a449e8f7 Branch: refs/heads/cassandra-3.0 Commit: a449e8f70f047081b2fd5892219ad2659d0027bd Parents: d38bf9f Author: Paulo Motta <pauloricard...@gmail.com> Authored: Sun Nov 20 18:47:54 2016 -0200 Committer: Carl Yeksigian <c...@apache.org> Committed: Mon Dec 5 11:40:37 2016 -0500 ---------------------------------------------------------------------- CHANGES.txt | 1 + src/java/org/apache/cassandra/service/StorageService.java | 9 +++++---- 2 files changed, 6 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/a449e8f7/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index d951b07..302becf 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.2.9 + * Use saved tokens when setting local tokens on StorageService.joinRing (CASSANDRA-12935) * cqlsh: fix DESC TYPES errors (CASSANDRA-12914) * Fix leak on skipped SSTables in sstableupgrade (CASSANDRA-12899) * Avoid blocking gossip during pending range calculation (CASSANDRA-12281) http://git-wip-us.apache.org/repos/asf/cassandra/blob/a449e8f7/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 58c2bac..c2996d7 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -194,6 +194,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE /** This method updates the local token on disk */ public void setTokens(Collection<Token> tokens) { + assert tokens != null && !tokens.isEmpty() : "Node needs at least one token."; if (logger.isDebugEnabled()) logger.debug("Setting tokens to {}", tokens); SystemKeyspace.updateTokens(tokens); @@ -907,7 +908,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE { if (dataAvailable) { - finishJoiningRing(); + finishJoiningRing(bootstrapTokens); // remove the existing info about the replaced node. if (!current.isEmpty()) @@ -959,15 +960,15 @@ public class StorageService extends NotificationBroadcasterSupport implements IE { isSurveyMode = false; logger.info("Leaving write survey mode and joining ring at operator request"); - finishJoiningRing(); + finishJoiningRing(SystemKeyspace.getSavedTokens()); } } - private void finishJoiningRing() + private void finishJoiningRing(Collection<Token> tokens) { // start participating in the ring. SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.COMPLETED); - setTokens(bootstrapTokens); + setTokens(tokens); assert tokenMetadata.sortedTokens().size() > 0; doAuthSetup();