Author: jbellis
Date: Fri Aug 7 20:48:45 2009
New Revision: 802180
URL: http://svn.apache.org/viewvc?rev=802180&view=rev
Log:
- switch from trying to get the next endpoint by increasing offset to asking
tokenMetadata for "the next
one." this will always be correct where the offset approach will not (usually
you want offset to just be 1,
but sometimes you have to keep increasing it if no results are found but the
range is still not finished)
- merge results differently when the endpoint responsible for where the ring
wraps is involved, since
that endpoint can hold keys from both the beginning and end of the range.
patch by jbellis; tested by Mark Robson for CASSANDRA-348
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/LoadVerbHandler.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RangeReply.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractStrategy.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/IReplicaPlacementStrategy.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/RackAwareStrategy.java
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
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/tools/KeyChecker.java
incubator/cassandra/trunk/test/unit/org/apache/cassandra/locator/RackUnawareStrategyTest.java
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/LoadVerbHandler.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/LoadVerbHandler.java?rev=802180&r1=802179&r2=802180&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/LoadVerbHandler.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/LoadVerbHandler.java
Fri Aug 7 20:48:45 2009
@@ -44,7 +44,7 @@
buffer.reset(body, body.length);
RowMutationMessage rmMsg =
RowMutationMessage.serializer().deserialize(buffer);
- EndPoint[] endpoints =
StorageService.instance().getNStorageEndPoint(rmMsg.getRowMutation().key(), 0);
+ EndPoint[] endpoints =
StorageService.instance().getNStorageEndPoint(rmMsg.getRowMutation().key());
Message messageInternal = new
Message(StorageService.getLocalStorageEndPoint(),
StorageService.mutationStage_,
@@ -68,4 +68,4 @@
}
}
-}
\ No newline at end of file
+}
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RangeReply.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RangeReply.java?rev=802180&r1=802179&r2=802180&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RangeReply.java
(original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RangeReply.java
Fri Aug 7 20:48:45 2009
@@ -35,7 +35,7 @@
public RangeReply(List<String> keys, boolean rangeCompletedLocally)
{
- this.keys = Collections.unmodifiableList(keys);
+ this.keys = keys;
this.rangeCompletedLocally = rangeCompletedLocally;
}
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractStrategy.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractStrategy.java?rev=802180&r1=802179&r2=802180&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractStrategy.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractStrategy.java
Fri Aug 7 20:48:45 2009
@@ -104,7 +104,7 @@
{
List<EndPoint> liveList = new ArrayList<EndPoint>();
Map<EndPoint, EndPoint> map = new HashMap<EndPoint, EndPoint>();
- EndPoint[] topN = getStorageEndPoints( token, 0);
+ EndPoint[] topN = getStorageEndPoints( token );
for( int i = 0 ; i < topN.length ; i++)
{
@@ -131,5 +131,5 @@
return map;
}
- public abstract EndPoint[] getStorageEndPoints(Token token, int offset);
+ public abstract EndPoint[] getStorageEndPoints(Token token);
}
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/IReplicaPlacementStrategy.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/IReplicaPlacementStrategy.java?rev=802180&r1=802179&r2=802180&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/IReplicaPlacementStrategy.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/IReplicaPlacementStrategy.java
Fri Aug 7 20:48:45 2009
@@ -33,8 +33,8 @@
*/
public interface IReplicaPlacementStrategy
{
- public EndPoint[] getStorageEndPoints(Token token, int offset);
- public Map<String, EndPoint[]> getStorageEndPoints(String[] keys, int
offset);
- public EndPoint[] getStorageEndPoints(Token token, Map<Token, EndPoint>
tokenToEndPointMap, int offset);
+ public EndPoint[] getStorageEndPoints(Token token);
+ public Map<String, EndPoint[]> getStorageEndPoints(String[] keys);
+ public EndPoint[] getStorageEndPoints(Token token, Map<Token, EndPoint>
tokenToEndPointMap);
public Map<EndPoint, EndPoint> getHintedStorageEndPoints(Token token);
-}
\ No newline at end of file
+}
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/RackAwareStrategy.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/RackAwareStrategy.java?rev=802180&r1=802179&r2=802180&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/RackAwareStrategy.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/RackAwareStrategy.java
Fri Aug 7 20:48:45 2009
@@ -45,7 +45,7 @@
super(tokenMetadata, partitioner, replicas, storagePort);
}
- public EndPoint[] getStorageEndPoints(Token token, int offset)
+ public EndPoint[] getStorageEndPoints(Token token)
{
int startIndex;
List<EndPoint> list = new ArrayList<EndPoint>();
@@ -123,19 +123,19 @@
return list.toArray(new EndPoint[list.size()]);
}
- public Map<String, EndPoint[]> getStorageEndPoints(String[] keys, int
offset)
+ public Map<String, EndPoint[]> getStorageEndPoints(String[] keys)
{
Map<String, EndPoint[]> results = new HashMap<String, EndPoint[]>();
for ( String key : keys )
{
- results.put(key,
getStorageEndPoints(partitioner_.getInitialToken(key), offset));
+ results.put(key,
getStorageEndPoints(partitioner_.getInitialToken(key)));
}
return results;
}
- public EndPoint[] getStorageEndPoints(Token token, Map<Token, EndPoint>
tokenToEndPointMap, int offset)
+ public EndPoint[] getStorageEndPoints(Token token, Map<Token, EndPoint>
tokenToEndPointMap)
{
throw new UnsupportedOperationException("This operation is not
currently supported");
}
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=802180&r1=802179&r2=802180&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 Aug 7 20:48:45 2009
@@ -41,12 +41,12 @@
super(tokenMetadata, partitioner, replicas, storagePort);
}
- public EndPoint[] getStorageEndPoints(Token token, int offset)
+ public EndPoint[] getStorageEndPoints(Token token)
{
- return getStorageEndPoints(token,
tokenMetadata_.cloneTokenEndPointMap(), offset);
+ return getStorageEndPoints(token,
tokenMetadata_.cloneTokenEndPointMap());
}
- public EndPoint[] getStorageEndPoints(Token token, Map<Token, EndPoint>
tokenToEndPointMap, int offset)
+ public EndPoint[] getStorageEndPoints(Token token, Map<Token, EndPoint>
tokenToEndPointMap)
{
int startIndex;
List<EndPoint> list = new ArrayList<EndPoint>();
@@ -61,9 +61,8 @@
index = 0;
}
int totalNodes = tokens.size();
- int realIndex = (index + offset) % totalNodes;
// Add the node at the index by default
- list.add(tokenToEndPointMap.get(tokens.get(realIndex)));
+ list.add(tokenToEndPointMap.get(tokens.get(index)));
foundCount++;
startIndex = (index + 1)%totalNodes;
// If we found N number of nodes we are good. This loop will just
exit. Otherwise just
@@ -80,15 +79,15 @@
return list.toArray(new EndPoint[list.size()]);
}
- public Map<String, EndPoint[]> getStorageEndPoints(String[] keys, int
offset)
+ public Map<String, EndPoint[]> getStorageEndPoints(String[] keys)
{
Map<String, EndPoint[]> results = new HashMap<String, EndPoint[]>();
for ( String key : keys )
{
- results.put(key,
getStorageEndPoints(partitioner_.getInitialToken(key), offset));
+ results.put(key,
getStorageEndPoints(partitioner_.getInitialToken(key)));
}
return results;
}
-}
\ No newline at end of file
+}
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=802180&r1=802179&r2=802180&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 Aug 7 20:48:45 2009
@@ -18,9 +18,7 @@
package org.apache.cassandra.locator;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -122,7 +120,43 @@
lock_.readLock().unlock();
}
}
+
+ public EndPoint getFirstEndpoint()
+ {
+ lock_.readLock().lock();
+ try
+ {
+ ArrayList<Token> tokens = new
ArrayList<Token>(tokenToEndPointMap_.keySet());
+ if (tokens.isEmpty())
+ return null;
+ Collections.sort(tokens);
+ return tokenToEndPointMap_.get(tokens.get(0));
+ }
+ finally
+ {
+ lock_.readLock().unlock();
+ }
+ }
+
+ public EndPoint getNextEndpoint(EndPoint endPoint)
+ {
+ lock_.readLock().lock();
+ try
+ {
+ ArrayList<Token> tokens = new
ArrayList<Token>(tokenToEndPointMap_.keySet());
+ if (tokens.isEmpty())
+ return null;
+ Collections.sort(tokens);
+ int i = tokens.indexOf(endPointToTokenMap_.get(endPoint)); // TODO
binary search
+ return tokenToEndPointMap_.get(tokens.get((i + 1) %
tokens.size()));
+ }
+ finally
+ {
+ lock_.readLock().unlock();
+ }
+ }
+
/*
* Returns a safe clone of tokenToEndPointMap_.
*/
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=802180&r1=802179&r2=802180&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 Aug 7 20:48:45 2009
@@ -18,12 +18,7 @@
package org.apache.cassandra.service;
import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.lang.management.ManagementFactory;
@@ -38,6 +33,9 @@
import org.apache.cassandra.net.Message;
import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.utils.TimedStatsDeque;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.TokenMetadata;
+
import org.apache.log4j.Logger;
import javax.management.MBeanServer;
@@ -164,7 +162,7 @@
}
try
{
- EndPoint[] endpoints =
StorageService.instance().getNStorageEndPoint(rm.key(), 0);
+ EndPoint[] endpoints =
StorageService.instance().getNStorageEndPoint(rm.key());
if (endpoints.length < (DatabaseDescriptor.getReplicationFactor()
/ 2) + 1)
{
throw new UnavailableException();
@@ -312,7 +310,7 @@
long startTime = System.currentTimeMillis();
Row row;
- EndPoint[] endpoints =
StorageService.instance().getNStorageEndPoint(command.key, 0);
+ EndPoint[] endpoints =
StorageService.instance().getNStorageEndPoint(command.key);
if (consistency_level == ConsistencyLevel.ONE)
{
@@ -405,7 +403,7 @@
DatabaseDescriptor.getQuorum(),
readResponseResolver);
EndPoint dataPoint =
StorageService.instance().findSuitableEndPoint(command.key);
- List<EndPoint> endpointList = new
ArrayList<EndPoint>(Arrays.asList(StorageService.instance().getNStorageEndPoint(command.key,
0)));
+ List<EndPoint> endpointList = new
ArrayList<EndPoint>(Arrays.asList(StorageService.instance().getNStorageEndPoint(command.key)));
/* Remove the local storage endpoint from the list. */
endpointList.remove(dataPoint);
EndPoint[] endPoints = new EndPoint[endpointList.size() + 1];
@@ -619,15 +617,17 @@
return row;
}
- static List<String> getKeyRange(RangeCommand command) throws IOException
+ static List<String> getKeyRange(RangeCommand rawCommand) throws IOException
{
long startTime = System.currentTimeMillis();
- int endpointOffset = 0;
+ Comparator<String> comparator =
StorageService.getPartitioner().getDecoratedKeyComparator();
+ TokenMetadata tokenMetadata =
StorageService.instance().getTokenMetadata();
List<String> allKeys = new ArrayList<String>();
- int maxResults = command.maxResults;
+ RangeCommand command = rawCommand;
- EndPoint endPoint =
StorageService.instance().findSuitableEndPoint(command.startWith,
endpointOffset);
- String firstEndpoint = endPoint.toString();
+ EndPoint endPoint =
StorageService.instance().findSuitableEndPoint(command.startWith);
+ EndPoint startEndpoint = endPoint;
+ EndPoint wrapEndpoint = tokenMetadata.getFirstEndpoint();
do
{
@@ -649,30 +649,64 @@
RangeReply rangeReply = RangeReply.read(responseBody);
List<String> rangeKeys = rangeReply.keys;
- // deal with key overlaps
- if (allKeys.size() > 0 && rangeKeys != null && rangeKeys.size() >
0 && allKeys.get(allKeys.size() - 1).equals(rangeKeys.get(0)))
- {
- allKeys.remove(allKeys.size() - 1);
- allKeys.addAll(rangeKeys);
- }
- else if (rangeKeys != null && rangeKeys.size() > 0)
+ if (rangeKeys.size() > 0)
{
- allKeys.addAll(rangeKeys);
+ if (allKeys.size() > 0)
+ {
+ if (comparator.compare(rangeKeys.get(rangeKeys.size() -
1), allKeys.get(0)) <= 0)
+ {
+ // unlikely, but possible
+ if (rangeKeys.get(rangeKeys.size() -
1).equals(allKeys.get(0)))
+ {
+ rangeKeys.remove(rangeKeys.size() - 1);
+ }
+ rangeKeys.addAll(allKeys);
+ allKeys = rangeKeys;
+ }
+ else if (comparator.compare(allKeys.get(allKeys.size() -
1), rangeKeys.get(0)) <= 0)
+ {
+ // common case. deal with simple start/end key overlaps
+ if (allKeys.get(allKeys.size() -
1).equals(rangeKeys.get(0)))
+ {
+ allKeys.remove(allKeys.size() - 1);
+ }
+ allKeys.addAll(rangeKeys);
+ }
+ else
+ {
+ // deal with potential large overlap from scanning the
first endpoint, which contains
+ // both the smallest and largest keys
+ HashSet<String> keys = new HashSet<String>(allKeys);
+ keys.addAll(rangeKeys);
+ allKeys = new ArrayList<String>(keys);
+ Collections.sort(allKeys);
+ }
+ }
+ else
+ {
+ allKeys = rangeKeys;
+ }
}
- if (allKeys.size() >= maxResults ||
rangeReply.rangeCompletedLocally)
+ if (allKeys.size() >= rawCommand.maxResults ||
rangeReply.rangeCompletedLocally)
{
break;
}
- String newStartWith = (allKeys.size() > 0) ?
allKeys.get(allKeys.size() - 1) : command.startWith;
- command = new RangeCommand(command.table, command.columnFamily,
newStartWith, command.stopAt, command.maxResults - allKeys.size());
- endPoint =
StorageService.instance().findSuitableEndPoint(command.startWith,
++endpointOffset);
- } while (!endPoint.toString().equals(firstEndpoint));
+ // the first endpoint contains the range from the last endpoint,
up to and including its own token.
+ // so it will include both the smallest keys, and the largest. if
that is what we just scanned,
+ // leave startWith unchanged. Otherwise, start with the largest
key found.
+ String newStartWith = endPoint.equals(wrapEndpoint)
+ ? rawCommand.startWith
+ : allKeys.size() > 0 ?
allKeys.get(allKeys.size() - 1) : command.startWith;
+ endPoint = tokenMetadata.getNextEndpoint(endPoint); // TODO move
this into the Strategies & modify for RackAwareStrategy
+ int maxResults = endPoint == wrapEndpoint ? rawCommand.maxResults
: rawCommand.maxResults - allKeys.size();
+ command = new RangeCommand(command.table, command.columnFamily,
newStartWith, command.stopAt, maxResults);
+ } while (!endPoint.equals(startEndpoint));
rangeStats.add(System.currentTimeMillis() - startTime);
- return (allKeys.size() > maxResults)
- ? allKeys.subList(0, maxResults)
+ return (allKeys.size() > rawCommand.maxResults)
+ ? allKeys.subList(0, rawCommand.maxResults)
: allKeys;
}
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java?rev=802180&r1=802179&r2=802180&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
Fri Aug 7 20:48:45 2009
@@ -898,21 +898,17 @@
* @param key - key for which we need to find the endpoint return value -
* the endpoint responsible for this key
*/
- public EndPoint[] getNStorageEndPoint(String key, int offset)
+ public EndPoint[] getNStorageEndPoint(String key)
{
- return
nodePicker_.getStorageEndPoints(partitioner_.getInitialToken(key), offset);
- }
-
- private Map<String, EndPoint[]> getNStorageEndPoints(String[] keys, int
offset)
- {
- return nodePicker_.getStorageEndPoints(keys, offset);
+ return
nodePicker_.getStorageEndPoints(partitioner_.getInitialToken(key));
}
private Map<String, EndPoint[]> getNStorageEndPoints(String[] keys)
{
- return getNStorageEndPoints(keys, 0);
+ return nodePicker_.getStorageEndPoints(keys);
}
+
/**
* This method attempts to return N endpoints that are responsible for
storing the
* specified key i.e for replication.
@@ -923,7 +919,7 @@
public List<EndPoint> getNLiveStorageEndPoint(String key)
{
List<EndPoint> liveEps = new ArrayList<EndPoint>();
- EndPoint[] endpoints = getNStorageEndPoint(key, 0);
+ EndPoint[] endpoints = getNStorageEndPoint(key);
for ( EndPoint endpoint : endpoints )
{
@@ -954,7 +950,7 @@
*/
public EndPoint[] getNStorageEndPoint(Token token)
{
- return nodePicker_.getStorageEndPoints(token, 0);
+ return nodePicker_.getStorageEndPoints(token);
}
/**
@@ -967,21 +963,16 @@
*/
protected EndPoint[] getNStorageEndPoint(Token token, Map<Token, EndPoint>
tokenToEndPointMap)
{
- return nodePicker_.getStorageEndPoints(token, tokenToEndPointMap, 0);
- }
-
- public EndPoint findSuitableEndPoint(String key) throws IOException
- {
- return findSuitableEndPoint(key, 0);
+ return nodePicker_.getStorageEndPoints(token, tokenToEndPointMap);
}
/**
* This function finds the most suitable endpoint given a key.
* It checks for locality and alive test.
*/
- public EndPoint findSuitableEndPoint(String key, int offset) throws
IOException
+ public EndPoint findSuitableEndPoint(String key) throws IOException
{
- EndPoint[] endpoints = getNStorageEndPoint(key, offset);
+ EndPoint[] endpoints = getNStorageEndPoint(key);
for(EndPoint endPoint: endpoints)
{
if(endPoint.equals(StorageService.getLocalStorageEndPoint()))
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/tools/KeyChecker.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/tools/KeyChecker.java?rev=802180&r1=802179&r2=802180&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/tools/KeyChecker.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/tools/KeyChecker.java
Fri Aug 7 20:48:45 2009
@@ -40,7 +40,7 @@
*/
private static boolean checkIfProcessKey(String key)
{
- EndPoint[] endPoints =
StorageService.instance().getNStorageEndPoint(key, 0);
+ EndPoint[] endPoints =
StorageService.instance().getNStorageEndPoint(key);
EndPoint localEndPoint = StorageService.getLocalStorageEndPoint();
for(EndPoint endPoint : endPoints)
{
@@ -96,4 +96,4 @@
System.out.println("DONE checking keys ...");
raf.close();
}
-}
\ No newline at end of file
+}
Modified:
incubator/cassandra/trunk/test/unit/org/apache/cassandra/locator/RackUnawareStrategyTest.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/locator/RackUnawareStrategyTest.java?rev=802180&r1=802179&r2=802180&view=diff
==============================================================================
---
incubator/cassandra/trunk/test/unit/org/apache/cassandra/locator/RackUnawareStrategyTest.java
(original)
+++
incubator/cassandra/trunk/test/unit/org/apache/cassandra/locator/RackUnawareStrategyTest.java
Fri Aug 7 20:48:45 2009
@@ -78,7 +78,7 @@
for (int i = 0; i < keyTokens.length; i++)
{
- EndPoint[] endPoints = strategy.getStorageEndPoints(keyTokens[i],
0);
+ EndPoint[] endPoints = strategy.getStorageEndPoints(keyTokens[i]);
assert endPoints.length == 3;
for (int j = 0; j < endPoints.length; j++)
{