You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2014/01/09 03:55:05 UTC
[3/5] git commit: Wait for gossip to settle before accepting client
connections patch by Chris Burroughs;
reviewed by Tyler Hobbs and jbellis for CASSANDRA-4288
Wait for gossip to settle before accepting client connections
patch by Chris Burroughs; reviewed by Tyler Hobbs and jbellis for CASSANDRA-4288
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/de19f963
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/de19f963
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/de19f963
Branch: refs/heads/trunk
Commit: de19f963aeed2752374d2f84c1b230f6cab253f1
Parents: f8fd7db
Author: Jonathan Ellis <jb...@apache.org>
Authored: Wed Jan 8 20:53:46 2014 -0600
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Wed Jan 8 20:53:46 2014 -0600
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../cassandra/service/CassandraDaemon.java | 50 ++++++++++++++++++++
2 files changed, 51 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/de19f963/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d0b63a0..e96a8e0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
2.0.5
+ * Wait for gossip to settle before accepting client connections (CASSANDRA-4288)
* Delete unfinished compaction incrementally (CASSANDRA-6086)
* Allow specifying custom secondary index options in CQL3 (CASSANDRA-6480)
Merged from 1.2:
http://git-wip-us.apache.org/repos/asf/cassandra/blob/de19f963/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 d36b0db..d497a38 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -34,10 +34,14 @@ import javax.management.StandardMBean;
import com.addthis.metrics.reporter.config.ReporterConfig;
import com.google.common.collect.Iterables;
+import com.google.common.util.concurrent.Uninterruptibles;
import org.apache.log4j.PropertyConfigurator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
+import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.concurrent.StageManager;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.Schema;
@@ -365,6 +369,8 @@ public class CassandraDaemon
}
}
+ waitForGossipToSettle();
+
// Thift
InetAddress rpcAddr = DatabaseDescriptor.getRpcAddress();
int rpcPort = DatabaseDescriptor.getRpcPort();
@@ -489,6 +495,50 @@ public class CassandraDaemon
destroy();
}
+
+ private void waitForGossipToSettle()
+ {
+ int forceAfter = Integer.getInteger("cassandra.skip_wait_for_gossip_to_settle", -1);
+ if (forceAfter == 0)
+ {
+ return;
+ }
+ final int GOSSIP_SETTLE_MIN_WAIT_MS = 5000;
+ final int GOSSIP_SETTLE_POLL_INTERVAL_MS = 1000;
+ final int GOSSIP_SETTLE_POLL_SUCCESSES_REQUIRED = 3;
+
+ logger.info("waiting for gossip to settle before accepting client requests...");
+ Uninterruptibles.sleepUninterruptibly(GOSSIP_SETTLE_MIN_WAIT_MS, TimeUnit.MILLISECONDS);
+ int totalPolls = 0;
+ int numOkay = 0;
+ JMXEnabledThreadPoolExecutor gossipStage = (JMXEnabledThreadPoolExecutor)StageManager.getStage(Stage.GOSSIP);
+ while (numOkay < GOSSIP_SETTLE_POLL_SUCCESSES_REQUIRED)
+ {
+ Uninterruptibles.sleepUninterruptibly(GOSSIP_SETTLE_POLL_INTERVAL_MS, TimeUnit.MILLISECONDS);
+ long completed = gossipStage.getCompletedTasks();
+ long active = gossipStage.getActiveCount();
+ long pending = gossipStage.getPendingTasks();
+ totalPolls++;
+ if (active == 0 && pending == 0)
+ {
+ logger.debug("gossip looks settled. CompletedTasks: {}", completed);
+ numOkay++;
+ }
+ else
+ {
+ logger.info("gossip not settled after {} polls. Gossip Stage active/pending/completed: {}/{}/{}", totalPolls, active, pending, completed);
+ numOkay = 0;
+ }
+ if (forceAfter > 0 && totalPolls > forceAfter)
+ {
+ logger.warn("Gossip not settled but startup forced by cassandra.skip_wait_for_gossip_to_settle. Gossip Stage active/pending/completed: {}/{}/{}",
+ totalPolls, active, pending, completed);
+ break;
+ }
+ }
+ logger.info("gossip settled after {} extra polls; proceeding", totalPolls - GOSSIP_SETTLE_POLL_SUCCESSES_REQUIRED);
+ }
+
public static void stop(String[] args)
{
instance.deactivate();