cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From tylerho...@apache.org
Subject [1/2] git commit: Fix failure to stream ranges when moving
Date Wed, 27 Aug 2014 16:03:28 GMT
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 44a9d8cc6 -> 4327c29be


Fix failure to stream ranges when moving

Patch by Tyler Hobbs; reviewed by Yuki Morishita for CASSANDRA-7836


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

Branch: refs/heads/cassandra-2.1
Commit: 3b3afb35cd20636c937ab953f497ee7193ab1ffc
Parents: ce3fcbe
Author: Tyler Hobbs <tyler@datastax.com>
Authored: Wed Aug 27 11:02:24 2014 -0500
Committer: Tyler Hobbs <tyler@datastax.com>
Committed: Wed Aug 27 11:02:24 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                               |  1 +
 src/java/org/apache/cassandra/service/StorageService.java | 10 ++++++++++
 src/java/org/apache/cassandra/streaming/StreamPlan.java   |  2 +-
 3 files changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/3b3afb35/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9c75426..0b45cef 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.0
+ * Fix failure to stream ranges when moving (CASSANDRA-7836)
  * Correctly remove tmplink files (CASSANDRA-7803)
  * (cqlsh) Fix column name formatting for functions, CAS operations,
    and UDT field selections (CASSANDRA-7806)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3b3afb35/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 b4aa5ba..735bf7b 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -3179,6 +3179,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
IE
 
             for (String keyspace : keyspaceNames)
             {
+                logger.debug("Calculating ranges to stream and request for keyspace {}",
keyspace);
                 for (Token newToken : newTokens)
                 {
                     // replication strategy of the current keyspace (aka table)
@@ -3263,17 +3264,26 @@ public class StorageService extends NotificationBroadcasterSupport
implements IE
                         Set<InetAddress> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(toStream.right,
tokenMetaCloneAllSettled));
                         logger.debug("Range: {} Current endpoints: {} New endpoints: {}",
toStream, currentEndpoints, newEndpoints);
                         for (InetAddress address : Sets.difference(newEndpoints, currentEndpoints))
+                        {
+                            logger.debug("Range {} has new owner {}", toStream, address);
                             endpointRanges.put(address, toStream);
+                        }
                     }
 
                     // stream ranges
                     for (InetAddress address : endpointRanges.keySet())
+                    {
+                        logger.debug("Will stream range {} of keyspace {} to endpoint {}",
endpointRanges.get(address), keyspace, address);
                         streamPlan.transferRanges(address, keyspace, endpointRanges.get(address));
+                    }
 
                     // stream requests
                     Multimap<InetAddress, Range<Token>> workMap = RangeStreamer.getWorkMap(rangesToFetchWithPreferredEndpoints);
                     for (InetAddress address : workMap.keySet())
+                    {
+                        logger.debug("Will request range {} of keyspace {} from endpoint
{}", workMap.get(address), keyspace, address);
                         streamPlan.requestRanges(address, keyspace, workMap.get(address));
+                    }
 
                     if (logger.isDebugEnabled())
                         logger.debug("Keyspace {}: work map {}.", keyspace, workMap);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3b3afb35/src/java/org/apache/cassandra/streaming/StreamPlan.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamPlan.java b/src/java/org/apache/cassandra/streaming/StreamPlan.java
index e775c90..feb116f 100644
--- a/src/java/org/apache/cassandra/streaming/StreamPlan.java
+++ b/src/java/org/apache/cassandra/streaming/StreamPlan.java
@@ -143,7 +143,7 @@ public class StreamPlan
      */
     public boolean isEmpty()
     {
-        return coordinator.hasActiveSessions();
+        return !coordinator.hasActiveSessions();
     }
 
     /**


Mime
View raw message