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:59 UTC

[1/4] cassandra git commit: Do not enable native transport if bootstrap is pending

Repository: cassandra
Updated Branches:
  refs/heads/trunk 11384c327 -> e7a603da4


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/trunk
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


[3/4] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by jz...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


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

Branch: refs/heads/trunk
Commit: 48a48c7fcd06213c77a9490f0d2c8b32baa2b1e4
Parents: 08363af e2ee204
Author: Jay Zhuang <ja...@yahoo.com>
Authored: Sun Dec 30 21:53:05 2018 -0800
Committer: Jay Zhuang <ja...@yahoo.com>
Committed: Sun Dec 30 21:54:56 2018 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/service/CassandraDaemon.java      | 47 ++++++++++++++
 .../cassandra/service/StorageService.java       | 66 +++++++++++++++++---
 .../cassandra/service/StorageServiceMBean.java  |  7 +++
 .../org/apache/cassandra/tools/NodeProbe.java   |  5 ++
 .../apache/cassandra/tools/nodetool/Join.java   |  1 +
 6 files changed, 117 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/48a48c7f/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 0d97d3c,d29cdc1..3d41d83
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -35,28 -33,10 +35,29 @@@ Merged from 3.0
   * Returns null instead of NaN or Infinity in JSON strings (CASSANDRA-14377)
  
  
 -3.0.17
 +3.11.3
 + * Validate supported column type with SASI analyzer (CASSANDRA-13669)
 + * Remove BTree.Builder Recycler to reduce memory usage (CASSANDRA-13929)
 + * Reduce nodetool GC thread count (CASSANDRA-14475)
 + * Fix New SASI view creation during Index Redistribution (CASSANDRA-14055)
 + * Remove string formatting lines from BufferPool hot path (CASSANDRA-14416)
 + * Update metrics to 3.1.5 (CASSANDRA-12924)
 + * Detect OpenJDK jvm type and architecture (CASSANDRA-12793)
 + * Don't use guava collections in the non-system keyspace jmx attributes (CASSANDRA-12271)
 + * 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)
 + * Serialize empty buffer as empty string for json output format (CASSANDRA-14245)
 + * Allow logging implementation to be interchanged for embedded testing (CASSANDRA-13396)
 + * SASI tokenizer for simple delimiter based entries (CASSANDRA-14247)
 + * Fix Loss of digits when doing CAST from varint/bigint to decimal (CASSANDRA-14170)
 + * RateBasedBackPressure unnecessarily invokes a lock on the Guava RateLimiter (CASSANDRA-14163)
 + * Fix wildcard GROUP BY queries (CASSANDRA-14209)
 +Merged from 3.0:
++ * Don't enable client transports when bootstrap is pending (CASSANDRA-14525)
   * Fix corrupted static collection deletions in 3.0 -> 2.{1,2} messages (CASSANDRA-14568)
   * Fix potential IndexOutOfBoundsException with counters (CASSANDRA-14167)
 - * Restore resumable hints delivery, backport CASSANDRA-11960 (CASSANDRA-14419)
   * Always close RT markers returned by ReadCommand#executeLocally() (CASSANDRA-14515)
   * Reverse order queries with range tombstones can cause data loss (CASSANDRA-14513)
   * Fix regression of lagging commitlog flush log message (CASSANDRA-14451)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/48a48c7f/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/CassandraDaemon.java
index d9bd5c3,6869d2c..fbafd35
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@@ -642,13 -633,31 +666,36 @@@ public class CassandraDaemo
          }
      }
  
 +    public void applyConfig()
 +    {
 +        DatabaseDescriptor.daemonInitialization();
 +    }
 +
      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 (StorageService.instance.hasJoined())
+         {
+             if (StorageService.instance.isSurveyMode())
+             {
+                 if (StorageService.instance.isBootstrapMode() || DatabaseDescriptor.getAuthenticator().requireAuthentication())
+                 {
+                     throw new IllegalStateException("Not starting client transports 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`");
+                 }
+             }
+         }
+ 
          if (nativeTransportService == null)
              throw new IllegalStateException("setup() must be called first for CassandraDaemon");
          else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/48a48c7f/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index edd8571,55bc9ae..b504995
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -361,6 -364,29 +371,23 @@@ public class StorageService extends Not
          {
              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.hasJoined() &&
++                ((!StorageService.instance.isSurveyMode() && !SystemKeyspace.bootstrapComplete()) ||
++                (StorageService.instance.isSurveyMode() && StorageService.instance.isBootstrapMode())))
+         {
 -            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`");
 -                }
 -            }
++            throw new IllegalStateException("Node is not yet bootstrapped completely. Use nodetool to check bootstrap state and resume. For more, see `nodetool help bootstrap`");
++        }
++        else if (StorageService.instance.hasJoined() && StorageService.instance.isSurveyMode() &&
++                DatabaseDescriptor.getAuthenticator().requireAuthentication())
++        {
++            // Auth isn't initialised until we join the ring, so if we're in survey mode auth will always fail.
++            throw new IllegalStateException("Not starting RPC server as write_survey mode and authentication is enabled");
+         }
+ 
          daemon.thriftServer.start();
      }
  
@@@ -1053,9 -1018,24 +1083,24 @@@
          }
          else if (isSurveyMode)
          {
-             logger.info("Leaving write survey mode and joining ring at operator request");
-             finishJoiningRing(resumedBootstrap, SystemKeyspace.getSavedTokens());
-             isSurveyMode = false;
+             // 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());
++                finishJoiningRing(resumedBootstrap, 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.");
          }
      }
  
@@@ -1574,19 -1338,18 +1619,20 @@@
                  public void onSuccess(StreamState streamState)
                  {
                      bootstrapFinished();
 +                    // 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(true);
+                         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);
++                        finishJoiningRing(true, 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/48a48c7f/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 5613c35,835da17..e8aad77
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -532,30 -495,13 +532,37 @@@ public interface StorageServiceMBean ex
      public boolean isDrained();
      public boolean isDraining();
  
+     /** 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 setRpcTimeout(long value);
 +    public long getRpcTimeout();
 +
 +    public void setReadRpcTimeout(long value);
 +    public long getReadRpcTimeout();
 +
 +    public void setRangeRpcTimeout(long value);
 +    public long getRangeRpcTimeout();
 +
 +    public void setWriteRpcTimeout(long value);
 +    public long getWriteRpcTimeout();
 +
 +    public void setCounterWriteRpcTimeout(long value);
 +    public long getCounterWriteRpcTimeout();
 +
 +    public void setCasContentionTimeout(long value);
 +    public long getCasContentionTimeout();
 +
 +    public void setTruncateRpcTimeout(long value);
 +    public long getTruncateRpcTimeout();
 +
 +    public void setStreamingSocketTimeout(int value);
 +    public int getStreamingSocketTimeout();
 +
      public void setStreamThroughputMbPerSec(int value);
      public int getStreamThroughputMbPerSec();
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/48a48c7f/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------


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


[4/4] cassandra git commit: Merge branch 'cassandra-3.11' into trunk

Posted by jz...@apache.org.
Merge branch 'cassandra-3.11' into trunk


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

Branch: refs/heads/trunk
Commit: e7a603da48800ec1a3383239d3ff7b7b133e9f35
Parents: 11384c3 48a48c7
Author: Jay Zhuang <ja...@yahoo.com>
Authored: Sun Dec 30 21:57:13 2018 -0800
Committer: Jay Zhuang <ja...@yahoo.com>
Committed: Sun Dec 30 22:01:07 2018 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/service/CassandraDaemon.java      | 47 +++++++++++++++++
 .../cassandra/service/StorageService.java       | 54 +++++++++++++++-----
 .../cassandra/service/StorageServiceMBean.java  |  6 +++
 .../org/apache/cassandra/tools/NodeProbe.java   |  5 ++
 .../apache/cassandra/tools/nodetool/Join.java   |  3 +-
 6 files changed, 102 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e7a603da/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 66442e9,3d41d83..4254ae2
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,342 -1,3 +1,343 @@@
 +4.0
++ * Don't enable client transports when bootstrap is pending (CASSANDRA-14525)
 + * Make antiCompactGroup throw exception on error and anticompaction non cancellable
 +   again (CASSANDRA-14936)
 + * Catch empty/invalid bounds in SelectStatement (CASSANDRA-14849)
 + * Auto-expand replication_factor for NetworkTopologyStrategy (CASSANDRA-14303)
 + * Transient Replication: support EACH_QUORUM (CASSANDRA-14727)
 + * BufferPool: allocating thread for new chunks should acquire directly (CASSANDRA-14832)
 + * Send correct messaging version in internode messaging handshake's third message (CASSANDRA-14896)
 + * Make Read and Write Latency columns consistent for proxyhistograms and tablehistograms (CASSANDRA-11939)
 + * Make protocol checksum type option case insensitive (CASSANDRA-14716)
 + * Forbid re-adding static columns as regular and vice versa (CASSANDRA-14913)
 + * Audit log allows system keyspaces to be audited via configuration options (CASSANDRA-14498)
 + * Lower default chunk_length_in_kb from 64kb to 16kb (CASSANDRA-13241)
 + * Startup checker should wait for count rather than percentage (CASSANDRA-14297)
 + * Fix incorrect sorting of replicas in SimpleStrategy.calculateNaturalReplicas (CASSANDRA-14862)
 + * Partitioned outbound internode TCP connections can occur when nodes restart (CASSANDRA-14358)
 + * Don't write to system_distributed.repair_history, system_traces.sessions, system_traces.events in mixed version 3.X/4.0 clusters (CASSANDRA-14841)
 + * Avoid running query to self through messaging service (CASSANDRA-14807)
 + * Allow using custom script for chronicle queue BinLog archival (CASSANDRA-14373)
 + * Transient->Full range movements mishandle consistency level upgrade (CASSANDRA-14759)
 + * ReplicaCollection follow-up (CASSANDRA-14726)
 + * Transient node receives full data requests (CASSANDRA-14762)
 + * Enable snapshot artifacts publish (CASSANDRA-12704)
 + * Introduce RangesAtEndpoint.unwrap to simplify StreamSession.addTransferRanges (CASSANDRA-14770)
 + * LOCAL_QUORUM may speculate to non-local nodes, resulting in Timeout instead of Unavailable (CASSANDRA-14735)
 + * Avoid creating empty compaction tasks after truncate (CASSANDRA-14780)
 + * Fail incremental repair prepare phase if it encounters sstables from un-finalized sessions (CASSANDRA-14763)
 + * Add a check for receiving digest response from transient node (CASSANDRA-14750)
 + * Fail query on transient replica if coordinator only expects full data (CASSANDRA-14704)
 + * Remove mentions of transient replication from repair path (CASSANDRA-14698)
 + * Fix handleRepairStatusChangedNotification to remove first then add (CASSANDRA-14720)
 + * Allow transient node to serve as a repair coordinator (CASSANDRA-14693)
 + * DecayingEstimatedHistogramReservoir.EstimatedHistogramReservoirSnapshot returns wrong value for size() and incorrectly calculates count (CASSANDRA-14696)
 + * AbstractReplicaCollection equals and hash code should throw due to conflict between order sensitive/insensitive uses (CASSANDRA-14700)
 + * Detect inconsistencies in repaired data on the read path (CASSANDRA-14145)
 + * Add checksumming to the native protocol (CASSANDRA-13304)
 + * Make AuthCache more easily extendable (CASSANDRA-14662)
 + * Extend RolesCache to include detailed role info (CASSANDRA-14497)
 + * Add fqltool compare (CASSANDRA-14619)
 + * Add fqltool replay (CASSANDRA-14618)
 + * Log keyspace in full query log (CASSANDRA-14656)
 + * Transient Replication and Cheap Quorums (CASSANDRA-14404)
 + * Log server-generated timestamp and nowInSeconds used by queries in FQL (CASSANDRA-14675)
 + * Add diagnostic events for read repairs (CASSANDRA-14668)
 + * Use consistent nowInSeconds and timestamps values within a request (CASSANDRA-14671)
 + * Add sampler for query time and expose with nodetool (CASSANDRA-14436)
 + * Clean up Message.Request implementations (CASSANDRA-14677)
 + * Disable old native protocol versions on demand (CASANDRA-14659)
 + * Allow specifying now-in-seconds in native protocol (CASSANDRA-14664)
 + * Improve BTree build performance by avoiding data copy (CASSANDRA-9989)
 + * Make monotonic read / read repair configurable (CASSANDRA-14635)
 + * Refactor CompactionStrategyManager (CASSANDRA-14621)
 + * Flush netty client messages immediately by default (CASSANDRA-13651)
 + * Improve read repair blocking behavior (CASSANDRA-10726)
 + * Add a virtual table to expose settings (CASSANDRA-14573)
 + * Fix up chunk cache handling of metrics (CASSANDRA-14628)
 + * Extend IAuthenticator to accept peer SSL certificates (CASSANDRA-14652)
 + * Incomplete handling of exceptions when decoding incoming messages (CASSANDRA-14574)
 + * Add diagnostic events for user audit logging (CASSANDRA-13668)
 + * Allow retrieving diagnostic events via JMX (CASSANDRA-14435)
 + * Add base classes for diagnostic events (CASSANDRA-13457)
 + * Clear view system metadata when dropping keyspace (CASSANDRA-14646)
 + * Allocate ReentrantLock on-demand in java11 AtomicBTreePartitionerBase (CASSANDRA-14637)
 + * Make all existing virtual tables use LocalPartitioner (CASSANDRA-14640)
 + * Revert 4.0 GC alg back to CMS (CASANDRA-14636)
 + * Remove hardcoded java11 jvm args in idea workspace files (CASSANDRA-14627)
 + * Update netty to 4.1.128 (CASSANDRA-14633)
 + * Add a virtual table to expose thread pools (CASSANDRA-14523)
 + * Add a virtual table to expose caches (CASSANDRA-14538, CASSANDRA-14626)
 + * Fix toDate function for timestamp arguments (CASSANDRA-14502)
 + * Revert running dtests by default in circleci (CASSANDRA-14614)
 + * Stream entire SSTables when possible (CASSANDRA-14556)
 + * Cell reconciliation should not depend on nowInSec (CASSANDRA-14592)
 + * Add experimental support for Java 11 (CASSANDRA-9608)
 + * Make PeriodicCommitLogService.blockWhenSyncLagsNanos configurable (CASSANDRA-14580)
 + * Improve logging in MessageInHandler's constructor (CASSANDRA-14576)
 + * Set broadcast address in internode messaging handshake (CASSANDRA-14579)
 + * Wait for schema agreement prior to building MVs (CASSANDRA-14571)
 + * Make all DDL statements idempotent and not dependent on global state (CASSANDRA-13426)
 + * Bump the hints messaging version to match the current one (CASSANDRA-14536)
 + * OffsetAwareConfigurationLoader doesn't set ssl storage port causing bind errors in CircleCI (CASSANDRA-14546)
 + * Report why native_transport_port fails to bind (CASSANDRA-14544)
 + * Optimize internode messaging protocol (CASSANDRA-14485)
 + * Internode messaging handshake sends wrong messaging version number (CASSANDRA-14540)
 + * Add a virtual table to expose active client connections (CASSANDRA-14458)
 + * Clean up and refactor client metrics (CASSANDRA-14524)
 + * Nodetool import row cache invalidation races with adding sstables to tracker (CASSANDRA-14529)
 + * Fix assertions in LWTs after TableMetadata was made immutable (CASSANDRA-14356)
 + * Abort compactions quicker (CASSANDRA-14397)
 + * Support light-weight transactions in cassandra-stress (CASSANDRA-13529)
 + * Make AsyncOneResponse use the correct timeout (CASSANDRA-14509)
 + * Add option to sanity check tombstones on reads/compactions (CASSANDRA-14467)
 + * Add a virtual table to expose all running sstable tasks (CASSANDRA-14457)
 + * Let nodetool import take a list of directories (CASSANDRA-14442)
 + * Avoid unneeded memory allocations / cpu for disabled log levels (CASSANDRA-14488)
 + * Implement virtual keyspace interface (CASSANDRA-7622)
 + * nodetool import cleanup and improvements (CASSANDRA-14417)
 + * Bump jackson version to >= 2.9.5 (CASSANDRA-14427)
 + * Allow nodetool toppartitions without specifying table (CASSANDRA-14360)
 + * Audit logging for database activity (CASSANDRA-12151)
 + * Clean up build artifacts in docs container (CASSANDRA-14432)
 + * Minor network authz improvements (Cassandra-14413)
 + * Automatic sstable upgrades (CASSANDRA-14197)
 + * Replace deprecated junit.framework.Assert usages with org.junit.Assert (CASSANDRA-14431)
 + * Cassandra-stress throws NPE if insert section isn't specified in user profile (CASSSANDRA-14426)
 + * List clients by protocol versions `nodetool clientstats --by-protocol` (CASSANDRA-14335)
 + * Improve LatencyMetrics performance by reducing write path processing (CASSANDRA-14281)
 + * Add network authz (CASSANDRA-13985)
 + * Use the correct IP/Port for Streaming when localAddress is left unbound (CASSANDRA-14389)
 + * nodetool listsnapshots is missing local system keyspace snapshots (CASSANDRA-14381)
 + * Remove StreamCoordinator.streamExecutor thread pool (CASSANDRA-14402)
 + * Rename nodetool --with-port to --print-port to disambiguate from --port (CASSANDRA-14392)
 + * Client TOPOLOGY_CHANGE messages have wrong port. (CASSANDRA-14398)
 + * Add ability to load new SSTables from a separate directory (CASSANDRA-6719)
 + * Eliminate background repair and probablistic read_repair_chance table options
 +   (CASSANDRA-13910)
 + * Bind to correct local address in 4.0 streaming (CASSANDRA-14362)
 + * Use standard Amazon naming for datacenter and rack in Ec2Snitch (CASSANDRA-7839)
 + * Fix junit failure for SSTableReaderTest (CASSANDRA-14387)
 + * Abstract write path for pluggable storage (CASSANDRA-14118)
 + * nodetool describecluster should be more informative (CASSANDRA-13853)
 + * Compaction performance improvements (CASSANDRA-14261) 
 + * Refactor Pair usage to avoid boxing ints/longs (CASSANDRA-14260)
 + * Add options to nodetool tablestats to sort and limit output (CASSANDRA-13889)
 + * Rename internals to reflect CQL vocabulary (CASSANDRA-14354)
 + * Add support for hybrid MIN(), MAX() speculative retry policies
 +   (CASSANDRA-14293, CASSANDRA-14338, CASSANDRA-14352)
 + * Fix some regressions caused by 14058 (CASSANDRA-14353)
 + * Abstract repair for pluggable storage (CASSANDRA-14116)
 + * Add meaningful toString() impls (CASSANDRA-13653)
 + * Add sstableloader option to accept target keyspace name (CASSANDRA-13884)
 + * Move processing of EchoMessage response to gossip stage (CASSANDRA-13713)
 + * Add coordinator write metric per CF (CASSANDRA-14232)
 + * Correct and clarify SSLFactory.getSslContext method and call sites (CASSANDRA-14314)
 + * Handle static and partition deletion properly on ThrottledUnfilteredIterator (CASSANDRA-14315)
 + * NodeTool clientstats should show SSL Cipher (CASSANDRA-14322)
 + * Add ability to specify driver name and version (CASSANDRA-14275)
 + * Abstract streaming for pluggable storage (CASSANDRA-14115)
 + * Forced incremental repairs should promote sstables if they can (CASSANDRA-14294)
 + * Use Murmur3 for validation compactions (CASSANDRA-14002)
 + * Comma at the end of the seed list is interpretated as localhost (CASSANDRA-14285)
 + * Refactor read executor and response resolver, abstract read repair (CASSANDRA-14058)
 + * Add optional startup delay to wait until peers are ready (CASSANDRA-13993)
 + * Add a few options to nodetool verify (CASSANDRA-14201)
 + * CVE-2017-5929 Security vulnerability and redefine default log rotation policy (CASSANDRA-14183)
 + * Use JVM default SSL validation algorithm instead of custom default (CASSANDRA-13259)
 + * Better document in code InetAddressAndPort usage post 7544, incorporate port into UUIDGen node (CASSANDRA-14226)
 + * Fix sstablemetadata date string for minLocalDeletionTime (CASSANDRA-14132)
 + * Make it possible to change neverPurgeTombstones during runtime (CASSANDRA-14214)
 + * Remove GossipDigestSynVerbHandler#doSort() (CASSANDRA-14174)
 + * Add nodetool clientlist (CASSANDRA-13665)
 + * Revert ProtocolVersion changes from CASSANDRA-7544 (CASSANDRA-14211)
 + * Non-disruptive seed node list reload (CASSANDRA-14190)
 + * Nodetool tablehistograms to print statics for all the tables (CASSANDRA-14185)
 + * Migrate dtests to use pytest and python3 (CASSANDRA-14134)
 + * Allow storage port to be configurable per node (CASSANDRA-7544)
 + * Make sub-range selection for non-frozen collections return null instead of empty (CASSANDRA-14182)
 + * BloomFilter serialization format should not change byte ordering (CASSANDRA-9067)
 + * Remove unused on-heap BloomFilter implementation (CASSANDRA-14152)
 + * Delete temp test files on exit (CASSANDRA-14153)
 + * Make PartitionUpdate and Mutation immutable (CASSANDRA-13867)
 + * Fix CommitLogReplayer exception for CDC data (CASSANDRA-14066)
 + * Fix cassandra-stress startup failure (CASSANDRA-14106)
 + * Remove initialDirectories from CFS (CASSANDRA-13928)
 + * Fix trivial log format error (CASSANDRA-14015)
 + * Allow sstabledump to do a json object per partition (CASSANDRA-13848)
 + * Add option to optimise merkle tree comparison across replicas (CASSANDRA-3200)
 + * Remove unused and deprecated methods from AbstractCompactionStrategy (CASSANDRA-14081)
 + * Fix Distribution.average in cassandra-stress (CASSANDRA-14090)
 + * Support a means of logging all queries as they were invoked (CASSANDRA-13983)
 + * Presize collections (CASSANDRA-13760)
 + * Add GroupCommitLogService (CASSANDRA-13530)
 + * Parallelize initial materialized view build (CASSANDRA-12245)
 + * Fix flaky SecondaryIndexManagerTest.assert[Not]MarkedAsBuilt (CASSANDRA-13965)
 + * Make LWTs send resultset metadata on every request (CASSANDRA-13992)
 + * Fix flaky indexWithFailedInitializationIsNotQueryableAfterPartialRebuild (CASSANDRA-13963)
 + * Introduce leaf-only iterator (CASSANDRA-9988)
 + * Upgrade Guava to 23.3 and Airline to 0.8 (CASSANDRA-13997)
 + * Allow only one concurrent call to StatusLogger (CASSANDRA-12182)
 + * Refactoring to specialised functional interfaces (CASSANDRA-13982)
 + * Speculative retry should allow more friendly params (CASSANDRA-13876)
 + * Throw exception if we send/receive repair messages to incompatible nodes (CASSANDRA-13944)
 + * Replace usages of MessageDigest with Guava's Hasher (CASSANDRA-13291)
 + * Add nodetool cmd to print hinted handoff window (CASSANDRA-13728)
 + * Fix some alerts raised by static analysis (CASSANDRA-13799)
 + * Checksum sstable metadata (CASSANDRA-13321, CASSANDRA-13593)
 + * Add result set metadata to prepared statement MD5 hash calculation (CASSANDRA-10786)
 + * Refactor GcCompactionTest to avoid boxing (CASSANDRA-13941)
 + * Expose recent histograms in JmxHistograms (CASSANDRA-13642)
 + * Fix buffer length comparison when decompressing in netty-based streaming (CASSANDRA-13899)
 + * Properly close StreamCompressionInputStream to release any ByteBuf (CASSANDRA-13906)
 + * Add SERIAL and LOCAL_SERIAL support for cassandra-stress (CASSANDRA-13925)
 + * LCS needlessly checks for L0 STCS candidates multiple times (CASSANDRA-12961)
 + * Correctly close netty channels when a stream session ends (CASSANDRA-13905)
 + * Update lz4 to 1.4.0 (CASSANDRA-13741)
 + * Optimize Paxos prepare and propose stage for local requests (CASSANDRA-13862)
 + * Throttle base partitions during MV repair streaming to prevent OOM (CASSANDRA-13299)
 + * Use compaction threshold for STCS in L0 (CASSANDRA-13861)
 + * Fix problem with min_compress_ratio: 1 and disallow ratio < 1 (CASSANDRA-13703)
 + * Add extra information to SASI timeout exception (CASSANDRA-13677)
 + * Add incremental repair support for --hosts, --force, and subrange repair (CASSANDRA-13818)
 + * Rework CompactionStrategyManager.getScanners synchronization (CASSANDRA-13786)
 + * Add additional unit tests for batch behavior, TTLs, Timestamps (CASSANDRA-13846)
 + * Add keyspace and table name in schema validation exception (CASSANDRA-13845)
 + * Emit metrics whenever we hit tombstone failures and warn thresholds (CASSANDRA-13771)
 + * Make netty EventLoopGroups daemon threads (CASSANDRA-13837)
 + * Race condition when closing stream sessions (CASSANDRA-13852)
 + * NettyFactoryTest is failing in trunk on macOS (CASSANDRA-13831)
 + * Allow changing log levels via nodetool for related classes (CASSANDRA-12696)
 + * Add stress profile yaml with LWT (CASSANDRA-7960)
 + * Reduce memory copies and object creations when acting on ByteBufs (CASSANDRA-13789)
 + * Simplify mx4j configuration (Cassandra-13578)
 + * Fix trigger example on 4.0 (CASSANDRA-13796)
 + * Force minumum timeout value (CASSANDRA-9375)
 + * Use netty for streaming (CASSANDRA-12229)
 + * Use netty for internode messaging (CASSANDRA-8457)
 + * Add bytes repaired/unrepaired to nodetool tablestats (CASSANDRA-13774)
 + * Don't delete incremental repair sessions if they still have sstables (CASSANDRA-13758)
 + * Fix pending repair manager index out of bounds check (CASSANDRA-13769)
 + * Don't use RangeFetchMapCalculator when RF=1 (CASSANDRA-13576)
 + * Don't optimise trivial ranges in RangeFetchMapCalculator (CASSANDRA-13664)
 + * Use an ExecutorService for repair commands instead of new Thread(..).start() (CASSANDRA-13594)
 + * Fix race / ref leak in anticompaction (CASSANDRA-13688)
 + * Expose tasks queue length via JMX (CASSANDRA-12758)
 + * Fix race / ref leak in PendingRepairManager (CASSANDRA-13751)
 + * Enable ppc64le runtime as unsupported architecture (CASSANDRA-13615)
 + * Improve sstablemetadata output (CASSANDRA-11483)
 + * Support for migrating legacy users to roles has been dropped (CASSANDRA-13371)
 + * Introduce error metrics for repair (CASSANDRA-13387)
 + * Refactoring to primitive functional interfaces in AuthCache (CASSANDRA-13732)
 + * Update metrics to 3.1.5 (CASSANDRA-13648)
 + * batch_size_warn_threshold_in_kb can now be set at runtime (CASSANDRA-13699)
 + * Avoid always rebuilding secondary indexes at startup (CASSANDRA-13725)
 + * Upgrade JMH from 1.13 to 1.19 (CASSANDRA-13727)
 + * Upgrade SLF4J from 1.7.7 to 1.7.25 (CASSANDRA-12996)
 + * Default for start_native_transport now true if not set in config (CASSANDRA-13656)
 + * Don't add localhost to the graph when calculating where to stream from (CASSANDRA-13583)
 + * Make CDC availability more deterministic via hard-linking (CASSANDRA-12148)
 + * Allow skipping equality-restricted clustering columns in ORDER BY clause (CASSANDRA-10271)
 + * Use common nowInSec for validation compactions (CASSANDRA-13671)
 + * Improve handling of IR prepare failures (CASSANDRA-13672)
 + * Send IR coordinator messages synchronously (CASSANDRA-13673)
 + * Flush system.repair table before IR finalize promise (CASSANDRA-13660)
 + * Fix column filter creation for wildcard queries (CASSANDRA-13650)
 + * Add 'nodetool getbatchlogreplaythrottle' and 'nodetool setbatchlogreplaythrottle' (CASSANDRA-13614)
 + * fix race condition in PendingRepairManager (CASSANDRA-13659)
 + * Allow noop incremental repair state transitions (CASSANDRA-13658)
 + * Run repair with down replicas (CASSANDRA-10446)
 + * Added started & completed repair metrics (CASSANDRA-13598)
 + * Added started & completed repair metrics (CASSANDRA-13598)
 + * Improve secondary index (re)build failure and concurrency handling (CASSANDRA-10130)
 + * Improve calculation of available disk space for compaction (CASSANDRA-13068)
 + * Change the accessibility of RowCacheSerializer for third party row cache plugins (CASSANDRA-13579)
 + * Allow sub-range repairs for a preview of repaired data (CASSANDRA-13570)
 + * NPE in IR cleanup when columnfamily has no sstables (CASSANDRA-13585)
 + * Fix Randomness of stress values (CASSANDRA-12744)
 + * Allow selecting Map values and Set elements (CASSANDRA-7396)
 + * Fast and garbage-free Streaming Histogram (CASSANDRA-13444)
 + * Update repairTime for keyspaces on completion (CASSANDRA-13539)
 + * Add configurable upper bound for validation executor threads (CASSANDRA-13521)
 + * Bring back maxHintTTL propery (CASSANDRA-12982)
 + * Add testing guidelines (CASSANDRA-13497)
 + * Add more repair metrics (CASSANDRA-13531)
 + * RangeStreamer should be smarter when picking endpoints for streaming (CASSANDRA-4650)
 + * Avoid rewrapping an exception thrown for cache load functions (CASSANDRA-13367)
 + * Log time elapsed for each incremental repair phase (CASSANDRA-13498)
 + * Add multiple table operation support to cassandra-stress (CASSANDRA-8780)
 + * Fix incorrect cqlsh results when selecting same columns multiple times (CASSANDRA-13262)
 + * Fix WriteResponseHandlerTest is sensitive to test execution order (CASSANDRA-13421)
 + * Improve incremental repair logging (CASSANDRA-13468)
 + * Start compaction when incremental repair finishes (CASSANDRA-13454)
 + * Add repair streaming preview (CASSANDRA-13257)
 + * Cleanup isIncremental/repairedAt usage (CASSANDRA-13430)
 + * Change protocol to allow sending key space independent of query string (CASSANDRA-10145)
 + * Make gc_log and gc_warn settable at runtime (CASSANDRA-12661)
 + * Take number of files in L0 in account when estimating remaining compaction tasks (CASSANDRA-13354)
 + * Skip building views during base table streams on range movements (CASSANDRA-13065)
 + * Improve error messages for +/- operations on maps and tuples (CASSANDRA-13197)
 + * Remove deprecated repair JMX APIs (CASSANDRA-11530)
 + * Fix version check to enable streaming keep-alive (CASSANDRA-12929)
 + * Make it possible to monitor an ideal consistency level separate from actual consistency level (CASSANDRA-13289)
 + * Outbound TCP connections ignore internode authenticator (CASSANDRA-13324)
 + * Upgrade junit from 4.6 to 4.12 (CASSANDRA-13360)
 + * Cleanup ParentRepairSession after repairs (CASSANDRA-13359)
 + * Upgrade snappy-java to 1.1.2.6 (CASSANDRA-13336)
 + * Incremental repair not streaming correct sstables (CASSANDRA-13328)
 + * Upgrade the jna version to 4.3.0 (CASSANDRA-13300)
 + * Add the currentTimestamp, currentDate, currentTime and currentTimeUUID functions (CASSANDRA-13132)
 + * Remove config option index_interval (CASSANDRA-10671)
 + * Reduce lock contention for collection types and serializers (CASSANDRA-13271)
 + * Make it possible to override MessagingService.Verb ids (CASSANDRA-13283)
 + * Avoid synchronized on prepareForRepair in ActiveRepairService (CASSANDRA-9292)
 + * Adds the ability to use uncompressed chunks in compressed files (CASSANDRA-10520)
 + * Don't flush sstables when streaming for incremental repair (CASSANDRA-13226)
 + * Remove unused method (CASSANDRA-13227)
 + * Fix minor bugs related to #9143 (CASSANDRA-13217)
 + * Output warning if user increases RF (CASSANDRA-13079)
 + * Remove pre-3.0 streaming compatibility code for 4.0 (CASSANDRA-13081)
 + * Add support for + and - operations on dates (CASSANDRA-11936)
 + * Fix consistency of incrementally repaired data (CASSANDRA-9143)
 + * Increase commitlog version (CASSANDRA-13161)
 + * Make TableMetadata immutable, optimize Schema (CASSANDRA-9425)
 + * Refactor ColumnCondition (CASSANDRA-12981)
 + * Parallelize streaming of different keyspaces (CASSANDRA-4663)
 + * Improved compactions metrics (CASSANDRA-13015)
 + * Speed-up start-up sequence by avoiding un-needed flushes (CASSANDRA-13031)
 + * Use Caffeine (W-TinyLFU) for on-heap caches (CASSANDRA-10855)
 + * Thrift removal (CASSANDRA-11115)
 + * Remove pre-3.0 compatibility code for 4.0 (CASSANDRA-12716)
 + * Add column definition kind to dropped columns in schema (CASSANDRA-12705)
 + * Add (automate) Nodetool Documentation (CASSANDRA-12672)
 + * Update bundled cqlsh python driver to 3.7.0 (CASSANDRA-12736)
 + * Reject invalid replication settings when creating or altering a keyspace (CASSANDRA-12681)
 + * Clean up the SSTableReader#getScanner API wrt removal of RateLimiter (CASSANDRA-12422)
 + * Use new token allocation for non bootstrap case as well (CASSANDRA-13080)
 + * Avoid byte-array copy when key cache is disabled (CASSANDRA-13084)
 + * Require forceful decommission if number of nodes is less than replication factor (CASSANDRA-12510)
 + * Allow IN restrictions on column families with collections (CASSANDRA-12654)
 + * Log message size in trace message in OutboundTcpConnection (CASSANDRA-13028)
 + * Add timeUnit Days for cassandra-stress (CASSANDRA-13029)
 + * Add mutation size and batch metrics (CASSANDRA-12649)
 + * Add method to get size of endpoints to TokenMetadata (CASSANDRA-12999)
 + * Expose time spent waiting in thread pool queue (CASSANDRA-8398)
 + * Conditionally update index built status to avoid unnecessary flushes (CASSANDRA-12969)
 + * cqlsh auto completion: refactor definition of compaction strategy options (CASSANDRA-12946)
 + * Add support for arithmetic operators (CASSANDRA-11935)
 + * Add histogram for delay to deliver hints (CASSANDRA-13234)
 + * Fix cqlsh automatic protocol downgrade regression (CASSANDRA-13307)
 + * Changing `max_hint_window_in_ms` at runtime (CASSANDRA-11720)
 + * Trivial format error in StorageProxy (CASSANDRA-13551)
 + * Nodetool repair can hang forever if we lose the notification for the repair completing/failing (CASSANDRA-13480)
 + * Anticompaction can cause noisy log messages (CASSANDRA-13684)
 + * Switch to client init for sstabledump (CASSANDRA-13683)
 + * CQLSH: Don't pause when capturing data (CASSANDRA-13743)
 + * nodetool clearsnapshot requires --all to clear all snapshots (CASSANDRA-13391)
 + * Correctly count range tombstones in traces and tombstone thresholds (CASSANDRA-8527)
 + * cqlshrc.sample uses incorrect option for time formatting (CASSANDRA-14243)
 +
 +
  3.11.4
   * Make stop-server.bat wait for Cassandra to terminate (CASSANDRA-14829)
   * Correct sstable sorting for garbagecollect and levelled compaction (CASSANDRA-14870)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e7a603da/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/CassandraDaemon.java
index 592419a,fbafd35..c8fddab
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@@ -505,10 -513,30 +505,34 @@@ public class CassandraDaemo
       */
      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.
 +        StartupClusterConnectivityChecker connectivityChecker = StartupClusterConnectivityChecker.create(DatabaseDescriptor.getBlockForPeersTimeoutInSeconds(),
 +                                                                                                         DatabaseDescriptor.getBlockForPeersInRemoteDatacenters());
 +        connectivityChecker.execute(Gossiper.instance.getEndpoints(), DatabaseDescriptor.getEndpointSnitch()::getDatacenter);
 +
++        // 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/e7a603da/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 68dba93,b504995..090b10b
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -207,7 -184,7 +207,8 @@@ public class StorageService extends Not
      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();
      private final AtomicBoolean isDecommissioning = new AtomicBoolean();
@@@ -1080,9 -1083,24 +1094,24 @@@
          }
          else if (isSurveyMode)
          {
-             logger.info("Leaving write survey mode and joining ring at operator request");
-             finishJoiningRing(resumedBootstrap, SystemKeyspace.getSavedTokens());
-             isSurveyMode = false;
+             // 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(resumedBootstrap, SystemKeyspace.getSavedTokens());
++                isSurveyMode = false;
+                 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.");
          }
      }
  
@@@ -1653,19 -1619,20 +1682,18 @@@
                  public void onSuccess(StreamState streamState)
                  {
                      bootstrapFinished();
--                    // start participating in the ring.
--                    // pretend we are in survey mode so we can use joinRing() here
-                     isSurveyMode = true;
-                     try
 -                    if (isSurveyMode)
++                    if(isSurveyMode)
                      {
-                         progressSupport.progress("bootstrap", ProgressEvent.createNotification("Joining ring..."));
-                         joinRing(true);
+                         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(true, 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/e7a603da/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index d11e997,e8aad77..47b6dbd
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -516,6 -531,13 +516,12 @@@ public interface StorageServiceMBean ex
      public boolean isJoined();
      public boolean isDrained();
      public boolean isDraining();
 -
+     /** 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 setRpcTimeout(long value);
      public long getRpcTimeout();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e7a603da/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e7a603da/src/java/org/apache/cassandra/tools/nodetool/Join.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Join.java
index 5a11ab8,a4a7cad..c77559c
--- a/src/java/org/apache/cassandra/tools/nodetool/Join.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Join.java
@@@ -41,4 -42,4 +42,4 @@@ public class Join extends NodeToolCm
              throw new RuntimeException("Error during joining the ring", e);
          }
      }
--}
++}


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


[2/4] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by jz...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/trunk
Commit: e2ee204580a0ee87b36e1df6aa2717830dfb33d0
Parents: 23d722e a6196a3
Author: Jay Zhuang <ja...@yahoo.com>
Authored: Sun Dec 30 21:41:28 2018 -0800
Committer: Jay Zhuang <ja...@yahoo.com>
Committed: Sun Dec 30 21:49:33 2018 -0800

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


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e2ee2045/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 8fd0200,16695c8..d29cdc1
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -33,39 -8,8 +33,40 @@@
   * Returns null instead of NaN or Infinity in JSON strings (CASSANDRA-14377)
  
  
 -2.2.13
 +3.0.17
 + * Fix corrupted static collection deletions in 3.0 -> 2.{1,2} messages (CASSANDRA-14568)
 + * Fix potential IndexOutOfBoundsException with counters (CASSANDRA-14167)
 + * Restore resumable hints delivery, backport CASSANDRA-11960 (CASSANDRA-14419)
 + * Always close RT markers returned by ReadCommand#executeLocally() (CASSANDRA-14515)
 + * Reverse order queries with range tombstones can cause data loss (CASSANDRA-14513)
 + * Fix regression of lagging commitlog flush log message (CASSANDRA-14451)
 + * Add Missing dependencies in pom-all (CASSANDRA-14422)
 + * Cleanup StartupClusterConnectivityChecker and PING Verb (CASSANDRA-14447)
 + * Fix deprecated repair error notifications from 3.x clusters to legacy JMX clients (CASSANDRA-13121)
 + * Cassandra not starting when using enhanced startup scripts in windows (CASSANDRA-14418)
 + * Fix progress stats and units in compactionstats (CASSANDRA-12244)
 + * Better handle missing partition columns in system_schema.columns (CASSANDRA-14379)
 + * Delay hints store excise by write timeout to avoid race with decommission (CASSANDRA-13740)
 + * Deprecate background repair and probablistic read_repair_chance table options
 +   (CASSANDRA-13910)
 + * Add missed CQL keywords to documentation (CASSANDRA-14359)
 + * Fix unbounded validation compactions on repair / revert CASSANDRA-13797 (CASSANDRA-14332)
 + * Avoid deadlock when running nodetool refresh before node is fully up (CASSANDRA-14310)
 + * Handle all exceptions when opening sstables (CASSANDRA-14202)
 + * Handle incompletely written hint descriptors during startup (CASSANDRA-14080)
 + * Handle repeat open bound from SRP in read repair (CASSANDRA-14330)
 + * Respect max hint window when hinting for LWT (CASSANDRA-14215)
 + * Adding missing WriteType enum values to v3, v4, and v5 spec (CASSANDRA-13697)
 + * Don't regenerate bloomfilter and summaries on startup (CASSANDRA-11163)
 + * Fix NPE when performing comparison against a null frozen in LWT (CASSANDRA-14087)
 + * Log when SSTables are deleted (CASSANDRA-14302)
 + * Fix batch commitlog sync regression (CASSANDRA-14292)
 + * Write to pending endpoint when view replica is also base replica (CASSANDRA-14251)
 + * Chain commit log marker potential performance regression in batch commit mode (CASSANDRA-14194)
 + * Fully utilise specified compaction threads (CASSANDRA-14210)
 + * Pre-create deletion log records to finish compactions quicker (CASSANDRA-12763)
 +Merged from 2.2:
+  * 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/e2ee2045/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/CassandraDaemon.java
index 177f8e3,e250050..6869d2c
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@@ -609,26 -597,6 +633,49 @@@ public class CassandraDaemo
          }
      }
  
 +    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 (StorageService.instance.hasJoined())
++        {
++            if (StorageService.instance.isSurveyMode())
++            {
++                if (StorageService.instance.isBootstrapMode() || DatabaseDescriptor.getAuthenticator().requireAuthentication())
++                {
++                    throw new IllegalStateException("Not starting client transports 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`");
++                }
++            }
++        }
++
 +        if (nativeTransportService == null)
 +            throw new IllegalStateException("setup() must be called first for CassandraDaemon");
 +        else
 +            nativeTransportService.start();
 +    }
 +
 +    public void stopNativeTransport()
 +    {
 +        if (nativeTransportService != null)
 +            nativeTransportService.stop();
 +    }
 +
 +    public boolean isNativeTransportRunning()
 +    {
 +        return nativeTransportService != null ? nativeTransportService.isRunning() : false;
 +    }
 +
 +
      /**
       * A convenience method to stop and destroy the daemon in one shot.
       */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e2ee2045/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 5f76f7d,03470b6..55bc9ae
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -175,7 -156,8 +175,9 @@@ public class StorageService extends Not
      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();
  
@@@ -211,6 -192,18 +213,16 @@@
  
      private final StreamStateStore streamStateStore = new StreamStateStore();
  
 -    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)
      {
@@@ -352,6 -337,30 +364,29 @@@
          {
              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");
++                    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`");
++                    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();
      }
  
@@@ -1284,20 -1303,19 +1337,19 @@@
                  @Override
                  public void onSuccess(StreamState streamState)
                  {
 -                    isBootstrapMode = false;
 +                    bootstrapFinished();
-                     // 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/e2ee2045/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 10d47f7,43d26c6..835da17
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -492,9 -490,14 +492,16 @@@ public interface StorageServiceMBean ex
      // allows a node that have been started without joining the ring to join it
      public void joinRing() throws IOException;
      public boolean isJoined();
 +    public boolean isDrained();
 +    public boolean isDraining();
  
+     /** 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/e2ee2045/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 0d3c078,a022595..7ed40bb
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -613,16 -574,11 +613,21 @@@ public class NodeProbe implements AutoC
          return ssProxy.isJoined();
      }
  
 +    public boolean isDrained()
 +    {
 +        return ssProxy.isDrained();
 +    }
 +
 +    public boolean isDraining()
 +    {
 +        return ssProxy.isDraining();
 +    }
 +
+     public boolean isBootstrapMode()
+     {
+         return ssProxy.isBootstrapMode();
+     }
+ 
      public void joinRing() throws IOException
      {
          ssProxy.joinRing();


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