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/06 23:17:09 UTC

svn commit: r833576 - in /incubator/cassandra/trunk/src/java/org/apache/cassandra: locator/RackUnawareStrategy.java locator/TokenMetadata.java service/StorageProxy.java

Author: jbellis
Date: Fri Nov  6 22:17:09 2009
New Revision: 833576

URL: http://svn.apache.org/viewvc?rev=833576&view=rev
Log:
fix TODO in tokenmetadata (bootstrap tokens aren't part of the official ring now).  r/m redundant getNextEndPoint method in favor of getSuccessor
patch by jbellis; reviewed by Jaakko Laine for CASSANDRA-525

Modified:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/RackUnawareStrategy.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/TokenMetadata.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/RackUnawareStrategy.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/RackUnawareStrategy.java?rev=833576&r1=833575&r2=833576&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/RackUnawareStrategy.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/RackUnawareStrategy.java Fri Nov  6 22:17:09 2009
@@ -40,17 +40,14 @@
         super(tokenMetadata, partitioner, replicas, storagePort);
     }
 
-    // TODO this doesn't need to worry about bootstrap tokens being in the normal tokens map
     public ArrayList<InetAddress> getNaturalEndpoints(Token token, TokenMetadata metadata)
     {
         int startIndex;
         List<Token> tokenList = new ArrayList<Token>();
-        int foundCount = 0;
         List tokens = new ArrayList<Token>(metadata.sortedTokens());
-        List<Token> bsTokens = null;
 
         int index = Collections.binarySearch(tokens, token);
-        if(index < 0)
+        if (index < 0)
         {
             index = (index + 1) * (-1);
             if (index >= tokens.size())
@@ -58,21 +55,14 @@
         }
         int totalNodes = tokens.size();
         // Add the token at the index by default
-        tokenList.add((Token)tokens.get(index));
-        if (bsTokens == null || !bsTokens.contains(tokens.get(index)))
-            foundCount++;
-        startIndex = (index + 1)%totalNodes;
+        tokenList.add((Token) tokens.get(index));
+        startIndex = (index + 1) % totalNodes;
         // If we found N number of nodes we are good. This loop will just exit. Otherwise just
         // loop through the list and add until we have N nodes.
-        for (int i = startIndex, count = 1; count < totalNodes && foundCount < replicas_; ++count, i = (i+1)%totalNodes)
+        for (int i = startIndex, count = 1; count < totalNodes && tokenList.size() < replicas_; ++count, i = (i + 1) % totalNodes)
         {
-            if(!tokenList.contains(tokens.get(i)))
-            {
-                tokenList.add((Token)tokens.get(i));
-                //Don't count bootstrapping tokens towards the count
-                if (bsTokens==null || !bsTokens.contains(tokens.get(i)))
-                    foundCount++;
-            }
+            assert !tokenList.contains(tokens.get(i));
+            tokenList.add((Token) tokens.get(i));
         }
         ArrayList<InetAddress> endpoints = new ArrayList<InetAddress>(tokenList.size());
         for (Token t : tokenList)

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/TokenMetadata.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/TokenMetadata.java?rev=833576&r1=833575&r2=833576&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/TokenMetadata.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/TokenMetadata.java Fri Nov  6 22:17:09 2009
@@ -176,36 +176,6 @@
             lock.readLock().unlock();
         }
     }
-    
-
-    public InetAddress getNextEndpoint(InetAddress endpoint) throws UnavailableException
-    {
-        assert endpoint != null;
-
-        lock.readLock().lock();
-        try
-        {
-            ArrayList<Token> tokens = new ArrayList<Token>(tokenToEndPointMap.keySet());
-            if (tokens.isEmpty())
-                return null;
-            Collections.sort(tokens);
-            int i = tokens.indexOf(tokenToEndPointMap.inverse().get(endpoint)); // TODO binary search
-            int j = 1;
-            InetAddress ep;
-            while (!FailureDetector.instance().isAlive((ep = tokenToEndPointMap.get(tokens.get((i + j) % tokens.size())))))
-            {
-                if (++j > DatabaseDescriptor.getReplicationFactor())
-                {
-                    throw new UnavailableException();
-                }
-            }
-            return ep;
-        }
-        finally
-        {
-            lock.readLock().unlock();
-        }
-    }
 
     public TokenMetadata cloneMe()
     {
@@ -300,6 +270,11 @@
         return (Token) ((index == (tokens.size() - 1)) ? tokens.get(0) : tokens.get(index + 1));
     }
 
+    public InetAddress getSuccessor(InetAddress endPoint)
+    {
+        return getEndPoint(getSuccessor(getToken(endPoint)));
+    }
+
     public Iterable<? extends Token> bootstrapTokens()
     {
         return bootstrapTokenMap.keySet();

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java?rev=833576&r1=833575&r2=833576&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java Fri Nov  6 22:17:09 2009
@@ -36,6 +36,7 @@
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.gms.FailureDetector;
 
 import org.apache.log4j.Logger;
 
@@ -567,7 +568,10 @@
             // but that won't work when you have a replication factor of more than one--any node, not just
             // the one holding the keys where the range wraps, could include both the smallest keys, and the largest,
             // so starting with the largest in our scan of the next node means we'd never see keys from the middle.
-            endPoint = tokenMetadata.getNextEndpoint(endPoint); // TODO move this into the Strategies & modify for RackAwareStrategy
+            do
+            {
+                endPoint = tokenMetadata.getSuccessor(endPoint); // TODO move this into the Strategies & modify for RackAwareStrategy
+            } while (!FailureDetector.instance().isAlive(endPoint));
             int maxResults = endPoint == wrapEndpoint ? rawCommand.maxResults : rawCommand.maxResults - allKeys.size();
             command = new RangeCommand(command.table, command.columnFamily, command.startWith, command.stopAt, maxResults);
         } while (!endPoint.equals(startEndpoint));