You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jz...@apache.org on 2018/12/31 06:14:17 UTC

cassandra git commit: Do not enable native transport if bootstrap is pending

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.2 505a03c77 -> a6196a3a7


Do not enable native transport if bootstrap is pending

patch by Jaydeepkumar Chovatia; reviewed by Kurt Greaves for CASSANDRA-14525


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

Branch: refs/heads/cassandra-2.2
Commit: a6196a3a79b67dc6577747e591456328e57c314f
Parents: 505a03c
Author: jaydeepkumar1984 <ch...@gmail.com>
Authored: Thu Jun 14 21:52:20 2018 -0700
Committer: Jay Zhuang <ja...@yahoo.com>
Committed: Sun Dec 30 21:35:08 2018 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/service/CassandraDaemon.java      | 24 +++++
 .../cassandra/service/StorageService.java       | 94 +++++++++++++++++---
 .../cassandra/service/StorageServiceMBean.java  |  7 ++
 .../org/apache/cassandra/tools/NodeProbe.java   |  5 ++
 .../apache/cassandra/tools/nodetool/Join.java   |  1 +
 6 files changed, 119 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a6196a3a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 58309fa..16695c8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@
 
 
 2.2.13
+ * Don't enable client transports when bootstrap is pending (CASSANDRA-14525)
  * Fix bug that prevented compaction of SSTables after full repairs (CASSANDRA-14423)
  * Incorrect counting of pending messages in OutboundTcpConnection (CASSANDRA-11551)
  * Fix compaction failure caused by reading un-flushed data (CASSANDRA-12743)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a6196a3a/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 18b568d..e250050 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -440,6 +440,30 @@ public class CassandraDaemon
      */
     public void start()
     {
+        // We only start transports if bootstrap has completed and we're not in survey mode, OR if we are in
+        // survey mode and streaming has completed but we're not using auth.
+        // OR if we have not joined the ring yet.
+        if (StorageService.instance.hasJoined())
+        {
+            if (StorageService.instance.isSurveyMode())
+            {
+                if (StorageService.instance.isBootstrapMode() || DatabaseDescriptor.getAuthenticator().requireAuthentication())
+                {
+                    logger.info("Not starting client transports in write_survey mode as it's bootstrapping or " +
+                            "auth is enabled");
+                    return;
+                }
+            }
+            else
+            {
+                if (!SystemKeyspace.bootstrapComplete())
+                {
+                    logger.info("Not starting client transports as bootstrap has not completed");
+                    return;
+                }
+            }
+        }
+
         String nativeFlag = System.getProperty("cassandra.start_native_transport");
         if ((nativeFlag != null && Boolean.parseBoolean(nativeFlag)) || (nativeFlag == null && DatabaseDescriptor.startNativeTransport()))
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a6196a3a/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 2c9ac4d..03470b6 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -156,7 +156,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     private volatile boolean isBootstrapMode;
 
     /* we bootstrap but do NOT join the ring unless told to do so */
-    private boolean isSurveyMode = Boolean.parseBoolean(System.getProperty("cassandra.write_survey", "false"));
+    private boolean isSurveyMode = Boolean.parseBoolean(System.getProperty
+            ("cassandra.write_survey", "false"));
     /* true if node is rebuilding and receiving data */
     private final AtomicBoolean isRebuilding = new AtomicBoolean();
 
@@ -193,6 +194,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     private final AtomicBoolean doneAuthSetup = new AtomicBoolean(false);
 
+    public boolean isSurveyMode()
+    {
+        return isSurveyMode;
+    }
+
+    public boolean hasJoined()
+    {
+        return joined;
+    }
+
     /** This method updates the local token on disk  */
     public void setTokens(Collection<Token> tokens)
     {
@@ -326,6 +337,30 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         {
             throw new IllegalStateException("No configured daemon");
         }
+
+        // We only start transports if bootstrap has completed and we're not in survey mode, OR if we are in
+        // survey mode and streaming has completed but we're not using auth.
+        // OR if we have not joined the ring yet.
+        if (StorageService.instance.hasJoined())
+        {
+            if (StorageService.instance.isSurveyMode())
+            {
+                if (StorageService.instance.isBootstrapMode() || DatabaseDescriptor.getAuthenticator().requireAuthentication())
+                {
+                    throw new IllegalStateException("Not starting RPC server in write_survey mode as " +
+                            "it's bootstrapping or auth is enabled");
+                }
+            }
+            else
+            {
+                if (!SystemKeyspace.bootstrapComplete())
+                {
+                    throw new IllegalStateException("Node is not yet bootstrapped completely. Use nodetool to check bootstrap" +
+                            " state and resume. For more, see `nodetool help bootstrap`");
+                }
+            }
+        }
+        
         daemon.thriftServer.start();
     }
 
@@ -350,6 +385,22 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public void startNativeTransport()
     {
+        // We only start transports if bootstrap has completed and we're not in survey mode, OR if we are in
+        // survey mode and streaming has completed but we're not using auth.
+        // OR if we have not joined the ring yet.
+        if (hasJoined() &&
+                ((!isSurveyMode() && !SystemKeyspace.bootstrapComplete()) ||
+                (isSurveyMode() && isBootstrapMode())))
+        {
+            throw new IllegalStateException("Node is not yet bootstrapped completely. Use nodetool to check bootstrap" +
+                    " state and resume. For more, see `nodetool help bootstrap`");
+        }
+        if (hasJoined() && isSurveyMode() && !SystemKeyspace.bootstrapComplete() &&
+                DatabaseDescriptor.getAuthenticator().requireAuthentication())
+        {
+            throw new IllegalStateException("Not starting client transports as write_survey mode is enabled");
+        }
+
         if (daemon == null)
         {
             throw new IllegalStateException("No configured daemon");
@@ -925,7 +976,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
         else
         {
-            logger.info("Startup complete, but write survey mode is active, not becoming an active ring member. Use JMX (StorageService->joinRing()) to finalize ring joining.");
+            if (dataAvailable)
+                logger.info("Startup complete, but write survey mode is active, not becoming an active ring member. Use JMX (StorageService->joinRing()) to finalize ring joining.");
+            else
+                logger.warn("Some data streaming failed. Use nodetool to check bootstrap state and resume. For more, see `nodetool help bootstrap`. {}", SystemKeyspace.getBootstrapState());
         }
     }
 
@@ -959,9 +1013,24 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
         else if (isSurveyMode)
         {
-            isSurveyMode = false;
-            logger.info("Leaving write survey mode and joining ring at operator request");
-            finishJoiningRing(SystemKeyspace.getSavedTokens());
+            // if isSurveyMode is on then verify isBootstrapMode
+            // node can join the ring even if isBootstrapMode is true which should not happen
+            if (!isBootstrapMode())
+            {
+                isSurveyMode = false;
+                logger.info("Leaving write survey mode and joining ring at operator request");
+                finishJoiningRing(SystemKeyspace.getSavedTokens());
+                daemon.start();
+            }
+            else
+            {
+                logger.warn("Can't join the ring because in write_survey mode and bootstrap hasn't completed");
+            }
+        }
+        else if (isBootstrapMode())
+        {
+            // bootstrap is not complete hence node cannot join the ring
+            logger.warn("Can't join the ring because bootstrap hasn't completed.");
         }
     }
 
@@ -1235,19 +1304,18 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 public void onSuccess(StreamState streamState)
                 {
                     isBootstrapMode = false;
-                    // start participating in the ring.
-                    // pretend we are in survey mode so we can use joinRing() here
-                    isSurveyMode = true;
-                    try
+                    if (isSurveyMode)
                     {
-                        progressSupport.progress("bootstrap", ProgressEvent.createNotification("Joining ring..."));
-                        joinRing();
+                        logger.info("Startup complete, but write survey mode is active, not becoming an active ring member. Use JMX (StorageService->joinRing()) to finalize ring joining.");
                     }
-                    catch (IOException ignore)
+                    else
                     {
-                        // joinRing with survey mode does not throw IOException
+                        isSurveyMode = false;
+                        progressSupport.progress("bootstrap", ProgressEvent.createNotification("Joining ring..."));
+                        finishJoiningRing(bootstrapTokens);
                     }
                     progressSupport.progress("bootstrap", new ProgressEvent(ProgressEventType.COMPLETE, 1, 1, "Resume bootstrap complete"));
+                    daemon.start();
                     logger.info("Resume complete");
                 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a6196a3a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index f336bcc..43d26c6 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -491,6 +491,13 @@ public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkDa
     public void joinRing() throws IOException;
     public boolean isJoined();
 
+    /** Check if currently bootstrapping.
+     * Note this becomes false before {@link org.apache.cassandra.db.SystemKeyspace#bootstrapComplete()} is called,
+     * as setting bootstrap to complete is called only when the node joins the ring.
+     * @return True prior to bootstrap streaming completing. False prior to start of bootstrap and post streaming.
+     */
+    public boolean isBootstrapMode();
+
     public void setStreamThroughputMbPerSec(int value);
     public int getStreamThroughputMbPerSec();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a6196a3a/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 17bef02..a022595 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -574,6 +574,11 @@ public class NodeProbe implements AutoCloseable
         return ssProxy.isJoined();
     }
 
+    public boolean isBootstrapMode()
+    {
+        return ssProxy.isBootstrapMode();
+    }
+
     public void joinRing() throws IOException
     {
         ssProxy.joinRing();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a6196a3a/src/java/org/apache/cassandra/tools/nodetool/Join.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Join.java b/src/java/org/apache/cassandra/tools/nodetool/Join.java
index 5815591..a4a7cad 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Join.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Join.java
@@ -32,6 +32,7 @@ public class Join extends NodeToolCmd
     public void execute(NodeProbe probe)
     {
         checkState(!probe.isJoined(), "This node has already joined the ring.");
+        checkState(!probe.isBootstrapMode(), "Cannot join the ring until bootstrap completes");
 
         try
         {


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