You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2012/02/13 22:45:03 UTC

[2/3] git commit: fix unsynchronized use of TokenMetadata.entrySet patch by Peter Schuller; reviewed by jbellis for CASSANDRA-3417

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/79050449
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/79050449
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/79050449

Branch: refs/heads/trunk
Commit: 79050449e7e953a301e275a755a2b5f3a5b0d06a
Parents: bcad068
Author: Jonathan Ellis <jb...@apache.org>
Authored: Mon Feb 13 15:31:43 2012 -0600
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Mon Feb 13 15:44:29 2012 -0600

----------------------------------------------------------------------
 .../cassandra/locator/NetworkTopologyStrategy.java |    2 +-
 .../apache/cassandra/locator/TokenMetadata.java    |   28 +++++++++++----
 .../apache/cassandra/service/StorageService.java   |    6 ++--
 3 files changed, 25 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/79050449/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 ffbabd6..382e224 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -90,7 +90,7 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
             // collect endpoints in this DC; add in bulk to token meta data for computational complexity
             // reasons (CASSANDRA-3831).
             Set<Pair<Token, InetAddress>> dcTokensToUpdate = new HashSet<Pair<Token, InetAddress>>();
-            for (Entry<Token, InetAddress> tokenEntry : tokenMetadata.entrySet())
+            for (Entry<Token, InetAddress> tokenEntry : tokenMetadata.getTokenToEndpointMapForReading().entrySet())
             {
                 if (snitch.getDatacenter(tokenEntry.getValue()).equals(dcName))
                     dcTokensToUpdate.add(Pair.create(tokenEntry.getKey(), tokenEntry.getValue()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/79050449/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 b02daae..4d89f92 100644
--- a/src/java/org/apache/cassandra/locator/TokenMetadata.java
+++ b/src/java/org/apache/cassandra/locator/TokenMetadata.java
@@ -436,11 +436,6 @@ public class TokenMetadata
         }
     }
 
-    public Set<Map.Entry<Token,InetAddress>> entrySet()
-    {
-        return tokenToEndpointMap.entrySet();
-    }
-
     public InetAddress getEndpoint(Token token)
     {
         lock.readLock().lock();
@@ -741,9 +736,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/79050449/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 c1681b9..9bcd54d 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -908,7 +908,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
     public Map<String, String> getTokenToEndpointMap()
     {
-        Map<Token, InetAddress> mapInetAddress = tokenMetadata_.getTokenToEndpointMap();
+        Map<Token, InetAddress> mapInetAddress = tokenMetadata_.getNormalAndBootstrappingTokenToEndpointMap();
         // in order to preserve tokens in ascending order, we use LinkedHashMap here
         Map<String, String> mapString = new LinkedHashMap<String, String>(mapInetAddress.size());
         List<Token> tokens = new ArrayList<Token>(mapInetAddress.keySet());
@@ -2102,7 +2102,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;
@@ -2605,7 +2605,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
     public Map<String, Float> getOwnership()
     {
-        List<Token> sortedTokens = new ArrayList<Token>(tokenMetadata_.getTokenToEndpointMap().keySet());
+        List<Token> sortedTokens = new ArrayList<Token>(tokenMetadata_.getTokenToEndpointMapForReading().keySet());
         Collections.sort(sortedTokens);
         Map<Token, Float> token_map = partitioner.describeOwnership(sortedTokens);
         Map<String, Float> string_map = new HashMap<String, Float>();