fix unsynchronized use of TokenMetadata.entrySet patch by Peter Schuller; reviewed by jbellis for CASSANDRA-3417
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/4ab6fad9 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/4ab6fad9 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/4ab6fad9 Branch: refs/heads/trunk Commit: 4ab6fad945cada90497a8cf523a4c868932834c2 Parents: 1047914 Author: Jonathan Ellis <jbel...@apache.org> Authored: Mon Feb 13 15:31:43 2012 -0600 Committer: Jonathan Ellis <jbel...@apache.org> Committed: Mon Feb 13 15:44:50 2012 -0600 ---------------------------------------------------------------------- .../cassandra/locator/NetworkTopologyStrategy.java | 2 +- .../apache/cassandra/locator/TokenMetadata.java | 28 +++++++++++---- .../apache/cassandra/service/StorageService.java | 4 +- 3 files changed, 24 insertions(+), 10 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ab6fad9/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java index 2ae0a98..b6a99b2 100644 --- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java +++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java @@ -88,7 +88,7 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy // collect endpoints in this DC TokenMetadata dcTokens = new TokenMetadata(); - for (Entry<Token, InetAddress> tokenEntry : tokenMetadata.entrySet()) + for (Entry<Token, InetAddress> tokenEntry : tokenMetadata.getTokenToEndpointMapForReading().entrySet()) { if (snitch.getDatacenter(tokenEntry.getValue()).equals(dcName)) dcTokens.updateNormalToken(tokenEntry.getKey(), tokenEntry.getValue()); http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ab6fad9/src/java/org/apache/cassandra/locator/TokenMetadata.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/locator/TokenMetadata.java b/src/java/org/apache/cassandra/locator/TokenMetadata.java index ebb094b..0942a5d 100644 --- a/src/java/org/apache/cassandra/locator/TokenMetadata.java +++ b/src/java/org/apache/cassandra/locator/TokenMetadata.java @@ -408,11 +408,6 @@ public class TokenMetadata } } - public Set<Map.Entry<Token,InetAddress>> entrySet() - { - return tokenToEndpointMap.entrySet(); - } - public InetAddress getEndpoint(Token token) { lock.readLock().lock(); @@ -713,9 +708,28 @@ public class TokenMetadata } /** - * Return the Token to Endpoint map for all the node in the cluster, including bootstrapping ones. + * @return a token to endpoint map to consider for read operations on the cluster. + */ + public Map<Token, InetAddress> getTokenToEndpointMapForReading() + { + lock.readLock().lock(); + try + { + Map<Token, InetAddress> map = new HashMap<Token, InetAddress>(tokenToEndpointMap.size()); + map.putAll(tokenToEndpointMap); + return map; + } + finally + { + lock.readLock().unlock(); + } + } + + /** + * @return a (stable copy, won't be modified) Token to Endpoint map for all the normal and bootstrapping nodes + * in the cluster. */ - public Map<Token, InetAddress> getTokenToEndpointMap() + public Map<Token, InetAddress> getNormalAndBootstrappingTokenToEndpointMap() { lock.readLock().lock(); try http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ab6fad9/src/java/org/apache/cassandra/service/StorageService.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 1f7a18d..f82fe32 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -854,7 +854,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe public Map<Token, String> getTokenToEndpointMap() { - Map<Token, InetAddress> mapInetAddress = tokenMetadata_.getTokenToEndpointMap(); + Map<Token, InetAddress> mapInetAddress = tokenMetadata_.getNormalAndBootstrappingTokenToEndpointMap(); Map<Token, String> mapString = new HashMap<Token, String>(mapInetAddress.size()); for (Map.Entry<Token, InetAddress> entry : mapInetAddress.entrySet()) { @@ -2074,7 +2074,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe if (token instanceof StringToken) { token = new StringToken(((String)token.token).replaceAll(VersionedValue.DELIMITER_STR, "")); - if (tokenMetadata_.getTokenToEndpointMap().containsKey(token)) + if (tokenMetadata_.getNormalAndBootstrappingTokenToEndpointMap().containsKey(token)) throw new RuntimeException("Unable to compute unique token for new node -- specify one manually with initial_token"); } return token;