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/08/27 00:20:15 UTC

svn commit: r808205 - in /incubator/cassandra/trunk: conf/ src/java/org/apache/cassandra/db/marshal/ src/java/org/apache/cassandra/dht/ src/java/org/apache/cassandra/service/ src/java/org/apache/cassandra/utils/ test/conf/ test/unit/org/apache/cassandr...

Author: jbellis
Date: Wed Aug 26 22:20:13 2009
New Revision: 808205

URL: http://svn.apache.org/viewvc?rev=808205&view=rev
Log:
refactors COPP to use BytesToken.  patch by Stu Hood; reviewed by jbellis for CASSANDRA-242

Added:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BytesToken.java
      - copied, changed from r808161, incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BigIntegerToken.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/marshal/BytesTypeTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/CollatingOrderPreservingPartitionerTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/OrderPreservingPartitionerTest.java
      - copied, changed from r808161, incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BigIntegerToken.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/PartitionerTestCase.java
Modified:
    incubator/cassandra/trunk/conf/storage-conf.xml
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/marshal/BytesType.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BigIntegerToken.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/dht/StringToken.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/Token.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java
    incubator/cassandra/trunk/test/conf/storage-conf.xml
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/BootstrapTest.java

Modified: incubator/cassandra/trunk/conf/storage-conf.xml
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/conf/storage-conf.xml?rev=808205&r1=808204&r2=808205&view=diff
==============================================================================
--- incubator/cassandra/trunk/conf/storage-conf.xml (original)
+++ incubator/cassandra/trunk/conf/storage-conf.xml Wed Aug 26 22:20:13 2009
@@ -100,8 +100,8 @@
    ~ org.apache.cassandra.dht.CollatingOrderPreservingPartitioner.
    ~ (CollatingOPP colates according to EN,US rules, not naive byte
    ~ ordering.  Use this as an example if you need locale-aware collation.)
-   ~ Range queries require using OrderPreservingPartitioner or a subclass.
-
+   ~ Range queries require using an order-preserving partitioner.
+   ~
    ~ Achtung!  Changing this parameter requires wiping your data
    ~ directories, since the partitioner can modify the sstable on-disk
    ~ format.
@@ -109,7 +109,7 @@
   <Partitioner>org.apache.cassandra.dht.RandomPartitioner</Partitioner>
 
   <!--
-   ~ If you are using the OrderPreservingPartitioner and you know your key
+   ~ If you are using an order-preserving partitioner 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

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/marshal/BytesType.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/marshal/BytesType.java?rev=808205&r1=808204&r2=808205&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/marshal/BytesType.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/marshal/BytesType.java Wed Aug 26 22:20:13 2009
@@ -23,30 +23,13 @@
 
 import java.util.Arrays;
 
+import org.apache.cassandra.utils.FBUtilities;
+
 public class BytesType extends AbstractType
 {
     public int compare(byte[] o1, byte[] o2)
     {
-        int length = Math.max(o1.length, o2.length);
-        for (int i = 0; i < length; i++)
-        {
-            int index = i + 1;
-            if (index > o1.length && index <= o2.length)
-            {
-                return -1;
-            }
-            if (index > o2.length && index <= o1.length)
-            {
-                return 1;
-            }
-
-            int delta = o1[i] - o2[i];
-            if (delta != 0)
-            {
-                return delta;
-            }
-        }
-        return 0;
+        return FBUtilities.compareByteArrays(o1, o2);
     }
 
     public String getString(byte[] bytes)

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BigIntegerToken.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BigIntegerToken.java?rev=808205&r1=808204&r2=808205&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BigIntegerToken.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BigIntegerToken.java Wed Aug 26 22:20:13 2009
@@ -31,4 +31,10 @@
     public BigIntegerToken(String token) {
         this(new BigInteger(token));
     }
+
+    @Override
+    public int compareTo(Token<BigInteger> o)
+    {
+        return token.compareTo(o.token);
+    }
 }

Copied: incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BytesToken.java (from r808161, incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BigIntegerToken.java)
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BytesToken.java?p2=incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BytesToken.java&p1=incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BigIntegerToken.java&r1=808161&r2=808205&rev=808205&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BigIntegerToken.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BytesToken.java Wed Aug 26 22:20:13 2009
@@ -20,15 +20,24 @@
 
 import java.math.BigInteger;
 
-public class BigIntegerToken extends Token<BigInteger>
+import org.apache.cassandra.utils.FBUtilities;
+
+public class BytesToken extends Token<byte[]>
 {
-    public BigIntegerToken(BigInteger token)
+    public BytesToken(byte... token)
     {
         super(token);
     }
+    
+    @Override
+    public String toString()
+    {
+        return "Token(bytes[" + FBUtilities.bytesToHex(token) + "])";
+    }
 
-    // convenience method for testing
-    public BigIntegerToken(String token) {
-        this(new BigInteger(token));
+    @Override
+    public int compareTo(Token<byte[]> o)
+    {
+        return FBUtilities.compareByteArrays(token, o.token);
     }
 }

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=808205&r1=808204&r2=808205&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 Wed Aug 26 22:20:13 2009
@@ -19,17 +19,25 @@
 package org.apache.cassandra.dht;
 
 import java.io.UnsupportedEncodingException;
+import java.math.BigInteger;
 import java.text.Collator;
+import java.util.Arrays;
 import java.util.Comparator;
 import java.util.Locale;
 import java.util.Random;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.utils.FBUtilities;
 
-public class CollatingOrderPreservingPartitioner extends OrderPreservingPartitioner
+public class CollatingOrderPreservingPartitioner implements IPartitioner<BytesToken>
 {
     static final Collator collator = Collator.getInstance(new Locale("en", "US"));
 
+    public static final BytesToken MINIMUM = new BytesToken(new byte[0]);
+
+    /**
+     * Comparators for decorated keys.
+     */
     private static final Comparator<String> comparator = new Comparator<String>() {
         public int compare(String o1, String o2)
         {
@@ -43,6 +51,16 @@
         }
     };
 
+    public String decorateKey(String key)
+    {
+        return key;
+    }
+
+    public String undecorateKey(String decoratedKey)
+    {
+        return decoratedKey;
+    }
+
     public Comparator<String> getDecoratedKeyComparator()
     {
         return comparator;
@@ -52,4 +70,60 @@
     {
         return reverseComparator;
     }
+
+    public BytesToken getMinimumToken()
+    {
+        return MINIMUM;
+    }
+
+    public BytesToken getDefaultToken()
+    {
+        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);
+        return new BytesToken(buffer);
+    }
+
+    private final Token.TokenFactory<byte[]> tokenFactory = new Token.TokenFactory<byte[]>() {
+        public byte[] toByteArray(Token<byte[]> bytesToken)
+        {
+            return bytesToken.token;
+        }
+
+        public Token<byte[]> fromByteArray(byte[] bytes)
+        {
+            return new BytesToken(bytes);
+        }
+
+        public String toString(Token<byte[]> bytesToken)
+        {
+            return FBUtilities.bytesToHex(bytesToken.token);
+        }
+
+        public Token<byte[]> fromString(String string)
+        {
+            return new BytesToken(FBUtilities.hexToBytes(string));
+        }
+    };
+
+    public Token.TokenFactory<byte[]> getTokenFactory()
+    {
+        return tokenFactory;
+    }
+
+    public boolean preservesOrder()
+    {
+        return true;
+    }
+
+    public BytesToken getToken(String key)
+    {
+        return new BytesToken(collator.getCollationKey(key).toByteArray());
+    }
 }

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=808205&r1=808204&r2=808205&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 Wed Aug 26 22:20:13 2009
@@ -20,11 +20,11 @@
 
 import java.util.Comparator;
 
-public interface IPartitioner
+public interface IPartitioner<T extends Token>
 {
     /**
-     * transform key to on-disk format s.t. keys are stored in node comparison order.
-     * this lets bootstrap rip out parts of the sstable sequentially instead of doing random seeks.
+     * Transform key to on-disk format s.t. keys are stored in node comparison order.
+     * This lets bootstrap rip out parts of the sstable sequentially instead of doing random seeks.
      *
      * @param key the raw, client-facing key
      * @return decorated on-disk version of key
@@ -37,17 +37,28 @@
 
     public Comparator<String> getReverseDecoratedKeyComparator();
 
+	/**
+	 * @return The minimum possible Token in the range that is being partitioned.
+	 */
+	public T getMinimumToken();
+
     /**
      * @return a Token that can be used to route a given key
      */
-    public Token getToken(String key);
+    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.
      */
-    public Token getDefaultToken();
+    public T getDefaultToken();
 
     public Token.TokenFactory getTokenFactory();
+    
+    /**
+     * @return True if the implementing class preserves key order in the Tokens
+     * it generates.
+     */
+    public boolean preservesOrder();
 }

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=808205&r1=808204&r2=808205&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 Wed Aug 26 22:20:13 2009
@@ -19,13 +19,19 @@
 package org.apache.cassandra.dht;
 
 import java.io.UnsupportedEncodingException;
+import java.util.Arrays;
 import java.util.Comparator;
 import java.util.Random;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 
-public class OrderPreservingPartitioner implements IPartitioner
+public class OrderPreservingPartitioner implements IPartitioner<StringToken>
 {
+    public static final StringToken MINIMUM = new StringToken("");
+
+    /**
+     * Comparators for decorated keys.
+     */
     private static final Comparator<String> comparator = new Comparator<String>() {
         public int compare(String o1, String o2)
         {
@@ -59,6 +65,11 @@
         return reverseComparator;
     }
 
+    public StringToken getMinimumToken()
+    {
+        return MINIMUM;
+    }
+
     public StringToken getDefaultToken()
     {
         String initialToken = DatabaseDescriptor.getInitialToken();
@@ -116,8 +127,13 @@
         return tokenFactory;
     }
 
-    public Token getToken(String key)
+    public boolean preservesOrder()
+    {
+        return true;
+    }
+
+    public StringToken getToken(String key)
     {
         return new StringToken(key);
     }
-}
\ No newline at end of file
+}

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=808205&r1=808204&r2=808205&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 Wed Aug 26 22:20:13 2009
@@ -29,8 +29,10 @@
 /**
  * This class generates a BigIntegerToken using MD5 hash.
  */
-public class RandomPartitioner implements IPartitioner
+public class RandomPartitioner implements IPartitioner<BigIntegerToken>
 {
+    public static final BigIntegerToken MINIMUM = new BigIntegerToken("0");
+
     private static final Comparator<String> comparator = new Comparator<String>()
     {
         public int compare(String o1, String o2)
@@ -79,6 +81,11 @@
         return rcomparator;
     }
 
+	public BigIntegerToken getMinimumToken()
+    {
+        return MINIMUM;
+    }
+
     public BigIntegerToken getDefaultToken()
     {
         String initialToken = DatabaseDescriptor.getInitialToken();
@@ -120,7 +127,12 @@
         return tokenFactory;
     }
 
-    public Token getToken(String key)
+    public boolean preservesOrder()
+    {
+        return false;
+    }
+
+    public BigIntegerToken getToken(String key)
     {
         return new BigIntegerToken(FBUtilities.hash(key));
     }

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/StringToken.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/StringToken.java?rev=808205&r1=808204&r2=808205&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/StringToken.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/StringToken.java Wed Aug 26 22:20:13 2009
@@ -27,9 +27,9 @@
         super(token);
     }
 
+    @Override
     public int compareTo(Token<String> o)
     {
-        assert StorageService.getPartitioner() instanceof OrderPreservingPartitioner;
-        return StorageService.getPartitioner().getDecoratedKeyComparator().compare(this.token, o.token);
+        return token.compareTo(o.token);
     }
 }

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/Token.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/Token.java?rev=808205&r1=808204&r2=808205&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/Token.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/Token.java Wed Aug 26 22:20:13 2009
@@ -26,7 +26,7 @@
 import org.apache.cassandra.io.ICompactSerializer;
 import org.apache.cassandra.service.StorageService;
 
-public abstract class Token<T extends Comparable> implements Comparable<Token<T>>, Serializable
+public abstract class Token<T> implements Comparable<Token<T>>, Serializable
 {
     private static final TokenSerializer serializer = new TokenSerializer();
     public static TokenSerializer serializer()
@@ -44,10 +44,7 @@
     /**
      * This determines the comparison for node destination purposes.
      */
-    public int compareTo(Token<T> o)
-    {
-        return token.compareTo(o.token);
-    }
+    abstract public int compareTo(Token<T> o);
 
     public String toString()
     {
@@ -67,7 +64,7 @@
         return token.hashCode();
     }
 
-    public static abstract class TokenFactory<T extends Comparable>
+    public static abstract class TokenFactory<T>
     {
         public abstract byte[] toByteArray(Token<T> token);
         public abstract Token<T> fromByteArray(byte[] bytes);

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java?rev=808205&r1=808204&r2=808205&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java Wed Aug 26 22:20:13 2009
@@ -36,7 +36,7 @@
 import org.apache.cassandra.net.EndPoint;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.LogUtil;
-import org.apache.cassandra.dht.OrderPreservingPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Token;
 import org.apache.thrift.TException;
 
@@ -481,7 +481,7 @@
         if (logger.isDebugEnabled())
             logger.debug("get_key_range");
         ThriftValidation.validateCommand(tablename, columnFamily);
-        if (!(StorageService.getPartitioner() instanceof OrderPreservingPartitioner))
+        if (!StorageService.getPartitioner().preservesOrder())
         {
             throw new InvalidRequestException("range queries may only be performed against an order-preserving partitioner");
         }

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java?rev=808205&r1=808204&r2=808205&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java Wed Aug 26 22:20:13 2009
@@ -26,6 +26,7 @@
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Date;
 import java.util.List;
 import java.util.StringTokenizer;
@@ -41,6 +42,8 @@
 {
     private static Logger logger_ = Logger.getLogger(FBUtilities.class);
 
+    private static final char[] HEX_CHARS = "0123456789abcdef".toCharArray();
+    
     private static InetAddress localInetAddress_;
 
     public static String getTimestamp()
@@ -387,4 +390,26 @@
         }
         return bytes;
     }
+    
+    public static byte[] hexToBytes(String str)
+    {
+        assert str.length() % 2 == 0;
+        byte[] bytes = new byte[str.length()/2];
+        for (int i = 0; i < bytes.length; i++)
+        {
+            bytes[i] = (byte)Integer.parseInt(str.substring(i, i+2), 16);
+        }
+        return bytes;
+    }
+
+    public static String bytesToHex(byte[] buf)
+    {
+        char[] chars = new char[2*buf.length];
+        for (int i = 0; i < buf.length; i++)
+        {
+            chars[i*2] = HEX_CHARS[(buf[i] & 0xF0) >>> 4];
+            chars[i*2+1] = HEX_CHARS[buf[i] & 0x0F];
+        }
+        return new String(chars);
+    }
 }

Modified: incubator/cassandra/trunk/test/conf/storage-conf.xml
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/conf/storage-conf.xml?rev=808205&r1=808204&r2=808205&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/conf/storage-conf.xml (original)
+++ incubator/cassandra/trunk/test/conf/storage-conf.xml Wed Aug 26 22:20:13 2009
@@ -22,7 +22,7 @@
    <FlushIndexBufferSizeInMB>0.1</FlushIndexBufferSizeInMB>
    <CommitLogSync>batch</CommitLogSync>
    <CommitLogSyncBatchWindowInMS>1.0</CommitLogSyncBatchWindowInMS>
-   <Partitioner>org.apache.cassandra.dht.CollatingOrderPreservingPartitioner</Partitioner>
+   <Partitioner>org.apache.cassandra.dht.OrderPreservingPartitioner</Partitioner>
    <EndPointSnitch>org.apache.cassandra.locator.EndPointSnitch</EndPointSnitch>
    <ReplicaPlacementStrategy>org.apache.cassandra.locator.RackUnawareStrategy</ReplicaPlacementStrategy>
    <ReplicationFactor>1</ReplicationFactor>

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/BootstrapTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/BootstrapTest.java?rev=808205&r1=808204&r2=808205&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/BootstrapTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/BootstrapTest.java Wed Aug 26 22:20:13 2009
@@ -38,7 +38,6 @@
 public class BootstrapTest
 {
     /**
-     * 
      * Writes out a bunch of keys into an SSTable, then runs anticompaction on a range.
      * Checks to see if anticompaction returns true.
      */

Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/marshal/BytesTypeTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/marshal/BytesTypeTest.java?rev=808205&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/marshal/BytesTypeTest.java (added)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/marshal/BytesTypeTest.java Wed Aug 26 22:20:13 2009
@@ -0,0 +1,21 @@
+package org.apache.cassandra.db.marshal;
+
+import org.apache.commons.lang.ArrayUtils;
+
+import org.junit.Test;
+
+public class BytesTypeTest
+{
+    @Test
+    public void testCompare()
+    {
+        BytesType comparator = new BytesType();
+        assert comparator.compare(ArrayUtils.EMPTY_BYTE_ARRAY, "asdf".getBytes()) < 0;
+        assert comparator.compare("asdf".getBytes(), ArrayUtils.EMPTY_BYTE_ARRAY) > 0;
+        assert comparator.compare(ArrayUtils.EMPTY_BYTE_ARRAY, ArrayUtils.EMPTY_BYTE_ARRAY) == 0;
+        assert comparator.compare("z".getBytes(), "a".getBytes()) > 0;
+        assert comparator.compare("a".getBytes(), "z".getBytes()) < 0;
+        assert comparator.compare("asdf".getBytes(), "asdf".getBytes()) == 0;
+        assert comparator.compare("asdz".getBytes(), "asdf".getBytes()) > 0;
+    }
+}

Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/CollatingOrderPreservingPartitionerTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/CollatingOrderPreservingPartitionerTest.java?rev=808205&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/CollatingOrderPreservingPartitionerTest.java (added)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/CollatingOrderPreservingPartitionerTest.java Wed Aug 26 22:20:13 2009
@@ -0,0 +1,65 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.dht;
+
+import java.math.BigInteger;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.utils.FBUtilities;
+
+public class CollatingOrderPreservingPartitionerTest extends PartitionerTestCase<BytesToken> {
+    @Override
+    public IPartitioner<BytesToken> getPartitioner()
+    {
+        return new CollatingOrderPreservingPartitioner();
+    }
+
+    @Override
+    public BytesToken tok(String string)
+    {
+        // we just need some kind of byte array
+        try
+        {
+            return new BytesToken(string.getBytes("US-ASCII"));
+        }
+        catch(Exception e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public String tos(BytesToken token)
+    {
+        return FBUtilities.bytesToHex(token.token);
+    }
+
+    /**
+     * Test that a non-UTF-8 byte array can still be encoded.
+     */
+    @Test
+    public void testTokenFactoryStringsNonUTF()
+    {
+        Token.TokenFactory factory = this.part.getTokenFactory();
+        BytesToken tok = new BytesToken((byte)0xFF, (byte)0xFF);
+        assert tok.compareTo(factory.fromString(factory.toString(tok))) == 0;
+    }
+}

Copied: incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/OrderPreservingPartitionerTest.java (from r808161, incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BigIntegerToken.java)
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/OrderPreservingPartitionerTest.java?p2=incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/OrderPreservingPartitionerTest.java&p1=incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BigIntegerToken.java&r1=808161&r2=808205&rev=808205&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BigIntegerToken.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/OrderPreservingPartitionerTest.java Wed Aug 26 22:20:13 2009
@@ -20,15 +20,27 @@
 
 import java.math.BigInteger;
 
-public class BigIntegerToken extends Token<BigInteger>
-{
-    public BigIntegerToken(BigInteger token)
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.utils.FBUtilities;
+
+public class OrderPreservingPartitionerTest extends PartitionerTestCase<StringToken> {
+    @Override
+    public IPartitioner<StringToken> getPartitioner()
+    {
+        return new OrderPreservingPartitioner();
+    }
+
+    @Override
+    public StringToken tok(String string)
     {
-        super(token);
+        return new StringToken(string);
     }
 
-    // convenience method for testing
-    public BigIntegerToken(String token) {
-        this(new BigInteger(token));
+    @Override
+    public String tos(StringToken token)
+    {
+        return FBUtilities.bytesToHex(token.token.getBytes());
     }
 }

Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/PartitionerTestCase.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/PartitionerTestCase.java?rev=808205&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/PartitionerTestCase.java (added)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/PartitionerTestCase.java Wed Aug 26 22:20:13 2009
@@ -0,0 +1,66 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.dht;
+
+import java.math.BigInteger;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.utils.FBUtilities;
+
+public abstract class PartitionerTestCase<T extends Token> {
+    protected IPartitioner<T> part;
+
+    public abstract IPartitioner<T> getPartitioner();
+    public abstract T tok(String string);
+    public abstract String tos(T token);
+
+    @Before
+    public void clean()
+    {
+        this.part = this.getPartitioner();
+    }
+
+    @Test
+    public void testCompare()
+    {
+        assert tok("").compareTo(tok("asdf")) < 0;
+        assert tok("asdf").compareTo(tok("")) > 0;
+        assert tok("").compareTo(tok("")) == 0;
+        assert tok("z").compareTo(tok("a")) > 0;
+        assert tok("a").compareTo(tok("z")) < 0;
+        assert tok("asdf").compareTo(tok("asdf")) == 0;
+        assert tok("asdz").compareTo(tok("asdf")) > 0;
+    }
+
+    @Test
+    public void testTokenFactoryBytes()
+    {
+        Token.TokenFactory factory = this.part.getTokenFactory();
+        assert tok("a").compareTo(factory.fromByteArray(factory.toByteArray(tok("a")))) == 0;
+    }
+    
+    @Test
+    public void testTokenFactoryStrings()
+    {
+        Token.TokenFactory factory = this.part.getTokenFactory();
+        assert tok("a").compareTo(factory.fromString(factory.toString(tok("a")))) == 0;
+    }
+}