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/11/03 03:27:44 UTC
svn commit: r832272 - in /incubator/cassandra/trunk:
src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/dht/
src/java/org/apache/cassandra/service/ test/unit/org/apache/cassandra/dht/
Author: jbellis
Date: Tue Nov 3 02:27:43 2009
New Revision: 832272
URL: http://svn.apache.org/viewvc?rev=832272&view=rev
Log:
refactor getDefaultToken into default checking + getRandomToken. Decorate OPP keys so we don't have to special case IndexedDKs for getSplits.
patch by jbellis and Vijay Parthasarathy for CASSANDRA-513
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/CollatingOrderPreservingPartitioner.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/IPartitioner.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java?rev=832272&r1=832271&r2=832272&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java Tue Nov 3 02:27:43 2009
@@ -28,11 +28,12 @@
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.utils.BasicUtilities;
-import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.db.filter.IdentityQueryFilter;
import org.apache.cassandra.db.filter.QueryPath;
import org.apache.cassandra.db.filter.QueryFilter;
import org.apache.cassandra.db.filter.NamesQueryFilter;
+import org.apache.cassandra.config.DatabaseDescriptor;
+
import java.net.InetAddress;
public class SystemTable
@@ -121,7 +122,13 @@
IPartitioner p = StorageService.getPartitioner();
if (cf == null)
{
- Token token = p.getDefaultToken();
+ Token token;
+ String initialToken = DatabaseDescriptor.getInitialToken();
+ if (initialToken == null)
+ token = p.getRandomToken();
+ else
+ token = p.getToken(initialToken);
+
logger.info("Saved Token not found. Using " + token);
// seconds-since-epoch isn't a foolproof new generation
// (where foolproof is "guaranteed to be larger than the last one seen at this ip address"),
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/CollatingOrderPreservingPartitioner.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/CollatingOrderPreservingPartitioner.java?rev=832272&r1=832271&r2=832272&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/CollatingOrderPreservingPartitioner.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/CollatingOrderPreservingPartitioner.java Tue Nov 3 02:27:43 2009
@@ -147,14 +147,8 @@
return MINIMUM;
}
- public BytesToken getDefaultToken()
+ public BytesToken getRandomToken()
{
- String initialToken = DatabaseDescriptor.getInitialToken();
- if (initialToken != null)
- // assume that the user specified the intial Token as a String key
- return getToken(initialToken);
-
- // generate random token
Random r = new Random();
byte[] buffer = new byte[16];
r.nextBytes(buffer);
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/IPartitioner.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/IPartitioner.java?rev=832272&r1=832271&r2=832272&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/IPartitioner.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/IPartitioner.java Tue Nov 3 02:27:43 2009
@@ -76,11 +76,9 @@
public T getToken(String key);
/**
- * @return the default Token to represent this node if none was saved.
- * Uses the one given in the InitialToken configuration directive,
- * or picks one automatically if that was not given.
+ * @return a randomly generated token
*/
- public T getDefaultToken();
+ public T getRandomToken();
public Token.TokenFactory getTokenFactory();
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=832272&r1=832271&r2=832272&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 Tue Nov 3 02:27:43 2009
@@ -43,12 +43,12 @@
public DecoratedKey<StringToken> decorateKey(String key)
{
- return new DecoratedKey<StringToken>(null, key);
+ return new DecoratedKey<StringToken>(new StringToken(key), key);
}
public DecoratedKey<StringToken> convertFromDiskFormat(String key)
{
- return new DecoratedKey<StringToken>(null, key);
+ return new DecoratedKey<StringToken>(new StringToken(key), key);
}
public String convertToDiskFormat(DecoratedKey<StringToken> key)
@@ -164,13 +164,8 @@
return MINIMUM;
}
- public StringToken getDefaultToken()
+ public StringToken getRandomToken()
{
- String initialToken = DatabaseDescriptor.getInitialToken();
- if (initialToken != null)
- return new StringToken(initialToken);
-
- // generate random token
String chars = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789";
Random r = new Random();
StringBuilder buffer = new StringBuilder();
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=832272&r1=832271&r2=832272&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 Tue Nov 3 02:27:43 2009
@@ -104,13 +104,8 @@
return MINIMUM;
}
- public BigIntegerToken getDefaultToken()
+ public BigIntegerToken getRandomToken()
{
- 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 )
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java?rev=832272&r1=832271&r2=832272&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java Tue Nov 3 02:27:43 2009
@@ -844,10 +844,22 @@
tokens.add(range.left().toString());
List<DecoratedKey> decoratedKeys = SSTableReader.getIndexedDecoratedKeys();
- for (int i = 1; i < splits; i++)
+ if (decoratedKeys.size() < splits)
{
- int index = i * (decoratedKeys.size() / splits);
- tokens.add(decoratedKeys.get(index).token.toString());
+ // not enough keys to generate good splits -- generate random ones instead
+ // (since this only happens when we don't have many keys, it doesn't really matter that the splits are poor)
+ for (int i = 1; i < splits; i++)
+ {
+ tokens.add(partitioner_.getRandomToken().toString());
+ }
+ }
+ else
+ {
+ for (int i = 1; i < splits; i++)
+ {
+ int index = i * (decoratedKeys.size() / splits);
+ tokens.add(decoratedKeys.get(index).token.toString());
+ }
}
tokens.add(range.right().toString());
Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java?rev=832272&r1=832271&r2=832272&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java Tue Nov 3 02:27:43 2009
@@ -50,7 +50,7 @@
generateFakeEndpoints(numOldNodes);
- Token myToken = StorageService.getPartitioner().getDefaultToken();
+ Token myToken = StorageService.getPartitioner().getRandomToken();
InetAddress myEndpoint = InetAddress.getByName("127.0.0.1");
TokenMetadata tmd = ss.getTokenMetadata();
@@ -94,7 +94,7 @@
for (int i = 1; i <= numOldNodes; i++)
{
// leave .1 for myEndpoint
- tmd.update(p.getDefaultToken(), InetAddress.getByName("127.0.0." + (i + 1)));
+ tmd.update(p.getRandomToken(), InetAddress.getByName("127.0.0." + (i + 1)));
}
}
}
\ No newline at end of file