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 2009/05/15 17:55:36 UTC

svn commit: r775203 - in /incubator/cassandra/trunk: conf/ src/java/org/apache/cassandra/config/ src/java/org/apache/cassandra/dht/

Author: jbellis
Date: Fri May 15 15:55:35 2009
New Revision: 775203

URL: http://svn.apache.org/viewvc?rev=775203&view=rev
Log:
allow specifying initial token.  patch by Jun Rao; reviewed by jbellis for CASSANDRA-181

Modified:
    incubator/cassandra/trunk/conf/storage-conf.xml
    incubator/cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java

Modified: incubator/cassandra/trunk/conf/storage-conf.xml
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/conf/storage-conf.xml?rev=775203&r1=775202&r2=775203&view=diff
==============================================================================
--- incubator/cassandra/trunk/conf/storage-conf.xml (original)
+++ incubator/cassandra/trunk/conf/storage-conf.xml Fri May 15 15:55:35 2009
@@ -32,6 +32,18 @@
     -->
     <Partitioner>org.apache.cassandra.dht.RandomPartitioner</Partitioner>
 
+    <!-- If you are using the OrderPreservingPartitioner and you know your key
+         distribution, you can specify the token for this node to use.
+         (Keys are sent to the node with the "closest" token, so distributing
+         your tokens equally along the key distribution space will spread
+         keys evenly across your cluster.)  This setting is only checked the
+         first time a node is started. 
+
+         This can also be useful with RandomPartitioner to force equal
+         spacing of tokens around the hash space, especially for
+         clusters with a small number of nodes. -->
+    <InitialToken></InitialToken>
+
     <!-- RackAware: Setting this to true instructs Cassandra to try and place the replicas in
          a different rack in the same datacenter and one in a different datacenter
     -->

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java?rev=775203&r1=775202&r2=775203&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java Fri May 15 15:55:35 2009
@@ -119,12 +119,15 @@
 
     // the path qualified config file (storage-conf.xml) name
     private static String configFileName_;
+    /* initial token in the ring */
+    private static String initialToken_ = null;
     
     static
     {
         try
         {
             configFileName_ = System.getProperty("storage-config") + System.getProperty("file.separator") + "storage-conf.xml";
+            logger_.debug("Loading settings from " + configFileName_);
             XMLUtils xmlUtils = new XMLUtils(configFileName_);
 
             /* Cluster Name */
@@ -164,6 +167,8 @@
             if ( gcGrace != null )
                 gcGraceInSeconds_ = Integer.parseInt(gcGrace);
 
+            initialToken_ = xmlUtils.getNodeValue("/Storage/InitialToken");
+
             /* Zookeeper's session timeout */
             String zkSessionTimeout = xmlUtils.getNodeValue("/Storage/ZookeeperSessionTimeout");
             if ( zkSessionTimeout != null )
@@ -536,6 +541,11 @@
       return memtableLifetime_;
     }
 
+    public static String getInitialToken()
+    {
+      return initialToken_;
+    }
+
     public static int getMemtableSize()
     {
       return memtableSize_;

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java?rev=775203&r1=775202&r2=775203&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java Fri May 15 15:55:35 2009
@@ -24,6 +24,8 @@
 import java.util.Locale;
 import java.util.Random;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
+
 public class OrderPreservingPartitioner implements IPartitioner
 {
     // TODO make locale configurable.  But don't just leave it up to the OS or you could really screw
@@ -65,6 +67,11 @@
 
     public StringToken getDefaultToken()
     {
+        String initialToken = DatabaseDescriptor.getInitialToken();
+        if (initialToken != null)
+            return new StringToken(initialToken);
+
+        // generate random token
         String chars = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789";
         Random r = new Random();
         StringBuffer buffer = new StringBuffer();

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java?rev=775203&r1=775202&r2=775203&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java Fri May 15 15:55:35 2009
@@ -21,6 +21,7 @@
 import java.math.BigInteger;
 import java.util.Comparator;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.GuidGenerator;
 
@@ -74,6 +75,11 @@
 
     public BigIntegerToken getDefaultToken()
     {
+        String initialToken = DatabaseDescriptor.getInitialToken();
+        if (initialToken != null)
+            return new BigIntegerToken(new BigInteger(initialToken));
+
+        // generate random token
         String guid = GuidGenerator.guid();
         BigInteger token = FBUtilities.hash(guid);
         if ( token.signum() == -1 )