You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2011/09/19 18:58:20 UTC

svn commit: r1172701 - in /cassandra/branches/cassandra-1.0.0: CHANGES.txt src/java/org/apache/cassandra/dht/BootStrapper.java src/java/org/apache/cassandra/service/StorageService.java

Author: jbellis
Date: Mon Sep 19 16:58:20 2011
New Revision: 1172701

URL: http://svn.apache.org/viewvc?rev=1172701&view=rev
Log:
Base choice of random or "balanced" token onbootstrap on whether schema definitions were found
patch by slebresne; reviewed by jbellis for CASSANDRA-3219

Modified:
    cassandra/branches/cassandra-1.0.0/CHANGES.txt
    cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/dht/BootStrapper.java
    cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/service/StorageService.java

Modified: cassandra/branches/cassandra-1.0.0/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0.0/CHANGES.txt?rev=1172701&r1=1172700&r2=1172701&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0.0/CHANGES.txt (original)
+++ cassandra/branches/cassandra-1.0.0/CHANGES.txt Mon Sep 19 16:58:20 2011
@@ -8,6 +8,8 @@
    dies (CASSANDRA-3216)
  * Remove dynamic_snitch boolean from example configuration (defaulting to 
    true) and set default badness threshold to 0.1 (CASSANDRA-3229)
+ * Base choice of random or "balanced" token on bootstrap on whether
+   schema definitions were found (CASSANDRA-3219)
 
 
 1.0.0-beta1

Modified: cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/dht/BootStrapper.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/dht/BootStrapper.java?rev=1172701&r1=1172700&r2=1172701&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/dht/BootStrapper.java (original)
+++ cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/dht/BootStrapper.java Mon Sep 19 16:58:20 2011
@@ -127,24 +127,6 @@ public class BootStrapper
         }
     }
 
-    /**
-     * if initialtoken was specified, use that.
-     * otherwise, pick a token to assume half the load of the most-loaded node.
-     */
-    public static Token getBootstrapToken(final TokenMetadata metadata, final Map<InetAddress, Double> load) throws IOException, ConfigurationException
-    {
-        if (DatabaseDescriptor.getInitialToken() != null)
-        {
-            logger.debug("token manually specified as " + DatabaseDescriptor.getInitialToken());
-            Token token = StorageService.getPartitioner().getTokenFactory().fromString(DatabaseDescriptor.getInitialToken());
-            if (metadata.getEndpoint(token) != null)
-                throw new ConfigurationException("Bootstraping to existing token " + token + " is not allowed (decommission/removetoken the old node first).");
-            return token;
-        }
-
-        return getBalancedToken(metadata, load);
-    }
-
     public static Token getBalancedToken(TokenMetadata metadata, Map<InetAddress, Double> load)
     {
         InetAddress maxEndpoint = getBootstrapSource(metadata, load);

Modified: cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/service/StorageService.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/service/StorageService.java?rev=1172701&r1=1172700&r2=1172701&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/service/StorageService.java (original)
+++ cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/service/StorageService.java Mon Sep 19 16:58:20 2011
@@ -27,13 +27,11 @@ import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.*;
-import javax.lang.model.type.TypeKind;
 import java.util.concurrent.atomic.AtomicInteger;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
 import com.google.common.collect.*;
-import org.apache.cassandra.config.*;
 import org.apache.log4j.Level;
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
@@ -42,6 +40,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
+import org.apache.cassandra.config.*;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.Table;
 import org.apache.cassandra.db.commitlog.CommitLog;
@@ -518,7 +517,7 @@ public class StorageService implements I
                     throw new UnsupportedOperationException(s);
                 }
                 setMode("Joining: getting bootstrap token", true);
-                token = BootStrapper.getBootstrapToken(tokenMetadata_, LoadBroadcaster.instance.getLoadInfo());
+                token = getNewToken();
             }
             else
             {
@@ -552,23 +551,9 @@ public class StorageService implements I
         {
             token = SystemTable.getSavedToken();
             if (token == null)
-            {
-                String initialToken = DatabaseDescriptor.getInitialToken();
-                if (initialToken == null)
-                {
-                    token = partitioner.getRandomToken();
-                    logger_.warn("Generated random token " + token + ". Random tokens will result in an unbalanced ring; see http://wiki.apache.org/cassandra/Operations");
-                }
-                else
-                {
-                    token = partitioner.getTokenFactory().fromString(initialToken);
-                    logger_.info("Saved token not found. Using " + token + " from configuration");
-                }
-            }
+                token = getNewToken();
             else
-            {
                 logger_.info("Using saved token " + token);
-            }
         }
 
         // start participating in the ring.
@@ -578,6 +563,33 @@ public class StorageService implements I
         assert tokenMetadata_.sortedTokens().size() > 0;
     }
 
+    /**
+     * Return a new token for this node.
+     */
+    private Token getNewToken() throws ConfigurationException
+    {
+        Token token;
+        if (DatabaseDescriptor.getInitialToken() != null)
+        {
+            logger_.debug("token manually specified as {}", DatabaseDescriptor.getInitialToken());
+            token = StorageService.getPartitioner().getTokenFactory().fromString(DatabaseDescriptor.getInitialToken());
+        }
+        else if (Schema.instance.getNonSystemTables().size() > 0)
+        {
+            // We are not bootstrapping, we are an initial node, getBalancedToken is not safe.
+            token = partitioner.getRandomToken();
+            logger_.warn("Generated random token " + token + ". Random tokens will result in an unbalanced ring; see http://wiki.apache.org/cassandra/Operation");
+        }
+        else
+        {
+            token = BootStrapper.getBalancedToken(tokenMetadata_, LoadBroadcaster.instance.getLoadInfo());
+        }
+
+        if (tokenMetadata_.getEndpoint(token) != null)
+            throw new ConfigurationException("Bootstraping to existing token " + token + " is not allowed (decommission/removetoken the old node first)");
+        return token;
+    }
+
     public synchronized void joinRing() throws IOException, org.apache.cassandra.config.ConfigurationException
     {
         if (!joined)