Author: jbellis
Date: Tue Nov 3 02:24:05 2009
New Revision: 832267
URL: http://svn.apache.org/viewvc?rev=832267&view=rev
Log:
brute-force replacement of throwing maps around with encapsulated TokenMetadata
patch by jbellis for CASSANDRA-522
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.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/CassandraServer.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java?rev=832267&r1=832266&r2=832267&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java
Tue Nov 3 02:24:05 2009
@@ -146,14 +146,13 @@
Multimap<Range, InetAddress> getRangesWithSources()
{
- Map<Token, InetAddress> map = tokenMetadata.cloneTokenEndPointMap();
- assert map.size() > 0;
- map.put(token, address);
- Collection<Range> myRanges =
replicationStrategy.getAddressRanges(map).get(address);
- map.remove(token);
+ TokenMetadata temp = tokenMetadata.cloneMe();
+ assert temp.sortedTokens().size() > 0;
+ temp.update(token, address);
+ Collection<Range> myRanges =
replicationStrategy.getAddressRanges(temp).get(address);
Multimap<Range, InetAddress> myRangeAddresses = HashMultimap.create();
- Multimap<Range, InetAddress> rangeAddresses =
replicationStrategy.getRangeAddresses(map);
+ Multimap<Range, InetAddress> rangeAddresses =
replicationStrategy.getRangeAddresses(tokenMetadata);
for (Range range : rangeAddresses.keySet())
{
for (Range myRange : myRanges)
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java?rev=832267&r1=832266&r2=832267&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
Tue Nov 3 02:24:05 2009
@@ -56,7 +56,7 @@
storagePort_ = storagePort;
}
- public abstract ArrayList<InetAddress> getNaturalEndpoints(Token token,
Map<Token, InetAddress> tokenToEndPointMap);
+ public abstract ArrayList<InetAddress> getNaturalEndpoints(Token token,
TokenMetadata metadata);
public <T> QuorumResponseHandler<T>
getResponseHandler(IResponseResolver<T> responseResolver, int blockFor, int
consistency_level) throws InvalidRequestException
{
@@ -65,7 +65,7 @@
public ArrayList<InetAddress> getNaturalEndpoints(Token token)
{
- return getNaturalEndpoints(token,
tokenMetadata_.cloneTokenEndPointMap());
+ return getNaturalEndpoints(token, tokenMetadata_);
}
/*
@@ -88,29 +88,21 @@
*/
public ArrayList<InetAddress> getWriteEndpoints(Token token,
Collection<InetAddress> naturalEndpoints)
{
- Map<Token, InetAddress> tokenToEndPointMap =
tokenMetadata_.cloneTokenEndPointMap();
- Map<Token, InetAddress> bootstrapTokensToEndpointMap =
tokenMetadata_.cloneBootstrapNodes();
ArrayList<InetAddress> endpoints = new
ArrayList<InetAddress>(naturalEndpoints);
- for (Token t : bootstrapTokensToEndpointMap.keySet())
+ for (Token t : tokenMetadata_.bootstrapTokens())
{
- InetAddress ep = bootstrapTokensToEndpointMap.get(t);
- tokenToEndPointMap.put(t, ep);
- try
+ TokenMetadata temp = tokenMetadata_.cloneMe();
+ InetAddress ep = tokenMetadata_.getBootstrapEndpoint(t);
+ temp.update(t, ep);
+ for (Range r : getAddressRanges(temp).get(ep))
{
- for (Range r : getAddressRanges(tokenToEndPointMap).get(ep))
+ if (r.contains(token))
{
- if (r.contains(token))
- {
- endpoints.add(ep);
- break;
- }
+ endpoints.add(ep);
+ break;
}
}
- finally
- {
- tokenToEndPointMap.remove(t);
- }
}
return endpoints;
@@ -139,9 +131,7 @@
{
// find another endpoint to store a hint on. prefer endpoints
that aren't already in use
InetAddress hintLocation = null;
- Map<Token, InetAddress> tokenToEndPointMap =
tokenMetadata_.cloneTokenEndPointMap();
- List tokens = new ArrayList(tokenToEndPointMap.keySet());
- Collections.sort(tokens);
+ List tokens = tokenMetadata_.sortedTokens();
Token token = tokenMetadata_.getToken(ep);
int index = Collections.binarySearch(tokens, token);
if (index < 0)
@@ -154,7 +144,7 @@
int startIndex = (index + 1) % totalNodes;
for (int i = startIndex, count = 1; count < totalNodes;
++count, i = (i + 1) % totalNodes)
{
- InetAddress tmpEndPoint =
tokenToEndPointMap.get(tokens.get(i));
+ InetAddress tmpEndPoint =
tokenMetadata_.getEndPoint((Token) tokens.get(i));
if (FailureDetector.instance().isAlive(tmpEndPoint) &&
!targets.contains(tmpEndPoint) && !usedEndpoints.contains(tmpEndPoint))
{
hintLocation = tmpEndPoint;
@@ -176,13 +166,13 @@
// TODO this is pretty inefficient. also the inverse (getRangeAddresses)
below.
// fixing this probably requires merging tokenmetadata into
replicationstrategy, so we can cache/invalidate cleanly
- public Multimap<InetAddress, Range> getAddressRanges(Map<Token,
InetAddress> metadata)
+ public Multimap<InetAddress, Range> getAddressRanges(TokenMetadata
metadata)
{
Multimap<InetAddress, Range> map = HashMultimap.create();
- for (Token token : metadata.keySet())
+ for (Token token : metadata.sortedTokens())
{
- Range range = getPrimaryRangeFor(token, metadata);
+ Range range = metadata.getPrimaryRangeFor(token);
for (InetAddress ep : getNaturalEndpoints(token, metadata))
{
map.put(ep, range);
@@ -192,13 +182,13 @@
return map;
}
- public Multimap<Range, InetAddress> getRangeAddresses(Map<Token,
InetAddress> metadata)
+ public Multimap<Range, InetAddress> getRangeAddresses(TokenMetadata
metadata)
{
Multimap<Range, InetAddress> map = HashMultimap.create();
- for (Token token : metadata.keySet())
+ for (Token token : metadata.sortedTokens())
{
- Range range = getPrimaryRangeFor(token, metadata);
+ Range range = metadata.getPrimaryRangeFor(token);
for (InetAddress ep : getNaturalEndpoints(token, metadata))
{
map.put(range, ep);
@@ -210,27 +200,6 @@
public Multimap<InetAddress, Range> getAddressRanges()
{
- return getAddressRanges(tokenMetadata_.cloneTokenEndPointMap());
- }
-
- public Range getPrimaryRangeFor(Token right, Map<Token, InetAddress>
tokenToEndPointMap)
- {
- return new Range(getPredecessor(right, tokenToEndPointMap), right);
- }
-
- public Token getPredecessor(Token token, Map<Token, InetAddress>
tokenToEndPointMap)
- {
- List tokens = new ArrayList<Token>(tokenToEndPointMap.keySet());
- Collections.sort(tokens);
- int index = Collections.binarySearch(tokens, token);
- return (Token) (index == 0 ? tokens.get(tokens.size() - 1) :
tokens.get(--index));
- }
-
- public Token getSuccessor(Token token, Map<Token, InetAddress>
tokenToEndPointMap)
- {
- List tokens = new ArrayList<Token>(tokenToEndPointMap.keySet());
- Collections.sort(tokens);
- int index = Collections.binarySearch(tokens, token);
- return (Token) ((index == (tokens.size() - 1)) ? tokens.get(0) :
tokens.get(++index));
+ return getAddressRanges(tokenMetadata_);
}
}
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java?rev=832267&r1=832266&r2=832267&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java
Tue Nov 3 02:24:05 2009
@@ -45,10 +45,8 @@
/**
* This Method will get the required information of the EndPoint from the
* DataCenterEndPointSnitch and poopulates this singleton class.
- *
- * @param tokenToEndPointMap - Provided the endpoint map which will be
mapped with the DC's
*/
- private synchronized void loadEndPoints(Map<Token, InetAddress>
tokenToEndPointMap, Collection<Token> tokens) throws IOException
+ private synchronized void loadEndPoints(TokenMetadata metadata) throws
IOException
{
endPointSnitch = (DatacenterEndPointSnitch)
StorageService.instance().getEndPointSnitch();
this.tokens = new ArrayList<Token>(tokens);
@@ -56,7 +54,7 @@
dcMap = new HashMap<String, List<Token>>();
for (Token token : this.tokens)
{
- InetAddress endPoint = tokenToEndPointMap.get(token);
+ InetAddress endPoint = metadata.getEndPoint(token);
String dataCenter = endPointSnitch.getLocation(endPoint);
if (dataCenter.equals(localDC))
{
@@ -96,25 +94,24 @@
assert (DatabaseDescriptor.getEndPointSnitch() instanceof
DatacenterEndPointSnitch);
}
- @Override
- public ArrayList<InetAddress> getNaturalEndpoints(Token searchToken,
Map<Token, InetAddress> tokenToEndPointMap)
+ public ArrayList<InetAddress> getNaturalEndpoints(Token token,
TokenMetadata metadata)
{
try
{
- return getNaturalEndpointsInternal(searchToken,
tokenToEndPointMap);
+ return getNaturalEndpointsInternal(token, metadata);
}
catch (IOException e)
{
- throw new IOError(e);
+ throw new IOError(e);
}
}
- private ArrayList<InetAddress> getNaturalEndpointsInternal(Token
searchToken, Map<Token, InetAddress> tokenToEndPointMap) throws IOException
+ private ArrayList<InetAddress> getNaturalEndpointsInternal(Token
searchToken, TokenMetadata metadata) throws IOException
{
ArrayList<InetAddress> endpoints = new ArrayList<InetAddress>();
- if (null == tokens || this.tokens.size() !=
tokenToEndPointMap.keySet().size())
+ if (null == tokens || tokens.size() != metadata.sortedTokens().size())
{
- loadEndPoints(tokenToEndPointMap, tokenToEndPointMap.keySet());
+ loadEndPoints(metadata);
}
for (String dc : dcMap.keySet())
@@ -136,7 +133,7 @@
}
int totalNodes = tokens.size();
// Add the node at the index by default
- InetAddress primaryHost =
tokenToEndPointMap.get(tokens.get(index));
+ InetAddress primaryHost = metadata.getEndPoint((Token)
tokens.get(index));
forloopReturn.add(primaryHost);
foundCount++;
if (replicas_ == 1)
@@ -147,7 +144,7 @@
int startIndex = (index + 1) % totalNodes;
for (int i = startIndex, count = 1; count < totalNodes &&
foundCount < replicas_; ++count, i = (i + 1) % totalNodes)
{
- InetAddress endPointOfIntrest =
tokenToEndPointMap.get(tokens.get(i));
+ InetAddress endPointOfIntrest = metadata.getEndPoint((Token)
tokens.get(i));
if ((replicas_ - 1) > foundCount)
{
forloopReturn.add(endPointOfIntrest);
@@ -163,7 +160,7 @@
{
if (!endPointSnitch.isOnSameRack(primaryHost,
endPointOfIntrest))
{
-
forloopReturn.add(tokenToEndPointMap.get(tokens.get(i)));
+ forloopReturn.add(metadata.getEndPoint((Token)
tokens.get(i)));
bOtherRack = true;
foundCount++;
}
@@ -182,9 +179,10 @@
*/
for (int i = startIndex, count = 1; count < totalNodes &&
foundCount < replicas_; ++count, i = (i + 1) % totalNodes)
{
- if
(!forloopReturn.contains(tokenToEndPointMap.get(tokens.get(i))))
+ Token t = (Token) tokens.get(i);
+ if (!forloopReturn.contains(metadata.getEndPoint(t)))
{
- forloopReturn.add(tokenToEndPointMap.get(tokens.get(i)));
+ forloopReturn.add(metadata.getEndPoint(t));
foundCount++;
}
}
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=832267&r1=832266&r2=832267&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
Tue Nov 3 02:24:05 2009
@@ -44,15 +44,14 @@
super(tokenMetadata, partitioner, replicas, storagePort);
}
- public ArrayList<InetAddress> getNaturalEndpoints(Token token, Map<Token,
InetAddress> tokenToEndPointMap)
+ public ArrayList<InetAddress> getNaturalEndpoints(Token token,
TokenMetadata metadata)
{
int startIndex;
ArrayList<InetAddress> endpoints = new ArrayList<InetAddress>();
boolean bDataCenter = false;
boolean bOtherRack = false;
int foundCount = 0;
- List tokens = new ArrayList(tokenToEndPointMap.keySet());
- Collections.sort(tokens);
+ List tokens = metadata.sortedTokens();
int index = Collections.binarySearch(tokens, token);
if(index < 0)
{
@@ -62,7 +61,8 @@
}
int totalNodes = tokens.size();
// Add the node at the index by default
- endpoints.add(tokenToEndPointMap.get(tokens.get(index)));
+ Token primaryToken = (Token) tokens.get(index);
+ endpoints.add(metadata.getEndPoint(primaryToken));
foundCount++;
if (replicas_ == 1)
{
@@ -70,31 +70,32 @@
}
startIndex = (index + 1)%totalNodes;
IEndPointSnitch endPointSnitch =
StorageService.instance().getEndPointSnitch();
-
- for (int i = startIndex, count = 1; count < totalNodes && foundCount <
replicas_; ++count, i = (i+1)%totalNodes)
+
+ for (int i = startIndex, count = 1; count < totalNodes && foundCount <
replicas_; ++count, i = (i + 1) % totalNodes)
{
try
{
// First try to find one in a different data center
-
if(!endPointSnitch.isInSameDataCenter(tokenToEndPointMap.get(tokens.get(index)),
tokenToEndPointMap.get(tokens.get(i))))
+ Token t = (Token) tokens.get(i);
+ if
(!endPointSnitch.isInSameDataCenter(metadata.getEndPoint(primaryToken),
metadata.getEndPoint(t)))
{
// If we have already found something in a diff datacenter
no need to find another
- if( !bDataCenter )
+ if (!bDataCenter)
{
- endpoints.add(tokenToEndPointMap.get(tokens.get(i)));
+ endpoints.add(metadata.getEndPoint(t));
bDataCenter = true;
foundCount++;
}
continue;
}
// Now try to find one on a different rack
-
if(!endPointSnitch.isOnSameRack(tokenToEndPointMap.get(tokens.get(index)),
tokenToEndPointMap.get(tokens.get(i))) &&
-
endPointSnitch.isInSameDataCenter(tokenToEndPointMap.get(tokens.get(index)),
tokenToEndPointMap.get(tokens.get(i))))
+ if
(!endPointSnitch.isOnSameRack(metadata.getEndPoint(primaryToken),
metadata.getEndPoint(t)) &&
+
endPointSnitch.isInSameDataCenter(metadata.getEndPoint(primaryToken),
metadata.getEndPoint(t)))
{
// If we have already found something in a diff rack no
need to find another
- if( !bOtherRack )
+ if (!bOtherRack)
{
- endpoints.add(tokenToEndPointMap.get(tokens.get(i)));
+ endpoints.add(metadata.getEndPoint(t));
bOtherRack = true;
foundCount++;
}
@@ -103,7 +104,7 @@
catch (UnknownHostException e)
{
if (logger_.isDebugEnabled())
- logger_.debug(LogUtil.throwableToString(e));
+ logger_.debug(LogUtil.throwableToString(e));
}
}
@@ -111,9 +112,10 @@
// loop through the list and add until we have N nodes.
for (int i = startIndex, count = 1; count < totalNodes && foundCount <
replicas_; ++count, i = (i+1)%totalNodes)
{
- if( ! endpoints.contains(tokenToEndPointMap.get(tokens.get(i))))
+ Token t = (Token) tokens.get(i);
+ if (!endpoints.contains(metadata.getEndPoint(t)))
{
- endpoints.add(tokenToEndPointMap.get(tokens.get(i)));
+ endpoints.add(metadata.getEndPoint(t));
foundCount++;
}
}
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=832267&r1=832266&r2=832267&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
Tue Nov 3 02:24:05 2009
@@ -40,15 +40,15 @@
super(tokenMetadata, partitioner, replicas, storagePort);
}
- public ArrayList<InetAddress> getNaturalEndpoints(Token token, Map<Token,
InetAddress> tokenToEndPointMap)
+ // TODO this doesn't need to worry about bootstrap tokens being in the
normal tokens map
+ public ArrayList<InetAddress> getNaturalEndpoints(Token token,
TokenMetadata metadata)
{
int startIndex;
List<Token> tokenList = new ArrayList<Token>();
int foundCount = 0;
- List tokens = new ArrayList<Token>(tokenToEndPointMap.keySet());
+ List tokens = new ArrayList<Token>(metadata.sortedTokens());
List<Token> bsTokens = null;
- Collections.sort(tokens);
int index = Collections.binarySearch(tokens, token);
if(index < 0)
{
@@ -75,8 +75,8 @@
}
}
ArrayList<InetAddress> endpoints = new
ArrayList<InetAddress>(tokenList.size());
- for (Token t: tokenList)
- endpoints.add(tokenToEndPointMap.get(t));
+ for (Token t : tokenList)
+ endpoints.add(metadata.getEndPoint(t));
return endpoints;
}
}
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=832267&r1=832266&r2=832267&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
Tue Nov 3 02:24:05 2009
@@ -23,6 +23,8 @@
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.dht.Range;
+
import java.net.InetAddress;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.gms.FailureDetector;
@@ -185,85 +187,105 @@
lock.readLock().unlock();
}
}
-
- public Map<Token, InetAddress> cloneBootstrapNodes()
+
+ public TokenMetadata cloneMe()
{
lock.readLock().lock();
try
- {
- return new HashMap<Token, InetAddress>(bootstrapTokenMap);
+ {
+ return new TokenMetadata(HashBiMap.create(tokenToEndPointMap),
HashBiMap.create(bootstrapTokenMap));
}
finally
{
lock.readLock().unlock();
}
-
}
- /*
- * Returns a safe clone of tokenToEndPointMap_.
- */
- public Map<Token, InetAddress> cloneTokenEndPointMap()
+ public String toString()
{
+ StringBuilder sb = new StringBuilder();
lock.readLock().lock();
try
- {
- return new HashMap<Token, InetAddress>(tokenToEndPointMap);
+ {
+ Set<InetAddress> eps = tokenToEndPointMap.inverse().keySet();
+
+ for (InetAddress ep : eps)
+ {
+ sb.append(ep);
+ sb.append(":");
+ sb.append(tokenToEndPointMap.inverse().get(ep));
+ sb.append(System.getProperty("line.separator"));
+ }
}
finally
{
lock.readLock().unlock();
}
+
+ return sb.toString();
}
-
- /*
- * Returns a safe clone of endPointTokenMap_.
- */
- public Map<InetAddress, Token> cloneEndPointTokenMap()
+
+ public InetAddress getEndPoint(Token token)
{
lock.readLock().lock();
try
- {
- return new HashMap<InetAddress,
Token>(tokenToEndPointMap.inverse());
+ {
+ return tokenToEndPointMap.get(token);
}
finally
{
lock.readLock().unlock();
}
}
-
- public String toString()
+
+ public void clearUnsafe()
{
- StringBuilder sb = new StringBuilder();
- Set<InetAddress> eps = tokenToEndPointMap.inverse().keySet();
+ tokenToEndPointMap.clear();
+ bootstrapTokenMap.clear();
+ }
- for ( InetAddress ep : eps )
- {
- sb.append(ep);
- sb.append(":");
- sb.append(tokenToEndPointMap.inverse().get(ep));
- sb.append(System.getProperty("line.separator"));
- }
-
- return sb.toString();
+ public Range getPrimaryRangeFor(Token right)
+ {
+ return new Range(getPredecessor(right), right);
}
- public InetAddress getEndPoint(Token token)
+ public List<Token> sortedTokens()
{
+ List<Token> tokens;
lock.readLock().lock();
try
{
- return tokenToEndPointMap.get(token);
+ tokens = new ArrayList<Token>(tokenToEndPointMap.keySet());
}
finally
{
lock.readLock().unlock();
}
+ Collections.sort(tokens);
+ return tokens;
}
- public void clearUnsafe()
+ public Token getPredecessor(Token token)
{
- tokenToEndPointMap.clear();
- bootstrapTokenMap.clear();
+ List tokens = sortedTokens();
+ int index = Collections.binarySearch(tokens, token);
+ return (Token) (index == 0 ? tokens.get(tokens.size() - 1) :
tokens.get(--index));
+ }
+
+ public Token getSuccessor(Token token)
+ {
+ List tokens = sortedTokens();
+ int index = Collections.binarySearch(tokens, token);
+ return (Token) ((index == (tokens.size() - 1)) ? tokens.get(0) :
tokens.get(++index));
+ }
+
+ public Iterable<? extends Token> bootstrapTokens()
+ {
+ return bootstrapTokenMap.keySet();
+ }
+
+ public InetAddress getBootstrapEndpoint(Token token)
+ {
+ return bootstrapTokenMap.get(token);
}
}
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java?rev=832267&r1=832266&r2=832267&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
Tue Nov 3 02:24:05 2009
@@ -508,11 +508,7 @@
}
else if (propertyName.equals(TOKEN_MAP))
{
- HashMap<String, String> tokenToHostMap = new
HashMap<String,String>();
- Map<Token, InetAddress> endpointMap =
storageService.getLiveEndPointMap();
- for (Map.Entry<Token, InetAddress> e : endpointMap.entrySet())
- tokenToHostMap.put(e.getKey().toString(),
e.getValue().getHostAddress());
- return new JSONSerializer().serialize(tokenToHostMap);
+ return new
JSONSerializer().serialize(storageService.getStringEndpointMap());
}
else if (propertyName.equals("version"))
{
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=832267&r1=832266&r2=832267&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
Tue Nov 3 02:24:05 2009
@@ -281,7 +281,7 @@
}
setAndBroadcastToken(storageMetadata_.getToken());
- assert tokenMetadata_.cloneTokenEndPointMap().size() > 0;
+ assert tokenMetadata_.sortedTokens().size() > 0;
}
public boolean isBootstrapMode()
@@ -322,10 +322,8 @@
public Map<Range, List<String>> getRangeToEndPointMap()
{
- /* Get the token to endpoint map. */
- Map<Token, InetAddress> tokenToEndPointMap =
tokenMetadata_.cloneTokenEndPointMap();
/* All the ranges for the tokens */
- Range[] ranges = getAllRanges(tokenToEndPointMap.keySet());
+ List<Range> ranges = getAllRanges(tokenMetadata_.sortedTokens());
Map<Range, List<String>> map = new HashMap<Range, List<String>>();
for (Map.Entry<Range,List<InetAddress>> entry :
constructRangeToEndPointMap(ranges).entrySet())
{
@@ -340,7 +338,7 @@
* @param ranges
* @return mapping of ranges to the replicas responsible for them.
*/
- public Map<Range, List<InetAddress>> constructRangeToEndPointMap(Range[]
ranges)
+ public Map<Range, List<InetAddress>>
constructRangeToEndPointMap(List<Range> ranges)
{
Map<Range, List<InetAddress>> rangeToEndPointMap = new HashMap<Range,
List<InetAddress>>();
for (Range range : ranges)
@@ -349,27 +347,6 @@
}
return rangeToEndPointMap;
}
-
- /**
- * Construct the range to endpoint mapping based on the view as dictated
- * by the mapping of token to endpoints passed in.
- * @param ranges
- * @param tokenToEndPointMap mapping of token to endpoints.
- * @return mapping of ranges to the replicas responsible for them.
- */
- public Map<Range, List<InetAddress>> constructRangeToEndPointMap(Range[]
ranges, Map<Token, InetAddress> tokenToEndPointMap)
- {
- if (logger_.isDebugEnabled())
- logger_.debug("Constructing range to endpoint map ...");
- Map<Range, List<InetAddress>> rangeToEndPointMap = new HashMap<Range,
List<InetAddress>>();
- for ( Range range : ranges )
- {
- rangeToEndPointMap.put(range,
replicationStrategy_.getNaturalEndpoints(range.right(), tokenToEndPointMap));
- }
- if (logger_.isDebugEnabled())
- logger_.debug("Done constructing range to endpoint map ...");
- return rangeToEndPointMap;
- }
/**
* Called when there is a change in application state. In particular
@@ -643,7 +620,7 @@
InetAddress getPredecessor(InetAddress ep)
{
Token token = tokenMetadata_.getToken(ep);
- return
tokenMetadata_.getEndPoint(replicationStrategy_.getPredecessor(token,
tokenMetadata_.cloneTokenEndPointMap()));
+ return
tokenMetadata_.getEndPoint(tokenMetadata_.getPredecessor(token));
}
/*
@@ -653,7 +630,7 @@
public InetAddress getSuccessor(InetAddress ep)
{
Token token = tokenMetadata_.getToken(ep);
- return
tokenMetadata_.getEndPoint(replicationStrategy_.getSuccessor(token,
tokenMetadata_.cloneTokenEndPointMap()));
+ return tokenMetadata_.getEndPoint(tokenMetadata_.getSuccessor(token));
}
/**
@@ -663,8 +640,7 @@
*/
public Range getPrimaryRangeForEndPoint(InetAddress ep)
{
- Token right = tokenMetadata_.getToken(ep);
- return replicationStrategy_.getPrimaryRangeFor(right,
tokenMetadata_.cloneTokenEndPointMap());
+ return tokenMetadata_.getPrimaryRangeFor(tokenMetadata_.getToken(ep));
}
/**
@@ -683,22 +659,23 @@
* ranges.
* @return ranges in sorted order
*/
- public Range[] getAllRanges(Set<Token> tokens)
+ public List<Range> getAllRanges(List<Token> sortedTokens)
{
if (logger_.isDebugEnabled())
- logger_.debug("computing ranges for " + StringUtils.join(tokens,
", "));
+ logger_.debug("computing ranges for " +
StringUtils.join(sortedTokens, ", "));
+
List<Range> ranges = new ArrayList<Range>();
- List<Token> allTokens = new ArrayList<Token>(tokens);
- Collections.sort(allTokens);
- int size = allTokens.size();
- for ( int i = 1; i < size; ++i )
+ Collections.sort(sortedTokens);
+ int size = sortedTokens.size();
+ for (int i = 1; i < size; ++i)
{
- Range range = new Range( allTokens.get(i - 1), allTokens.get(i) );
+ Range range = new Range(sortedTokens.get(i - 1),
sortedTokens.get(i));
ranges.add(range);
}
- Range range = new Range( allTokens.get(size - 1), allTokens.get(0) );
+ Range range = new Range(sortedTokens.get(size - 1),
sortedTokens.get(0));
ranges.add(range);
- return ranges.toArray( new Range[0] );
+
+ return ranges;
}
/**
@@ -712,11 +689,9 @@
{
InetAddress endpoint = FBUtilities.getLocalAddress();
Token token = partitioner_.getToken(key);
- Map<Token, InetAddress> tokenToEndPointMap =
tokenMetadata_.cloneTokenEndPointMap();
- List tokens = new ArrayList<Token>(tokenToEndPointMap.keySet());
+ List tokens = new ArrayList<Token>(tokenMetadata_.sortedTokens());
if (tokens.size() > 0)
{
- Collections.sort(tokens);
int index = Collections.binarySearch(tokens, token);
if (index >= 0)
{
@@ -724,15 +699,15 @@
* retrieve the endpoint based on the token at this index in
the
* tokens list
*/
- endpoint = tokenToEndPointMap.get(tokens.get(index));
+ endpoint = tokenMetadata_.getEndPoint((Token)
tokens.get(index));
}
else
{
index = (index + 1) * (-1);
if (index < tokens.size())
- endpoint = tokenToEndPointMap.get(tokens.get(index));
+ endpoint = tokenMetadata_.getEndPoint((Token)
tokens.get(index));
else
- endpoint = tokenToEndPointMap.get(tokens.get(0));
+ endpoint = tokenMetadata_.getEndPoint((Token)
tokens.get(0));
}
}
return endpoint;
@@ -834,9 +809,14 @@
throw new UnavailableException(); // no nodes that could contain key
are alive
}
- Map<Token, InetAddress> getLiveEndPointMap()
+ Map<String, String> getStringEndpointMap()
{
- return tokenMetadata_.cloneTokenEndPointMap();
+ HashMap<String, String> map = new HashMap<String, String>();
+ for (Token t : tokenMetadata_.sortedTokens())
+ {
+ map.put(t.toString(),
tokenMetadata_.getEndPoint(t).getHostAddress());
+ }
+ return map;
}
public void setLog4jLevel(String classQualifier, String rawLevel)
Modified:
incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java?rev=832267&r1=832266&r2=832267&view=diff
==============================================================================
---
incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
(original)
+++
incubator/cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
Tue Nov 3 02:24:05 2009
@@ -54,7 +54,7 @@
InetAddress myEndpoint = InetAddress.getByName("127.0.0.1");
TokenMetadata tmd = ss.getTokenMetadata();
- assertEquals(numOldNodes, tmd.cloneTokenEndPointMap().size());
+ assertEquals(numOldNodes, tmd.sortedTokens().size());
BootStrapper b = new BootStrapper(ss.getReplicationStrategy(),
myEndpoint, myToken, tmd);
Multimap<Range, InetAddress> res = b.getRangesWithSources();