Author: jbellis
Date: Mon Apr 27 15:01:42 2009
New Revision: 769018

URL: http://svn.apache.org/viewvc?rev=769018&view=rev
Log:
add test for ReplicaPlacementStrategy covering both Random and OrderPreserving 
partitioners
patch by jbellis; reviewed by Jun Rao for CASSANDRA-65

Added:
    incubator/cassandra/trunk/test/org/
    incubator/cassandra/trunk/test/org/apache/
    incubator/cassandra/trunk/test/org/apache/cassandra/
    incubator/cassandra/trunk/test/org/apache/cassandra/locator/
    
incubator/cassandra/trunk/test/org/apache/cassandra/locator/RackUnawareStrategyTest.java
Modified:
    incubator/cassandra/trunk/src/org/apache/cassandra/dht/StringToken.java
    
incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageService.java

Modified: 
incubator/cassandra/trunk/src/org/apache/cassandra/dht/StringToken.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/dht/StringToken.java?rev=769018&r1=769017&r2=769018&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/dht/StringToken.java 
(original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/dht/StringToken.java Mon 
Apr 27 15:01:42 2009
@@ -2,7 +2,7 @@
 
 public class StringToken extends Token<String>
 {
-    protected StringToken(String token)
+    public StringToken(String token)
     {
         super(token);
     }

Modified: 
incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageService.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageService.java?rev=769018&r1=769017&r2=769018&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageService.java 
(original)
+++ 
incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageService.java 
Mon Apr 27 15:01:42 2009
@@ -867,6 +867,9 @@
     
     public String getToken(EndPoint ep)
     {
+        // render a String representation of the Token corresponding to this 
endpoint
+        // for a human-facing UI.  If there is no such Token then we use "" 
since
+        // it is not a valid value either for BigIntegerToken or StringToken.
         EndPoint ep2 = new EndPoint(ep.getHost(), 
DatabaseDescriptor.getStoragePort());
         Token token = tokenMetadata_.getToken(ep2);
         // if there is no token for an endpoint, return an empty string to 
denote that

Added: 
incubator/cassandra/trunk/test/org/apache/cassandra/locator/RackUnawareStrategyTest.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/org/apache/cassandra/locator/RackUnawareStrategyTest.java?rev=769018&view=auto
==============================================================================
--- 
incubator/cassandra/trunk/test/org/apache/cassandra/locator/RackUnawareStrategyTest.java
 (added)
+++ 
incubator/cassandra/trunk/test/org/apache/cassandra/locator/RackUnawareStrategyTest.java
 Mon Apr 27 15:01:42 2009
@@ -0,0 +1,72 @@
+package org.apache.cassandra.locator;
+
+import java.util.List;
+import java.util.ArrayList;
+import java.math.BigInteger;
+
+import org.testng.annotations.Test;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.dht.BigIntegerToken;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.dht.OrderPreservingPartitioner;
+import org.apache.cassandra.dht.StringToken;
+import org.apache.cassandra.net.EndPoint;
+
+public class RackUnawareStrategyTest
+{
+    @Test
+    public void testBigIntegerStorageEndPoints()
+    {
+        TokenMetadata tmd = new TokenMetadata();
+        IPartitioner partitioner = new RandomPartitioner();
+        IReplicaPlacementStrategy strategy = new RackUnawareStrategy(tmd, 
partitioner, 3, 7000);
+
+        List<Token> endPointTokens = new ArrayList<Token>();
+        List<Token> keyTokens = new ArrayList<Token>();
+        for (int i = 0; i < 5; i++) {
+            endPointTokens.add(new BigIntegerToken(String.valueOf(10 * i)));
+            keyTokens.add(new BigIntegerToken(String.valueOf(10 * i + 5)));
+        }
+        testGetStorageEndPoints(tmd, strategy, endPointTokens.toArray(new 
Token[0]), keyTokens.toArray(new Token[0]));
+    }
+
+    @Test
+    public void testStringStorageEndPoints()
+    {
+        TokenMetadata tmd = new TokenMetadata();
+        IPartitioner partitioner = new OrderPreservingPartitioner();
+        IReplicaPlacementStrategy strategy = new RackUnawareStrategy(tmd, 
partitioner, 3, 7000);
+
+        List<Token> endPointTokens = new ArrayList<Token>();
+        List<Token> keyTokens = new ArrayList<Token>();
+        for (int i = 0; i < 5; i++) {
+            endPointTokens.add(new StringToken(String.valueOf((char)('a' + i * 
2))));
+            keyTokens.add(partitioner.getTokenForKey(String.valueOf((char)('a' 
+ i * 2 + 1))));
+        }
+        testGetStorageEndPoints(tmd, strategy, endPointTokens.toArray(new 
Token[0]), keyTokens.toArray(new Token[0]));
+    }
+
+    // given a list of endpoint tokens, and a set of key tokens falling 
between the endpoint tokens,
+    // make sure that the Strategy picks the right endpoints for the keys.
+    private void testGetStorageEndPoints(TokenMetadata tmd, 
IReplicaPlacementStrategy strategy, Token[] endPointTokens, Token[] keyTokens)
+    {
+        List<EndPoint> hosts = new ArrayList<EndPoint>();
+        for (int i = 0; i < endPointTokens.length; i++)
+        {
+            EndPoint ep = new EndPoint(String.valueOf(i), 7001);
+            tmd.update(endPointTokens[i], ep);
+            hosts.add(ep);
+        }
+
+        for (int i = 0; i < keyTokens.length; i++)
+        {
+            EndPoint[] endPoints = strategy.getStorageEndPoints(keyTokens[i]);
+            assert endPoints.length == 3;
+            for (int j = 0; j < endPoints.length; j++)
+            {
+                assert endPoints[j] == hosts.get((i + j + 1) % hosts.size());
+            }
+        }
+    }
+}


Reply via email to