Author: jbellis
Date: Tue May 18 21:54:43 2010
New Revision: 945914
URL: http://svn.apache.org/viewvc?rev=945914&view=rev
Log:
fix DSS to generate unique racks when possible
patch by jbellis; reviewed by Jeremy Hanna for CASSANDRA-1103
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=945914&r1=945913&r2=945914&view=diff
==============================================================================
---
cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStrategy.java
(original)
+++
cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStrategy.java
Tue May 18 21:54:43 2010
@@ -121,6 +121,8 @@ public class DatacenterShardStrategy ext
Collections.sort(valueList);
dcTokens.put(entry.getKey(), valueList);
}
+
+ // TODO verify that each DC has enough endpoints for the desired RF
}
public ArrayList<InetAddress> getNaturalEndpoints(Token searchToken,
TokenMetadata metadata, String table)
@@ -132,53 +134,36 @@ public class DatacenterShardStrategy ext
for (String dc : dcTokens.keySet())
{
- int replicas = getReplicationFactor(dc, table);
List<Token> tokens = dcTokens.get(dc);
- boolean bOtherRack = false;
+ Set<String> racks = new HashSet<String>();
// Add the node at the index by default
Iterator<Token> iter = TokenMetadata.ringIterator(tokens,
searchToken);
InetAddress initialDCHost = metadata.getEndpoint(iter.next());
assert initialDCHost != null;
endpoints.add(initialDCHost);
+ racks.add(snitch.getRack(initialDCHost));
+ // find replicas on unique racks
+ int replicas = getReplicationFactor(dc, table);
while (endpoints.size() < replicas && iter.hasNext())
{
Token t = iter.next();
- InetAddress endPointOfInterest = metadata.getEndpoint(t);
- if (endpoints.size() < replicas - 1)
- {
- endpoints.add(endPointOfInterest);
- continue;
- }
-
- // Now try to find one on a different rack
- if (!bOtherRack)
- {
- if
(!snitch.getRack(initialDCHost).equals(snitch.getRack(endPointOfInterest)))
- {
- endpoints.add(metadata.getEndpoint(t));
- bOtherRack = true;
- }
- }
- // If both already found exit loop.
- if (bOtherRack)
- break;
+ InetAddress endpoint = metadata.getEndpoint(t);
+ if (!racks.contains(snitch.getRack(endpoint)))
+ endpoints.add(endpoint);
}
- /*
- * 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.
- */
- if (endpoints.size() < replicas)
+ if (endpoints.size() == 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())
{
- iter = TokenMetadata.ringIterator(tokens, searchToken);
- while (endpoints.size() < replicas && iter.hasNext())
- {
- Token t = iter.next();
- if (!endpoints.contains(metadata.getEndpoint(t)))
- endpoints.add(metadata.getEndpoint(t));
- }
+ Token t = iter.next();
+ if (!endpoints.contains(metadata.getEndpoint(t)))
+ endpoints.add(metadata.getEndpoint(t));
}
}