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