You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2016/06/27 17:18:39 UTC

incubator-kudu git commit: Fix flakiness of RaftConsensusITest.TestLogIOErrorIsFatal

Repository: incubator-kudu
Updated Branches:
  refs/heads/master 11de5f2d1 -> cb8f17589


Fix flakiness of RaftConsensusITest.TestLogIOErrorIsFatal

This test was flaky because it was assuming that the order of entries in the
TServersDetail map matched the ordering of tablet servers in the minicluster's
tablet server vector. This is not always the case, so the assertions waiting
for nodes to crash sometimes timed out (since they were waiting on the wrong
nodes).

Change-Id: If62911f9517b60d82fac97bacd2ac01584be442f
Reviewed-on: http://gerrit.cloudera.org:8080/3493
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: cb8f175894862e33fc3ec58879b4a6f7d4e24e1a
Parents: 11de5f2
Author: Todd Lipcon <to...@cloudera.com>
Authored: Thu Jun 23 18:51:29 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Mon Jun 27 17:10:58 2016 +0000

----------------------------------------------------------------------
 .../integration-tests/raft_consensus-itest.cc   | 20 ++++++++++++--------
 1 file changed, 12 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cb8f1758/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 e112b82..08a9249 100644
--- a/src/kudu/integration-tests/raft_consensus-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest.cc
@@ -2651,6 +2651,10 @@ TEST_F(RaftConsensusITest, TestLogIOErrorIsFatal) {
   vector<TServerDetails*> tservers;
   AppendValuesFromMap(tablet_servers_, &tservers);
   ASSERT_EQ(3, tservers.size());
+  vector<ExternalTabletServer*> ext_tservers;
+  for (auto* details : tservers) {
+    ext_tservers.push_back(cluster_->tablet_server_by_uuid(details->uuid()));
+  }
 
   // Test failed replicates.
 
@@ -2662,17 +2666,17 @@ TEST_F(RaftConsensusITest, TestLogIOErrorIsFatal) {
   // Then, cause server 0 to start and win a leader election.
   // This will cause servers 0 and 1 to crash.
   for (int i = 1; i <= 2; i++) {
-    ASSERT_OK(cluster_->SetFlag(cluster_->tablet_server(i),
+    ASSERT_OK(cluster_->SetFlag(ext_tservers[i],
               "log_inject_io_error_on_append_fraction", "1.0"));
   }
   ASSERT_OK(StartElection(tservers[0], tablet_id_, MonoDelta::FromSeconds(10)));
   for (int i = 1; i <= 2; i++) {
-    ASSERT_OK(cluster_->tablet_server(i)->WaitForCrash(MonoDelta::FromSeconds(10)));
+    ASSERT_OK(ext_tservers[i]->WaitForCrash(MonoDelta::FromSeconds(10)));
   }
 
   // Now we know followers crash when they write to their log.
   // Let's verify the same for the leader (server 0).
-  ASSERT_OK(cluster_->SetFlag(cluster_->tablet_server(0),
+  ASSERT_OK(cluster_->SetFlag(ext_tservers[0],
             "log_inject_io_error_on_append_fraction", "1.0"));
 
   // Attempt to write to the leader, but with a short timeout.
@@ -2686,7 +2690,7 @@ TEST_F(RaftConsensusITest, TestLogIOErrorIsFatal) {
   workload.Start();
 
   // Leader should crash as well.
-  ASSERT_OK(cluster_->tablet_server(0)->WaitForCrash(MonoDelta::FromSeconds(10)));
+  ASSERT_OK(ext_tservers[0]->WaitForCrash(MonoDelta::FromSeconds(10)));
   workload.StopAndJoin();
 
   LOG(INFO) << "Everything crashed!";
@@ -2714,7 +2718,7 @@ TEST_F(RaftConsensusITest, TestLogIOErrorIsFatal) {
   // leader, set flags so that commits crash the server, then bring the
   // followers back up.
   for (int i = 1; i <= 2; i++) {
-    cluster_->tablet_server(i)->Shutdown();
+    ext_tservers[i]->Shutdown();
   }
 
   OpId prev_opid, cur_opid;
@@ -2729,13 +2733,13 @@ TEST_F(RaftConsensusITest, TestLogIOErrorIsFatal) {
     VLOG(1) << "Current OpId on server 0: " << OpIdToString(cur_opid);
   } while (consensus::OpIdEquals(prev_opid, cur_opid));
   workload.StopAndJoin();
-  ASSERT_OK(cluster_->SetFlag(cluster_->tablet_server(0),
+  ASSERT_OK(cluster_->SetFlag(ext_tservers[0],
             "log_inject_io_error_on_append_fraction", "1.0"));
   for (int i = 1; i <= 2; i++) {
-    ASSERT_OK(cluster_->tablet_server(i)->Restart());
+    ASSERT_OK(ext_tservers[i]->Restart());
   }
   // Leader will crash.
-  ASSERT_OK(cluster_->tablet_server(0)->WaitForCrash(MonoDelta::FromSeconds(10)));
+  ASSERT_OK(ext_tservers[0]->WaitForCrash(MonoDelta::FromSeconds(10)));
 }
 
 }  // namespace tserver