Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.11 95cfee623 -> 28ccf3fe3
  refs/heads/trunk 70d95359d -> 88b01c866


Don't fail startup if peers aren't live

Patch by Kurt Greaves; reviewed by Sam Tunnicliffe for CASSANDRA-13851


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

Branch: refs/heads/cassandra-3.11
Commit: 28ccf3fe3989d9d80063fe4d4bb048efe471936b
Parents: 95cfee6
Author: kurt <k...@instaclustr.com>
Authored: Wed Nov 1 08:58:54 2017 +0000
Committer: Sam Tunnicliffe <s...@beobal.com>
Committed: Tue Apr 17 17:59:48 2018 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 src/java/org/apache/cassandra/gms/Gossiper.java | 38 +++++++++++++++-----
 .../cassandra/service/StorageService.java       | 10 ++++--
 3 files changed, 38 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/28ccf3fe/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 42ea3b4..39213a1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.11.3
+ * Allow existing nodes to use all peers in shadow round (CASSANDRA-13851)
  * Fix cqlsh to read connection.ssl cqlshrc option again (CASSANDRA-14299)
  * Downgrade log level to trace for CommitLogSegmentManager (CASSANDRA-14370)
  * CQL fromJson(null) throws NullPointerException (CASSANDRA-13891)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/28ccf3fe/src/java/org/apache/cassandra/gms/Gossiper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java 
b/src/java/org/apache/cassandra/gms/Gossiper.java
index 2dac5c2..ea05525 100644
--- a/src/java/org/apache/cassandra/gms/Gossiper.java
+++ b/src/java/org/apache/cassandra/gms/Gossiper.java
@@ -1361,6 +1361,11 @@ public class Gossiper implements 
IFailureDetectionEventListener, GossiperMBean
                                                               
TimeUnit.MILLISECONDS);
     }
 
+    public synchronized Map<InetAddress, EndpointState> doShadowRound()
+    {
+        return doShadowRound(Collections.EMPTY_SET);
+    }
+
     /**
      * Do a single 'shadow' round of gossip by retrieving endpoint states that 
will be stored exclusively in the
      * map return value, instead of endpointStateMap.
@@ -1376,16 +1381,21 @@ public class Gossiper implements 
IFailureDetectionEventListener, GossiperMBean
      * caller of {@link Gossiper#doShadowRound()}. Therefor only a single 
shadow round execution is permitted at
      * the same time.
      *
+     * @param peers Additional peers to try gossiping with.
      * @return endpoint states gathered during shadow round or empty map
      */
-    public synchronized Map<InetAddress, EndpointState> doShadowRound()
+    public synchronized Map<InetAddress, EndpointState> 
doShadowRound(Set<InetAddress> peers)
     {
         buildSeedsList();
-        // it may be that the local address is the only entry in the seed
+        // it may be that the local address is the only entry in the seed + 
peers
         // list in which case, attempting a shadow round is pointless
-        if (seeds.isEmpty())
+        if (seeds.isEmpty() && peers.isEmpty())
             return endpointShadowStateMap;
 
+        boolean isSeed = 
DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress());
+        // We double RING_DELAY if we're not a seed to increase chance of 
successful startup during a full cluster bounce,
+        // giving the seeds a chance to startup before we fail the shadow round
+        int shadowRoundDelay =  isSeed ? StorageService.RING_DELAY : 
StorageService.RING_DELAY * 2;
         seedsInShadowRound.clear();
         endpointShadowStateMap.clear();
         // send a completely empty syn
@@ -1398,6 +1408,7 @@ public class Gossiper implements 
IFailureDetectionEventListener, GossiperMBean
                 GossipDigestSyn.serializer);
 
         inShadowRound = true;
+        boolean includePeers = false;
         int slept = 0;
         try
         {
@@ -1409,6 +1420,15 @@ public class Gossiper implements 
IFailureDetectionEventListener, GossiperMBean
 
                     for (InetAddress seed : seeds)
                         MessagingService.instance().sendOneWay(message, seed);
+
+                    // Send to any peers we already know about, but only if a 
seed didn't respond.
+                    if (includePeers)
+                    {
+                        logger.trace("Sending shadow round GOSSIP DIGEST SYN 
to known peers {}", peers);
+                        for (InetAddress peer : peers)
+                            MessagingService.instance().sendOneWay(message, 
peer);
+                    }
+                    includePeers = true;
                 }
 
                 Thread.sleep(1000);
@@ -1416,13 +1436,12 @@ public class Gossiper implements 
IFailureDetectionEventListener, GossiperMBean
                     break;
 
                 slept += 1000;
-                if (slept > StorageService.RING_DELAY)
+                if (slept > shadowRoundDelay)
                 {
-                    // if we don't consider ourself to be a seed, fail out
-                    if 
(!DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress()))
-                        throw new RuntimeException("Unable to gossip with any 
seeds");
+                    // if we got here no peers could be gossiped to. If we're 
a seed that's OK, but otherwise we stop. See CASSANDRA-13851
+                    if (!isSeed)
+                        throw new RuntimeException("Unable to gossip with any 
peers");
 
-                    logger.warn("Unable to gossip with any seeds but 
continuing since node is in its own seed list");
                     inShadowRound = false;
                     break;
                 }
@@ -1566,6 +1585,9 @@ public class Gossiper implements 
IFailureDetectionEventListener, GossiperMBean
         {
             if (!isInShadowRound)
             {
+                if (!seeds.contains(respondent))
+                    logger.warn("Received an ack from {}, who isn't a seed. 
Ensure your seed list includes a live node. Exiting shadow round",
+                                respondent);
                 logger.debug("Received a regular ack from {}, can now exit 
shadow round", respondent);
                 // respondent sent back a full ack, so we can exit our shadow 
round
                 endpointShadowStateMap.putAll(epStateMap);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/28ccf3fe/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 6faaa04..d262807 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -535,7 +535,7 @@ public class StorageService extends 
NotificationBroadcasterSupport implements IE
         return localHostId;
     }
 
-    private synchronized void checkForEndpointCollision(UUID localHostId) 
throws ConfigurationException
+    private synchronized void checkForEndpointCollision(UUID localHostId, 
Set<InetAddress> peers) throws ConfigurationException
     {
         if (Boolean.getBoolean("cassandra.allow_unsafe_join"))
         {
@@ -544,7 +544,11 @@ public class StorageService extends 
NotificationBroadcasterSupport implements IE
         }
 
         logger.debug("Starting shadow gossip round to check for endpoint 
collision");
-        Map<InetAddress, EndpointState> epStates = 
Gossiper.instance.doShadowRound();
+        Map<InetAddress, EndpointState> epStates = 
Gossiper.instance.doShadowRound(peers);
+
+        if (epStates.isEmpty() && 
DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress()))
+            logger.info("Unable to gossip with any peers but continuing anyway 
since node is in its own seed list");
+
         // If bootstrapping, check whether any previously known status for the 
endpoint makes it unsafe to do so.
         // If not bootstrapping, compare the host id for this endpoint learned 
from gossip (if any) with the local
         // one, which was either read from system.local or generated at 
startup. If a learned id is present &
@@ -797,7 +801,7 @@ public class StorageService extends 
NotificationBroadcasterSupport implements IE
             }
             else
             {
-                checkForEndpointCollision(localHostId);
+                checkForEndpointCollision(localHostId, 
SystemKeyspace.loadHostIds().keySet());
             }
 
             // have to start the gossip service before we can see any info on 
other nodes.  this is necessary


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to