You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2017/09/21 01:47:19 UTC

kudu git commit: KUDU-2149: avoid election stacking by restoring failure monitor semantics

Repository: kudu
Updated Branches:
  refs/heads/master 6b364a24c -> edd41cb40


KUDU-2149: avoid election stacking by restoring failure monitor semantics

Prior to commit 21b0f3d, the dedicated failure monitor thread invoked
RaftConsensus::StartElection() synchronously, thus preventing it from
surfacing additional failures during that time. This patch attempts to
restore these semantics by short-circuiting and ignoring any failures
detected while a Raft thread is in StartElection().

This is a super targeted fix geared towards a point release; a more correct
fix would be to completely disable failure detection while an election is
running, but that'll require more work.

Originally I had written a test that injects latency into
ConsensusMetadata::Flush(), toggles the fix, and compares the number of vote
request RPCs. I couldn't get it to be totally robust, and the "feature flag"
used in the toggle is likely to become obselete quickly. So in the end I
decided to drop the test from the patch.

Change-Id: Ifeaf99ce57f7d5cd01a6c786c178567a98438ced
Reviewed-on: http://gerrit.cloudera.org:8080/8107
Reviewed-by: Mike Percy <mp...@apache.org>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: edd41cb40fbad206e2c356983baba8fbc57199b5
Parents: 6b364a2
Author: Adar Dembo <ad...@cloudera.com>
Authored: Tue Sep 19 18:45:51 2017 -0700
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Thu Sep 21 01:46:40 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/raft_consensus.cc | 10 +++++++---
 src/kudu/consensus/raft_consensus.h  | 16 ++++++++++++++++
 2 files changed, 23 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/edd41cb4/src/kudu/consensus/raft_consensus.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus.cc b/src/kudu/consensus/raft_consensus.cc
index 78f9856..6b4e968 100644
--- a/src/kudu/consensus/raft_consensus.cc
+++ b/src/kudu/consensus/raft_consensus.cc
@@ -520,9 +520,13 @@ scoped_refptr<ConsensusRound> RaftConsensus::NewRound(
 }
 
 void RaftConsensus::ReportFailureDetectedTask() {
-  WARN_NOT_OK(StartElection(FLAGS_raft_enable_pre_election ?
-      PRE_ELECTION : NORMAL_ELECTION, ELECTION_TIMEOUT_EXPIRED),
-              LogPrefixThreadSafe() + "failed to trigger leader election");
+  std::unique_lock<simple_spinlock> try_lock(failure_detector_election_lock_,
+                                             std::try_to_lock);
+  if (try_lock.owns_lock()) {
+    WARN_NOT_OK(StartElection(FLAGS_raft_enable_pre_election ?
+        PRE_ELECTION : NORMAL_ELECTION, ELECTION_TIMEOUT_EXPIRED),
+                LogPrefixThreadSafe() + "failed to trigger leader election");
+  }
 }
 
 void RaftConsensus::ReportFailureDetected() {

http://git-wip-us.apache.org/repos/asf/kudu/blob/edd41cb4/src/kudu/consensus/raft_consensus.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus.h b/src/kudu/consensus/raft_consensus.h
index 72e7590..b82afb3 100644
--- a/src/kudu/consensus/raft_consensus.h
+++ b/src/kudu/consensus/raft_consensus.h
@@ -775,6 +775,22 @@ class RaftConsensus : public std::enable_shared_from_this<RaftConsensus>,
 
   std::shared_ptr<rpc::PeriodicTimer> failure_detector_;
 
+  // Lock held while starting a failure-triggered election.
+  //
+  // After reporting a failure and asynchronously starting an election, the
+  // failure detector immediately rearms. If the election starts slowly (i.e.
+  // there's a lot of contention on the consensus lock, or persisting votes is
+  // really slow due to other I/O), more elections may start and "stack" on
+  // top of the first. Forcing the starting of elections to serialize on this
+  // lock prevents that from happening. See KUDU-2149 for more details.
+  //
+  // Note: the lock is only ever acquired via try_lock(); if it cannot be
+  // acquired, a StartElection() is in progress so the next one is skipped.
+  //
+  // TODO(KUDU-2155): should be replaced with explicit disabling/enabling of
+  // the failure detector during elections.
+  simple_spinlock failure_detector_election_lock_;
+
   // If any RequestVote() RPC arrives before this timestamp,
   // the request will be ignored. This prevents abandoned or partitioned
   // nodes from disturbing the healthy leader.