You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by br...@apache.org on 2013/11/04 19:58:15 UTC

[3/6] git commit: make calculatePendingRanges asynchronous

make calculatePendingRanges asynchronous


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/46a26ed5
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/46a26ed5
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/46a26ed5

Branch: refs/heads/trunk
Commit: 46a26ed553b3ea9af9658d7e7d9ea1cdf598abf3
Parents: e768191
Author: Brandon Williams <br...@apache.org>
Authored: Mon Nov 4 12:46:19 2013 -0600
Committer: Brandon Williams <br...@apache.org>
Committed: Mon Nov 4 12:46:19 2013 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/service/StorageService.java       | 147 +++----------------
 .../cassandra/locator/SimpleStrategyTest.java   |   3 +-
 3 files changed, 21 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/46a26ed5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index fd93a5f..de416e2 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@
  * Require logging in for Thrift CQL2/3 statement preparation (CASSANDRA-6254)
  * restrict max_num_tokens to 1536 (CASSANDRA-6267)
  * Nodetool gets default JMX port from cassandra-env.sh (CASSANDRA-6273)
+ * make calculatePendingRanges asynchronous (CASSANDRA-6244)
 
 
 1.2.11

http://git-wip-us.apache.org/repos/asf/cassandra/blob/46a26ed5/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 96c2dd9..cd98689 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -689,6 +689,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 }
             }
             setMode(Mode.JOINING, "schema complete, ready to bootstrap", true);
+            setMode(Mode.JOINING, "waiting for pending range calculation", true);
+            PendingRangeCalculatorService.instance.blockUntilFinished();
+            setMode(Mode.JOINING, "calculation complete, ready to bootstrap", true);
 
 
             if (logger.isDebugEnabled())
@@ -1348,7 +1351,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
 
         tokenMetadata.addBootstrapTokens(tokens, endpoint);
-        calculatePendingRanges();
+        PendingRangeCalculatorService.instance.update();
 
         if (Gossiper.instance.usesHostId(endpoint))
             tokenMetadata.updateHostId(Gossiper.instance.getHostId(endpoint), endpoint);
@@ -1490,7 +1493,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             }
         }
 
-        calculatePendingRanges();
+        PendingRangeCalculatorService.instance.update();
     }
 
     /**
@@ -1525,7 +1528,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         // at this point the endpoint is certainly a member with this token, so let's proceed
         // normally
         tokenMetadata.addLeavingEndpoint(endpoint);
-        calculatePendingRanges();
+        PendingRangeCalculatorService.instance.update();
     }
 
     /**
@@ -1563,7 +1566,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
         tokenMetadata.addMovingEndpoint(token, endpoint);
 
-        calculatePendingRanges();
+        PendingRangeCalculatorService.instance.update();
     }
 
     /**
@@ -1583,7 +1586,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         logger.debug("Tokens {} are relocating to {}", tokens, endpoint);
         tokenMetadata.addRelocatingTokens(tokens, endpoint);
 
-        calculatePendingRanges();
+        PendingRangeCalculatorService.instance.update();
     }
 
     /**
@@ -1625,7 +1628,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
                 // Note that the endpoint is being removed
                 tokenMetadata.addLeavingEndpoint(endpoint);
-                calculatePendingRanges();
+                PendingRangeCalculatorService.instance.update();
 
                 // find the endpoint coordinating this removal that we need to notify when we're done
                 String[] coordinator = Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.REMOVAL_COORDINATOR).value.split(VersionedValue.DELIMITER_STR, -1);
@@ -1648,12 +1651,13 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         removeEndpoint(endpoint);
         tokenMetadata.removeEndpoint(endpoint);
         tokenMetadata.removeBootstrapTokens(tokens);
+
         if (!isClientMode)
         {
             for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers)
                 subscriber.onLeaveCluster(endpoint);
         }
-        calculatePendingRanges();
+        PendingRangeCalculatorService.instance.update();
     }
 
     private void excise(Collection<Token> tokens, InetAddress endpoint, long expireTime)
@@ -1693,124 +1697,6 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
     }
 
-    /**
-     * Calculate pending ranges according to bootsrapping and leaving nodes. Reasoning is:
-     *
-     * (1) When in doubt, it is better to write too much to a node than too little. That is, if
-     * there are multiple nodes moving, calculate the biggest ranges a node could have. Cleaning
-     * up unneeded data afterwards is better than missing writes during movement.
-     * (2) When a node leaves, ranges for other nodes can only grow (a node might get additional
-     * ranges, but it will not lose any of its current ranges as a result of a leave). Therefore
-     * we will first remove _all_ leaving tokens for the sake of calculation and then check what
-     * ranges would go where if all nodes are to leave. This way we get the biggest possible
-     * ranges with regard current leave operations, covering all subsets of possible final range
-     * values.
-     * (3) When a node bootstraps, ranges of other nodes can only get smaller. Without doing
-     * complex calculations to see if multiple bootstraps overlap, we simply base calculations
-     * on the same token ring used before (reflecting situation after all leave operations have
-     * completed). Bootstrapping nodes will be added and removed one by one to that metadata and
-     * checked what their ranges would be. This will give us the biggest possible ranges the
-     * node could have. It might be that other bootstraps make our actual final ranges smaller,
-     * but it does not matter as we can clean up the data afterwards.
-     *
-     * NOTE: This is heavy and ineffective operation. This will be done only once when a node
-     * changes state in the cluster, so it should be manageable.
-     */
-    private void calculatePendingRanges()
-    {
-        for (String table : Schema.instance.getNonSystemTables())
-            calculatePendingRanges(Table.open(table).getReplicationStrategy(), table);
-    }
-
-    // public & static for testing purposes
-    public static void calculatePendingRanges(AbstractReplicationStrategy strategy, String table)
-    {
-        TokenMetadata tm = StorageService.instance.getTokenMetadata();
-        Multimap<Range<Token>, InetAddress> pendingRanges = HashMultimap.create();
-        BiMultiValMap<Token, InetAddress> bootstrapTokens = tm.getBootstrapTokens();
-        Set<InetAddress> leavingEndpoints = tm.getLeavingEndpoints();
-
-        if (bootstrapTokens.isEmpty() && leavingEndpoints.isEmpty() && tm.getMovingEndpoints().isEmpty() && tm.getRelocatingRanges().isEmpty())
-        {
-            if (logger.isDebugEnabled())
-                logger.debug("No bootstrapping, leaving or moving nodes, and no relocating tokens -> empty pending ranges for {}", table);
-            tm.setPendingRanges(table, pendingRanges);
-            return;
-        }
-
-        Multimap<InetAddress, Range<Token>> addressRanges = strategy.getAddressRanges();
-
-        // Copy of metadata reflecting the situation after all leave operations are finished.
-        TokenMetadata allLeftMetadata = tm.cloneAfterAllLeft();
-
-        // get all ranges that will be affected by leaving nodes
-        Set<Range<Token>> affectedRanges = new HashSet<Range<Token>>();
-        for (InetAddress endpoint : leavingEndpoints)
-            affectedRanges.addAll(addressRanges.get(endpoint));
-
-        // for each of those ranges, find what new nodes will be responsible for the range when
-        // all leaving nodes are gone.
-        for (Range<Token> range : affectedRanges)
-        {
-            Set<InetAddress> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, tm.cloneOnlyTokenMap()));
-            Set<InetAddress> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, allLeftMetadata));
-            pendingRanges.putAll(range, Sets.difference(newEndpoints, currentEndpoints));
-        }
-
-        // At this stage pendingRanges has been updated according to leave operations. We can
-        // now continue the calculation by checking bootstrapping nodes.
-
-        // For each of the bootstrapping nodes, simply add and remove them one by one to
-        // allLeftMetadata and check in between what their ranges would be.
-        for (InetAddress endpoint : bootstrapTokens.inverse().keySet())
-        {
-            Collection<Token> tokens = bootstrapTokens.inverse().get(endpoint);
-
-            allLeftMetadata.updateNormalTokens(tokens, endpoint);
-            for (Range<Token> range : strategy.getAddressRanges(allLeftMetadata).get(endpoint))
-                pendingRanges.put(range, endpoint);
-            allLeftMetadata.removeEndpoint(endpoint);
-        }
-
-        // At this stage pendingRanges has been updated according to leaving and bootstrapping nodes.
-        // We can now finish the calculation by checking moving and relocating nodes.
-
-        // For each of the moving nodes, we do the same thing we did for bootstrapping:
-        // simply add and remove them one by one to allLeftMetadata and check in between what their ranges would be.
-        for (Pair<Token, InetAddress> moving : tm.getMovingEndpoints())
-        {
-            InetAddress endpoint = moving.right; // address of the moving node
-
-            //  moving.left is a new token of the endpoint
-            allLeftMetadata.updateNormalToken(moving.left, endpoint);
-
-            for (Range<Token> range : strategy.getAddressRanges(allLeftMetadata).get(endpoint))
-            {
-                pendingRanges.put(range, endpoint);
-            }
-
-            allLeftMetadata.removeEndpoint(endpoint);
-        }
-
-        // Ranges being relocated.
-        for (Map.Entry<Token, InetAddress> relocating : tm.getRelocatingRanges().entrySet())
-        {
-            InetAddress endpoint = relocating.getValue(); // address of the moving node
-            Token token = relocating.getKey();
-
-            allLeftMetadata.updateNormalToken(token, endpoint);
-
-            for (Range<Token> range : strategy.getAddressRanges(allLeftMetadata).get(endpoint))
-                pendingRanges.put(range, endpoint);
-
-            allLeftMetadata.removeEndpoint(endpoint);
-        }
-
-        tm.setPendingRanges(table, pendingRanges);
-
-        if (logger.isDebugEnabled())
-            logger.debug("Pending ranges:\n" + (pendingRanges.isEmpty() ? "<empty>" : tm.printPendingRanges()));
-    }
 
     /**
      * Finds living endpoints responsible for the given ranges
@@ -2012,7 +1898,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     public void onRemove(InetAddress endpoint)
     {
         tokenMetadata.removeEndpoint(endpoint);
-        calculatePendingRanges();
+        PendingRangeCalculatorService.instance.update();
     }
 
     public void onDead(InetAddress endpoint, EndpointState state)
@@ -2802,7 +2688,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.leaving(getLocalTokens()));
         tokenMetadata.addLeavingEndpoint(FBUtilities.getBroadcastAddress());
-        calculatePendingRanges();
+        PendingRangeCalculatorService.instance.update();
     }
 
     public void decommission() throws InterruptedException
@@ -2811,6 +2697,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             throw new UnsupportedOperationException("local node is not a member of the token ring yet");
         if (tokenMetadata.cloneAfterAllLeft().sortedTokens().size() < 2)
             throw new UnsupportedOperationException("no other normal nodes in the ring; decommission would be pointless");
+        PendingRangeCalculatorService.instance.blockUntilFinished();
         for (String table : Schema.instance.getNonSystemTables())
         {
             if (tokenMetadata.getPendingRanges(table, FBUtilities.getBroadcastAddress()).size() > 0)
@@ -2842,7 +2729,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         SystemTable.setBootstrapState(SystemTable.BootstrapState.NEEDS_BOOTSTRAP);
         tokenMetadata.removeEndpoint(FBUtilities.getBroadcastAddress());
-        calculatePendingRanges();
+        PendingRangeCalculatorService.instance.update();
 
         Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.left(getLocalTokens(),Gossiper.computeExpireTime()));
         int delay = Math.max(RING_DELAY, Gossiper.intervalInMillis * 2);
@@ -2973,6 +2860,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
         List<String> tablesToProcess = Schema.instance.getNonSystemTables();
 
+        PendingRangeCalculatorService.instance.blockUntilFinished();
         // checking if data is moving to this node
         for (String table : tablesToProcess)
         {
@@ -3309,7 +3197,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         removingNode = endpoint;
 
         tokenMetadata.addLeavingEndpoint(endpoint);
-        calculatePendingRanges();
+        PendingRangeCalculatorService.instance.update();
+
         // the gossiper will handle spoofing this node's state to REMOVING_TOKEN for us
         // we add our own token so other nodes to let us know when they're done
         Gossiper.instance.advertiseRemoving(endpoint, hostId, localHostId);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/46a26ed5/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
index a457df8..95edc03 100644
--- a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
@@ -27,6 +27,7 @@ import java.util.HashSet;
 import java.util.List;
 
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.service.PendingRangeCalculatorService;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
@@ -139,7 +140,7 @@ public class SimpleStrategyTest extends SchemaLoader
         {
             strategy = getStrategy(table, tmd);
 
-            StorageService.calculatePendingRanges(strategy, table);
+            PendingRangeCalculatorService.calculatePendingRanges(strategy, table);
 
             int replicationFactor = strategy.getReplicationFactor();