This is an automated email from the ASF dual-hosted git repository.

brandonwilliams pushed a commit to branch cassandra-3.0
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/cassandra-3.0 by this push:
     new c94ecec  Check for endpoint collision with hibernating nodes
c94ecec is described below

commit c94ececec0fcd87459858370396d6cd586853787
Author: Stefan Miklosovic <stefan.mikloso...@instaclustr.com>
AuthorDate: Mon Aug 3 08:37:00 2020 +0200

    Check for endpoint collision with hibernating nodes
    
    Patch by Stefan Miklosovic, reviewed by brandonwillias for
    CASSANDRA-14599
---
 CHANGES.txt                                     |  1 +
 src/java/org/apache/cassandra/gms/Gossiper.java | 15 +++++++++++++--
 2 files changed, 14 insertions(+), 2 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 182dca3..7d4b7a9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.22:
+ * Check for endpoint collision with hibernating nodes (CASSANDRA-14599)
  * Operational improvements and hardening for replica filtering protection 
(CASSANDRA-15907)
  * stop_paranoid disk failure policy is ignored on CorruptSSTableException 
after node is up (CASSANDRA-15191)
  * 3.x fails to start if commit log has range tombstones from a column which 
is also deleted (CASSANDRA-15970)
diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java 
b/src/java/org/apache/cassandra/gms/Gossiper.java
index 7984dd4..7d708c1 100644
--- a/src/java/org/apache/cassandra/gms/Gossiper.java
+++ b/src/java/org/apache/cassandra/gms/Gossiper.java
@@ -33,6 +33,7 @@ import com.google.common.util.concurrent.ListenableFutureTask;
 import com.google.common.util.concurrent.Uninterruptibles;
 
 import io.netty.util.concurrent.FastThreadLocal;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.utils.ExecutorUtils;
 import org.apache.cassandra.utils.MBeanWrapper;
 import org.apache.cassandra.utils.NoSpamLogger;
@@ -794,12 +795,22 @@ public class Gossiper implements 
IFailureDetectionEventListener, GossiperMBean
     {
         EndpointState epState = epStates.get(endpoint);
 
-        // if there's no previous state, or the node was previously removed 
from the cluster, we're good
-        if (epState == null || isDeadState(epState))
+        // if there's no previous state, we're good
+        if (epState == null)
             return true;
 
         String status = getGossipStatus(epState);
 
+        if (status.equals(VersionedValue.HIBERNATE)
+            && !SystemKeyspace.bootstrapComplete())
+        {
+            logger.warn("A node with the same IP in hibernate status was 
detected. Was a replacement already attempted?");
+            return false;
+        }
+
+        if (isDeadState(epState))
+            return true;
+
         // these states are not allowed to join the cluster as it would not be 
safe
         final List<String> unsafeStatuses = new ArrayList<String>() {{
             add(""); // failed bootstrap but we did start gossiping


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

Reply via email to