Author: jbellis
Date: Tue Sep 21 19:40:54 2010
New Revision: 999571
URL: http://svn.apache.org/viewvc?rev=999571&view=rev
Log:
Change Strategy to generate List of endpoints instead of Set to preserve order.
Document that SimpleSnitch maintains that order.
Modified:
cassandra/trunk/conf/cassandra.yaml
cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
cassandra/trunk/src/java/org/apache/cassandra/locator/LocalStrategy.java
cassandra/trunk/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
cassandra/trunk/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
cassandra/trunk/src/java/org/apache/cassandra/locator/SimpleSnitch.java
cassandra/trunk/src/java/org/apache/cassandra/locator/SimpleStrategy.java
cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
Modified: cassandra/trunk/conf/cassandra.yaml
URL:
http://svn.apache.org/viewvc/cassandra/trunk/conf/cassandra.yaml?rev=999571&r1=999570&r2=999571&view=diff
==============================================================================
--- cassandra/trunk/conf/cassandra.yaml (original)
+++ cassandra/trunk/conf/cassandra.yaml Tue Sep 21 19:40:54 2010
@@ -184,9 +184,16 @@ rpc_timeout_in_ms: 10000
# IEndpointSnitch, which will let Cassandra know enough
# about your network topology to route requests efficiently.
# Out of the box, Cassandra provides
-# org.apache.cassandra.locator.SimpleSnitch,
-# org.apache.cassandra.locator.RackInferringSnitch, and
-# org.apache.cassandra.locator.PropertyFileSnitch.
+# - org.apache.cassandra.locator.SimpleSnitch:
+# Treats Strategy order as proximity. This improves cache locality
+# when disabling read repair, which can further improve throughput.
+# - org.apache.cassandra.locator.RackInferringSnitch:
+# Proximity is determined by rack and data center, which are
+# assumed to correspond to the 3rd and 2nd octet of each node's
+# IP address, respectively
+# org.apache.cassandra.locator.PropertyFileSnitch:
+# - Proximity is determined by rack and data center, which are
+# explicitly configured in cassandra-rack.properties.
endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch
# dynamic_snitch -- This boolean controls whether the above snitch is
Modified:
cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java?rev=999571&r1=999570&r2=999571&view=diff
==============================================================================
---
cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
(original)
+++
cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
Tue Sep 21 19:40:54 2010
@@ -103,7 +103,7 @@ public abstract class AbstractReplicatio
* @return a copy of the natural endpoints for the given token
* @throws IllegalStateException if the number of requested replicas is
greater than the number of known endpoints
*/
- public abstract Set<InetAddress> calculateNaturalEndpoints(Token
searchToken, TokenMetadata tokenMetadata) throws IllegalStateException;
+ public abstract List<InetAddress> calculateNaturalEndpoints(Token
searchToken, TokenMetadata tokenMetadata) throws IllegalStateException;
public IWriteResponseHandler
getWriteResponseHandler(Collection<InetAddress> writeEndpoints,
Multimap<InetAddress,
InetAddress> hintedEndpoints,
Modified:
cassandra/trunk/src/java/org/apache/cassandra/locator/LocalStrategy.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/locator/LocalStrategy.java?rev=999571&r1=999570&r2=999571&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/locator/LocalStrategy.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/locator/LocalStrategy.java
Tue Sep 21 19:40:54 2010
@@ -20,9 +20,7 @@
package org.apache.cassandra.locator;
import java.net.InetAddress;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.dht.Token;
@@ -34,11 +32,8 @@ public class LocalStrategy extends Abstr
super(table, tokenMetadata, snitch, configOptions);
}
- public Set<InetAddress> calculateNaturalEndpoints(Token token,
TokenMetadata metadata)
+ public List<InetAddress> calculateNaturalEndpoints(Token token,
TokenMetadata metadata)
{
- Set<InetAddress> endpoints = new HashSet<InetAddress>(1);
- InetAddress local = FBUtilities.getLocalAddress();
- endpoints.add(local);
- return endpoints;
+ return Arrays.asList(FBUtilities.getLocalAddress());
}
}
Modified:
cassandra/trunk/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java?rev=999571&r1=999570&r2=999571&view=diff
==============================================================================
---
cassandra/trunk/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
(original)
+++
cassandra/trunk/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
Tue Sep 21 19:40:54 2010
@@ -67,12 +67,12 @@ public class NetworkTopologyStrategy ext
datacenters = Collections.unmodifiableMap(newDatacenters);
}
- public Set<InetAddress> calculateNaturalEndpoints(Token searchToken,
TokenMetadata tokenMetadata)
+ public List<InetAddress> calculateNaturalEndpoints(Token searchToken,
TokenMetadata tokenMetadata)
{
int totalReplicas = getReplicationFactor();
Map<String, Integer> remainingReplicas = new HashMap<String,
Integer>(datacenters);
Map<String, Set<String>> dcUsedRacks = new HashMap<String,
Set<String>>();
- Set<InetAddress> endpoints = new HashSet<InetAddress>(totalReplicas);
+ List<InetAddress> endpoints = new
ArrayList<InetAddress>(totalReplicas);
// first pass: only collect replicas on unique racks
for (Iterator<Token> iter =
TokenMetadata.ringIterator(tokenMetadata.sortedTokens(), searchToken);
Modified:
cassandra/trunk/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java?rev=999571&r1=999570&r2=999571&view=diff
==============================================================================
---
cassandra/trunk/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
(original)
+++
cassandra/trunk/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
Tue Sep 21 19:40:54 2010
@@ -38,10 +38,10 @@ public class OldNetworkTopologyStrategy
super(table, tokenMetadata, snitch, configOptions);
}
- public Set<InetAddress> calculateNaturalEndpoints(Token token,
TokenMetadata metadata)
+ public List<InetAddress> calculateNaturalEndpoints(Token token,
TokenMetadata metadata)
{
int replicas = getReplicationFactor();
- Set<InetAddress> endpoints = new HashSet<InetAddress>(replicas);
+ List<InetAddress> endpoints = new ArrayList<InetAddress>(replicas);
ArrayList<Token> tokens = metadata.sortedTokens();
if (tokens.isEmpty())
Modified:
cassandra/trunk/src/java/org/apache/cassandra/locator/SimpleSnitch.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/locator/SimpleSnitch.java?rev=999571&r1=999570&r2=999571&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/locator/SimpleSnitch.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/locator/SimpleSnitch.java Tue
Sep 21 19:40:54 2010
@@ -24,8 +24,9 @@ import java.util.Collection;
import java.util.List;
/**
- * A simple endpoint snitch implementation does not sort addresses by
- * proximity.
+ * A simple endpoint snitch implementation that treats Strategy order as
proximity,
+ * allowing non-read-repaired reads to prefer a single endpoint, which improves
+ * cache locality.
*/
public class SimpleSnitch extends AbstractEndpointSnitch
{
Modified:
cassandra/trunk/src/java/org/apache/cassandra/locator/SimpleStrategy.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/locator/SimpleStrategy.java?rev=999571&r1=999570&r2=999571&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/locator/SimpleStrategy.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/locator/SimpleStrategy.java
Tue Sep 21 19:40:54 2010
@@ -37,11 +37,11 @@ public class SimpleStrategy extends Abst
super(table, tokenMetadata, snitch, configOptions);
}
- public Set<InetAddress> calculateNaturalEndpoints(Token token,
TokenMetadata metadata)
+ public List<InetAddress> calculateNaturalEndpoints(Token token,
TokenMetadata metadata)
{
int replicas = getReplicationFactor();
ArrayList<Token> tokens = metadata.sortedTokens();
- Set<InetAddress> endpoints = new HashSet<InetAddress>(replicas);
+ List<InetAddress> endpoints = new ArrayList<InetAddress>(replicas);
if (tokens.isEmpty())
return endpoints;
Modified:
cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java?rev=999571&r1=999570&r2=999571&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
Tue Sep 21 19:40:54 2010
@@ -42,7 +42,6 @@ import org.slf4j.LoggerFactory;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.Multimap;
-import com.google.common.collect.Multimaps;
import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
import org.apache.cassandra.concurrent.NamedThreadFactory;
import org.apache.cassandra.concurrent.StageManager;
@@ -813,8 +812,8 @@ public class StorageService implements I
// all leaving nodes are gone.
for (Range range : affectedRanges)
{
- Set<InetAddress> currentEndpoints =
strategy.calculateNaturalEndpoints(range.right, tm);
- Set<InetAddress> newEndpoints =
strategy.calculateNaturalEndpoints(range.right, allLeftMetadata);
+ Collection<InetAddress> currentEndpoints =
strategy.calculateNaturalEndpoints(range.right, tm);
+ Collection<InetAddress> newEndpoints =
strategy.calculateNaturalEndpoints(range.right, allLeftMetadata);
newEndpoints.removeAll(currentEndpoints);
pendingRanges.putAll(range, newEndpoints);
}
@@ -919,7 +918,7 @@ public class StorageService implements I
if (logger_.isDebugEnabled())
logger_.debug("Node " + endpoint + " ranges [" +
StringUtils.join(ranges, ", ") + "]");
- Map<Range, Set<InetAddress>> currentReplicaEndpoints = new
HashMap<Range, Set<InetAddress>>();
+ Map<Range, List<InetAddress>> currentReplicaEndpoints = new
HashMap<Range, List<InetAddress>>();
// Find (for each range) all nodes that store replicas for these
ranges as well
for (Range range : ranges)
@@ -941,7 +940,7 @@ public class StorageService implements I
// range.
for (Range range : ranges)
{
- Set<InetAddress> newReplicaEndpoints =
getReplicationStrategy(table).calculateNaturalEndpoints(range.right, temp);
+ Collection<InetAddress> newReplicaEndpoints =
getReplicationStrategy(table).calculateNaturalEndpoints(range.right, temp);
newReplicaEndpoints.removeAll(currentReplicaEndpoints.get(range));
if (logger_.isDebugEnabled())
if (newReplicaEndpoints.isEmpty())