Repository: kudu
Updated Branches:
  refs/heads/master 03a3c6812 -> 077fe5762


[raft_consensus-itest] fix TestElectionMetrics flake

This patch fixes flakiness in the RaftConsensusITest.TestElectionMetrics
scenario.  The original TestElectionMetrics scenario was split into
two parts TestElectionMetricsPart[1,2] to separate tests that assume
no leader election happen and depend on the leader failure detection
mechanism, correspondingly.

Prior to this patch, the TestElectionMetrics scenario could fail if a
leader election happened inadvertently.

Before (1 out of 12 failed):
  http://dist-test.cloudera.org/job?job_id=aserbin.1530681412.7635

After  (not a single failure in 1K run):
  http://dist-test.cloudera.org/job?job_id=aserbin.1530918429.120803

Change-Id: I073c9989a6d5d5dc1eb104120a89d38cfce2ac6e
Reviewed-on: http://gerrit.cloudera.org:8080/10887
Tested-by: Alexey Serbin <aser...@cloudera.com>
Reviewed-by: Will Berkeley <wdberke...@gmail.com>
Reviewed-by: Attila Bukor <abu...@cloudera.com>


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

Branch: refs/heads/master
Commit: 077fe576288bfaa1aadc42596b3594c65ecf16e2
Parents: 03a3c68
Author: Alexey Serbin <aser...@cloudera.com>
Authored: Thu Jul 5 13:39:06 2018 -0700
Committer: Alexey Serbin <aser...@cloudera.com>
Committed: Wed Jul 11 19:42:22 2018 +0000

----------------------------------------------------------------------
 src/kudu/consensus/raft_consensus.cc            |  10 ++
 .../integration-tests/raft_consensus-itest.cc   | 136 +++++++++++++++----
 2 files changed, 120 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/077fe576/src/kudu/consensus/raft_consensus.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus.cc 
b/src/kudu/consensus/raft_consensus.cc
index c70abd9..f81637e 100644
--- a/src/kudu/consensus/raft_consensus.cc
+++ b/src/kudu/consensus/raft_consensus.cc
@@ -1320,6 +1320,16 @@ Status RaftConsensus::UpdateReplica(const 
ConsensusRequestPB* request,
 
     last_leader_communication_time_micros_ = GetMonoTimeMicros();
 
+    // Reset the 'failed_elections_since_stable_leader' metric now that we've
+    // accepted an update from the established leader. This is done in addition
+    // to the reset of the value in SetLeaderUuidUnlocked() because there is
+    // a potential race between resetting the failed elections count in
+    // SetLeaderUuidUnlocked() and incrementing after a failed election
+    // if another replica was elected leader in an election concurrent with
+    // the one called by this replica.
+    failed_elections_since_stable_leader_ = 0;
+    
num_failed_elections_metric_->set_value(failed_elections_since_stable_leader_);
+
     // We update the lag metrics here in addition to after appending to the 
queue so the
     // metrics get updated even when the operation is rejected.
     
queue_->UpdateLastIndexAppendedToLeader(request->last_idx_appended_to_leader());

http://git-wip-us.apache.org/repos/asf/kudu/blob/077fe576/src/kudu/integration-tests/raft_consensus-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/raft_consensus-itest.cc 
b/src/kudu/integration-tests/raft_consensus-itest.cc
index d1425d0..f7be807 100644
--- a/src/kudu/integration-tests/raft_consensus-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest.cc
@@ -2670,61 +2670,145 @@ int64_t GetTimeSinceLastLeaderHeartbeat(const 
ExternalTabletServer* ets,
   return ret;
 }
 
-TEST_F(RaftConsensusITest, TestElectionMetrics) {
+// Test for election-related metrics with the leader failure detection 
disabled.
+// This avoids inadvertent leader elections that might race with the tests
+// of the metric's behavior.
+TEST_F(RaftConsensusITest, TestElectionMetricsFailureDetectionDisabled) {
+  constexpr auto kNumReplicas = 3;
+  constexpr auto kNumTservers = 3;
   const vector<string> kTsFlags = {
     // Make leader elections faster so we can test
     // failed_elections_since_stable_leader faster.
     "--raft_heartbeat_interval_ms=100",
+
+    // For stability reasons, this scenario uses the 'manual election' mode
+    // and assumes no leader change before restarting tablet servers later on.
+    "--enable_leader_failure_detection=false",
+  };
+  const vector<string> kMasterFlags = {
+    // Corresponding flag for the tserver's 
'--enable_leader_failure_detection'.
+    "--catalog_manager_wait_for_new_tablets_to_elect_leader=false",
   };
+  const auto kTimeout = MonoDelta::FromSeconds(30);
 
-  FLAGS_num_replicas = 3;
-  FLAGS_num_tablet_servers = 3;
-  NO_FATALS(BuildAndStart(kTsFlags));
+  FLAGS_num_replicas = kNumReplicas;
+  FLAGS_num_tablet_servers = kNumTservers;
+  NO_FATALS(BuildAndStart(kTsFlags, kMasterFlags));
 
   vector<TServerDetails*> tservers;
   AppendValuesFromMap(tablet_servers_, &tservers);
 
-  ASSERT_OK(StartElection(tservers[0], tablet_id_, 
MonoDelta::FromSeconds(10)));
-  ASSERT_OK(WaitForServersToAgree(MonoDelta::FromSeconds(10), tablet_servers_, 
tablet_id_, 1));
+  const auto* leader = tablet_servers_.begin()->second;
+  const auto& leader_uuid = leader->uuid();
+  ASSERT_EVENTUALLY([&]() {
+    const auto kLeaderTimeout = MonoDelta::FromSeconds(10);
+    ASSERT_OK(StartElection(leader, tablet_id_, kLeaderTimeout));
+    TServerDetails* elected_leader;
+    ASSERT_OK(WaitForLeaderWithCommittedOp(
+        tablet_id_, kLeaderTimeout, &elected_leader));
+    ASSERT_EQ(leader->uuid(), elected_leader->uuid());
+  });
+  ASSERT_OK(WaitForServersToAgree(kTimeout, tablet_servers_, tablet_id_, 1));
 
-  int num_tablet_servers = cluster_->num_tablet_servers();
-  int leader_idx = cluster_->tablet_server_index_by_uuid(tservers[0]->uuid());
-  ExternalTabletServer* leader = cluster_->tablet_server(leader_idx);
-  ExternalTabletServer* follower = cluster_->tablet_server((leader_idx + 1) % 
num_tablet_servers);
+  auto* leader_srv = cluster_->tablet_server_by_uuid(leader_uuid);
+  auto* follower_srv = cluster_->tablet_server(
+      (cluster_->tablet_server_index_by_uuid(leader_uuid) + 1) % kNumTservers);
 
   // Leader should always report 0 since last leader heartbeat.
-  ASSERT_EQ(0, GetTimeSinceLastLeaderHeartbeat(leader, tablet_id_));
-  ASSERT_EQ(0, GetFailedElectionsSinceStableLeader(leader, tablet_id_));
+  EXPECT_EQ(0, GetTimeSinceLastLeaderHeartbeat(leader_srv, tablet_id_));
+  EXPECT_EQ(0, GetFailedElectionsSinceStableLeader(leader_srv, tablet_id_));
 
-  // Let's shut down all tablet servers except our chosen follower to make 
sure we don't
-  // have a leader.
-  for (auto i = 0; i < num_tablet_servers; ++i) {
-    if (cluster_->tablet_server(i) != follower) {
+  // Let's shut down all tablet servers except our chosen follower to make sure
+  // we don't have a leader.
+  for (auto i = 0; i < kNumTservers; ++i) {
+    if (cluster_->tablet_server(i) != follower_srv) {
       cluster_->tablet_server(i)->Shutdown();
     }
   }
 
   // Get two measurements with 500 ms sleep between them and see if the
   // difference between them is at least 500ms.
-  int64_t time_before_wait = GetTimeSinceLastLeaderHeartbeat(follower, 
tablet_id_);
+  const int64_t time_before_wait = 
GetTimeSinceLastLeaderHeartbeat(follower_srv,
+                                                                   tablet_id_);
   SleepFor(MonoDelta::FromMilliseconds(500));
-  int64_t time_after_wait = GetTimeSinceLastLeaderHeartbeat(follower, 
tablet_id_);
-  ASSERT_TRUE(time_after_wait >= time_before_wait + 500);
+  const int64_t time_after_wait = GetTimeSinceLastLeaderHeartbeat(follower_srv,
+                                                                  tablet_id_);
+  ASSERT_TRUE(time_after_wait >= time_before_wait + 500)
+      << "time_before_wait: " << time_before_wait << "; "
+      << "time_after_wait: " << time_after_wait;
+
+  // The follower doesn't start elections on itself because of
+  // --enable_leader_failure_detection=false flag, so the metric should read 0.
+  ASSERT_EQ(0, GetFailedElectionsSinceStableLeader(follower_srv, tablet_id_));
+}
+
+// Test for election-related metrics with the leader failure detection enabled.
+TEST_F(RaftConsensusITest, TestElectionMetricsFailureDetectionEnabled) {
+  constexpr auto kNumReplicas = 3;
+  constexpr auto kNumTservers = 3;
+  const vector<string> kTsFlags = {
+    // Make leader elections faster so we can test
+    // failed_elections_since_stable_leader faster.
+    "--raft_heartbeat_interval_ms=100",
+
+    // For the stability of the test, make leader election less likely in case
+    // of intermittent network failures and latency spikes. Otherwise, in case
+    // of on-going re-elections the metric value would not stabilize for a long
+    // time.
+    "--leader_failure_max_missed_heartbeat_periods=20",
+  };
+  const auto kTimeout = MonoDelta::FromSeconds(30);
+
+  FLAGS_num_replicas = kNumReplicas;
+  FLAGS_num_tablet_servers = kNumTservers;
+  NO_FATALS(BuildAndStart(kTsFlags));
+  ASSERT_OK(WaitForServersToAgree(kTimeout, tablet_servers_, tablet_id_, 1));
+
+  // Verify failed_elections_since_stable_leader is reset to 0 in the majority
+  // of all replicas after the cluster is in stable state, i.e. the leader
+  // is elected and functional.
+  ASSERT_EVENTUALLY([&]() {
+    for (auto i = 0; i < kNumTservers; ++i) {
+      ASSERT_EQ(0, GetFailedElectionsSinceStableLeader(
+          cluster_->tablet_server(i), tablet_id_));
+    }
+  });
+
+  auto* srv_ts = tablet_servers_.begin()->second;
+  auto* srv_ext = cluster_->tablet_server_by_uuid(srv_ts->uuid());
+  for (auto i = 0; i < kNumTservers; ++i) {
+    auto* s = cluster_->tablet_server(i);
+    if (s == srv_ext) {
+      continue;
+    }
+    s->Shutdown();
+  }
+
+  // If the server which is left alive hosts the leader replica for the tablet,
+  // make it step down. Otherwise, it's a no-op.
+  ignore_result(LeaderStepDown(srv_ts, tablet_id_, kTimeout));
 
   // Verify failed_elections_since_stable_leader is advanced eventually.
   ASSERT_EVENTUALLY([&]() {
-    ASSERT_TRUE(GetFailedElectionsSinceStableLeader(follower, tablet_id_) > 0);
+    ASSERT_GT(GetFailedElectionsSinceStableLeader(srv_ext, tablet_id_), 0);
   });
 
-  // Start the servers back up and verify failed_elections_since_stable_leader
-  // is reset to 0.
-  for (auto i = 0; i < num_tablet_servers; ++i) {
-    if (cluster_->tablet_server(i) != follower) {
-      cluster_->tablet_server(i)->Start();
+  // Restart the rest of tablet servers.
+  for (auto i = 0; i < kNumTservers; ++i) {
+    auto* s = cluster_->tablet_server(i);
+    if (s == srv_ext) {
+      continue;
     }
+    ASSERT_OK(s->Restart());
   }
+  ASSERT_OK(WaitForServersToAgree(kTimeout, tablet_servers_, tablet_id_, 1));
+
+  // Verify failed_elections_since_stable_leader is reset to 0 eventually.
   ASSERT_EVENTUALLY([&]() {
-    ASSERT_EQ(0, GetFailedElectionsSinceStableLeader(follower, tablet_id_));
+    for (auto i = 0; i < kNumTservers; ++i) {
+      ASSERT_EQ(0, GetFailedElectionsSinceStableLeader(
+          cluster_->tablet_server(i), tablet_id_));
+    }
   });
 }
 

Reply via email to