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/12/21 18:05:30 UTC

svn commit: r892882 - in /incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator: DatacenterShardStategy.java RackAwareStrategy.java RackUnawareStrategy.java

Author: jbellis
Date: Mon Dec 21 17:05:29 2009
New Revision: 892882

URL: http://svn.apache.org/viewvc?rev=892882&view=rev
Log:
Fix ArrayIndex exception in case the first gossip a new node sees is leaving or left.  patch by Jaakko Laine; reviewed by jbellis for CASSANDRA-572

Modified:
    incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java
    incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/RackAwareStrategy.java
    incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/RackUnawareStrategy.java

Modified: incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java?rev=892882&r1=892881&r2=892882&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java (original)
+++ incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java Mon Dec 21 17:05:29 2009
@@ -133,6 +133,10 @@
     private ArrayList<InetAddress> getNaturalEndpointsInternal(Token searchToken, TokenMetadata metadata) throws IOException
     {
         ArrayList<InetAddress> endpoints = new ArrayList<InetAddress>();
+
+        if (metadata.sortedTokens().size() == 0)
+            return endpoints;
+
         if (null == tokens || tokens.size() != metadata.sortedTokens().size())
         {
             loadEndPoints(metadata);

Modified: incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/RackAwareStrategy.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/RackAwareStrategy.java?rev=892882&r1=892881&r2=892882&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/RackAwareStrategy.java (original)
+++ incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/RackAwareStrategy.java Mon Dec 21 17:05:29 2009
@@ -52,6 +52,10 @@
         boolean bOtherRack = false;
         int foundCount = 0;
         List tokens = metadata.sortedTokens();
+
+        if (tokens.isEmpty())
+            return endpoints;
+
         int index = Collections.binarySearch(tokens, token);
         if(index < 0)
         {

Modified: incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/RackUnawareStrategy.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/RackUnawareStrategy.java?rev=892882&r1=892881&r2=892882&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/RackUnawareStrategy.java (original)
+++ incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/locator/RackUnawareStrategy.java Mon Dec 21 17:05:29 2009
@@ -45,6 +45,10 @@
         int startIndex;
         List<Token> tokenList = new ArrayList<Token>();
         List tokens = new ArrayList<Token>(metadata.sortedTokens());
+        ArrayList<InetAddress> endpoints = new ArrayList<InetAddress>(tokenList.size());
+
+        if (tokens.isEmpty())
+            return endpoints;
 
         int index = Collections.binarySearch(tokens, token);
         if (index < 0)
@@ -64,7 +68,6 @@
             assert !tokenList.contains(tokens.get(i));
             tokenList.add((Token) tokens.get(i));
         }
-        ArrayList<InetAddress> endpoints = new ArrayList<InetAddress>(tokenList.size());
         for (Token t : tokenList)
             endpoints.add(metadata.getEndPoint(t));
         return endpoints;