Author: jbellis
Date: Wed May 19 14:03:53 2010
New Revision: 946192
URL: http://svn.apache.org/viewvc?rev=946192&view=rev
Log:
add back localEndpoints variable to track "endpoints in this DC" separately
from "total endpoints selected so far." fixes regression introduced in
CASSANDRA-1103. patch by jbellis
Modified:
cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStrategy.java
Modified:
cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStrategy.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStrategy.java?rev=946192&r1=946191&r2=946192&view=diff
==============================================================================
---
cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStrategy.java
(original)
+++
cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStrategy.java
Wed May 19 14:03:53 2010
@@ -145,25 +145,32 @@ public class DatacenterShardStrategy ext
// find replicas on unique racks
int replicas = getReplicationFactor(dc, table);
- while (endpoints.size() < replicas && iter.hasNext())
+ int localEndpoints = 1;
+ while (localEndpoints < replicas && iter.hasNext())
{
Token t = iter.next();
InetAddress endpoint = metadata.getEndpoint(t);
if (!racks.contains(snitch.getRack(endpoint)))
+ {
endpoints.add(endpoint);
+ localEndpoints++;
+ }
}
- if (endpoints.size() == replicas)
+ if (localEndpoints == replicas)
continue;
// if not enough unique racks were found, re-loop and add other
endpoints
iter = TokenMetadata.ringIterator(tokens, searchToken);
iter.next(); // skip the first one since we already know it's used
- while (endpoints.size() < replicas && iter.hasNext())
+ while (localEndpoints < replicas && iter.hasNext())
{
Token t = iter.next();
if (!endpoints.contains(metadata.getEndpoint(t)))
+ {
+ localEndpoints++;
endpoints.add(metadata.getEndpoint(t));
+ }
}
}