You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by mp...@apache.org on 2017/05/02 02:32:23 UTC

kudu git commit: tests: AssertEventually and NO_FATALS improvements

Repository: kudu
Updated Branches:
  refs/heads/master 28869e086 -> 50b84ae6b


tests: AssertEventually and NO_FATALS improvements

As implemented, the macro NO_FATALS() is a synonym for
ASSERT_NO_FATAL_FAILURES(). Unfortunately, ASSERT_NO_FATAL_FAILURES()
only checks for fatal failures in its argument expression, not in the
surrounding scope.

Also, for some reason, older versions of GCC don't seem to be able to
tolerate AssertEventually() wrapped by a NO_FATALS() call, which means
we have to check for timeout failures after AssertEventually() returns.

There are a few cases where we are using NO_FATALS() wrong so this patch
corrects those.

Included in this patch are the following changes:

1. Add a new macro called NO_PENDING_FATALS() that checks the
   surrounding scope for fatals.
2. Replace useless empty NO_FATALS() calls with NO_PENDING_FATALS().
3. Add new macro called ASSERT_EVENTUALLY() that wraps the 1-arg form of
   AssertEventually() and checks for failures. Use it wherever possible.
4. Add a call to NO_PENDING_FATALS() after use of the 2-arg form of
   AssertEventually().

Change-Id: Ia004214d21a09ca0523ea11d88d2cd4ff783a1ca
Reviewed-on: http://gerrit.cloudera.org:8080/6730
Reviewed-by: Adar Dembo <ad...@cloudera.com>
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/50b84ae6
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/50b84ae6
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/50b84ae6

Branch: refs/heads/master
Commit: 50b84ae6b30a62db3d5d40e2e8f78545883e155a
Parents: 28869e0
Author: Mike Percy <mp...@apache.org>
Authored: Mon Apr 24 19:05:18 2017 -0700
Committer: Mike Percy <mp...@apache.org>
Committed: Tue May 2 02:32:09 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/consensus_peers-test.cc       |  2 +-
 src/kudu/integration-tests/delete_table-itest.cc |  6 +++---
 .../integration-tests/delete_tablet-itest.cc     |  5 ++---
 src/kudu/integration-tests/log-rolling-itest.cc  |  2 +-
 .../minidump_generation-itest.cc                 |  2 +-
 .../integration-tests/raft_consensus-itest.cc    | 12 ++++++------
 src/kudu/integration-tests/registration-test.cc  |  8 ++++----
 .../integration-tests/tablet_history_gc-itest.cc |  4 ++--
 src/kudu/integration-tests/token_signer-itest.cc |  1 +
 src/kudu/integration-tests/ts_recovery-itest.cc  |  8 +++-----
 src/kudu/master/master-test.cc                   |  2 +-
 src/kudu/rpc/exactly_once_rpc-test.cc            |  1 +
 src/kudu/tablet/diskrowset-test.cc               | 19 +++++++------------
 src/kudu/tablet/tablet-test.cc                   |  2 +-
 src/kudu/tools/kudu-admin-test.cc                |  2 +-
 src/kudu/util/debug-util-test.cc                 |  2 +-
 src/kudu/util/env-test.cc                        |  3 ++-
 src/kudu/util/env_util-test.cc                   |  2 +-
 src/kudu/util/file_cache-test.cc                 |  2 +-
 src/kudu/util/maintenance_manager-test.cc        |  8 ++++----
 src/kudu/util/minidump-test.cc                   |  2 +-
 src/kudu/util/test_macros.h                      |  8 +++++++-
 src/kudu/util/test_util.h                        |  9 +++++++++
 23 files changed, 61 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/consensus/consensus_peers-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_peers-test.cc b/src/kudu/consensus/consensus_peers-test.cc
index 8f23cbe..3b15534 100644
--- a/src/kudu/consensus/consensus_peers-test.cc
+++ b/src/kudu/consensus/consensus_peers-test.cc
@@ -118,7 +118,7 @@ class ConsensusPeersTest : public KuduTest {
   // is committed in the test consensus impl.
   // This must be called _before_ the operation is committed.
   void WaitForCommitIndex(int index) {
-    AssertEventually([&]() {
+    ASSERT_EVENTUALLY([&]() {
         ASSERT_GE(message_queue_->GetCommittedIndex(), index);
       });
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/integration-tests/delete_table-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/delete_table-itest.cc b/src/kudu/integration-tests/delete_table-itest.cc
index 0a91955..73f58ff 100644
--- a/src/kudu/integration-tests/delete_table-itest.cc
+++ b/src/kudu/integration-tests/delete_table-itest.cc
@@ -1013,7 +1013,7 @@ TEST_F(DeleteTableITest, TestWebPageForTombstonedTablet) {
 
   // Tombstone the tablet.
   ExternalTabletServer* ets = cluster_->tablet_server(0);
-  AssertEventually([&]() {
+  ASSERT_EVENTUALLY([&]() {
     ASSERT_OK(itest::DeleteTablet(ts_map_[ets->uuid()],
                                   tablet_id, TABLET_DATA_TOMBSTONED, boost::none, timeout));
   });
@@ -1069,7 +1069,7 @@ TEST_F(DeleteTableITest, TestUnknownTabletsAreNotDeleted) {
   ASSERT_OK(cluster_->master()->WaitForCatalogManager());
 
   int64_t num_delete_attempts;
-  AssertEventually([&]() {
+  ASSERT_EVENTUALLY([&]() {
     int64_t num_heartbeats;
     ASSERT_OK(cluster_->master()->GetInt64Metric(
         &METRIC_ENTITY_server, "kudu.master",
@@ -1096,7 +1096,7 @@ TEST_F(DeleteTableITest, TestUnknownTabletsAreNotDeleted) {
   cluster_->master()->mutable_flags()->push_back(
       "--catalog_manager_delete_orphaned_tablets");
   ASSERT_OK(cluster_->Restart());
-  AssertEventually([&]() {
+  ASSERT_EVENTUALLY([&]() {
     ASSERT_OK(cluster_->tablet_server(0)->GetInt64Metric(
         &METRIC_ENTITY_server, "kudu.tabletserver",
         &METRIC_handler_latency_kudu_tserver_TabletServerAdminService_DeleteTablet,

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/integration-tests/delete_tablet-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/delete_tablet-itest.cc b/src/kudu/integration-tests/delete_tablet-itest.cc
index e145701..5cc292c 100644
--- a/src/kudu/integration-tests/delete_tablet-itest.cc
+++ b/src/kudu/integration-tests/delete_tablet-itest.cc
@@ -54,13 +54,12 @@ TEST_F(DeleteTabletITest, TestDeleteFailedReplica) {
   auto* ts = ts_map[mts->uuid()];
 
   scoped_refptr<TabletPeer> tablet_peer;
-  AssertEventually([&] {
+  ASSERT_EVENTUALLY([&] {
     vector<scoped_refptr<TabletPeer>> tablet_peers;
     mts->server()->tablet_manager()->GetTabletPeers(&tablet_peers);
     ASSERT_EQ(1, tablet_peers.size());
     tablet_peer = tablet_peers[0];
   });
-  NO_FATALS();
 
   workload.Start();
   while (workload.rows_inserted() < 100) {
@@ -96,7 +95,7 @@ TEST_F(DeleteTabletITest, TestDeleteFailedReplica) {
   // We should still be able to delete the failed tablet.
   ASSERT_OK(itest::DeleteTablet(ts, tablet_peer->tablet_id(), tablet::TABLET_DATA_DELETED,
                                 boost::none, MonoDelta::FromSeconds(30)));
-  AssertEventually([&] {
+  ASSERT_EVENTUALLY([&] {
     vector<scoped_refptr<TabletPeer>> tablet_peers;
     mts->server()->tablet_manager()->GetTabletPeers(&tablet_peers);
     ASSERT_EQ(0, tablet_peers.size());

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/integration-tests/log-rolling-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/log-rolling-itest.cc b/src/kudu/integration-tests/log-rolling-itest.cc
index 37e7f4c..6397233 100644
--- a/src/kudu/integration-tests/log-rolling-itest.cc
+++ b/src/kudu/integration-tests/log-rolling-itest.cc
@@ -61,7 +61,7 @@ TEST(LogRollingITest, TestLogCleanupOnStartup) {
   ASSERT_OK(cluster.master()->WaitForCatalogManager());
 
   for (int i = 1; i <= 10; i++) {
-    AssertEventually([&] () {
+    ASSERT_EVENTUALLY([&] () {
         ASSERT_EQ(std::min(3, i), CountInfoLogs(cluster.master()->log_dir()));
     });
     cluster.master()->Shutdown();

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/integration-tests/minidump_generation-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/minidump_generation-itest.cc b/src/kudu/integration-tests/minidump_generation-itest.cc
index 46c8f23..a442564 100644
--- a/src/kudu/integration-tests/minidump_generation-itest.cc
+++ b/src/kudu/integration-tests/minidump_generation-itest.cc
@@ -41,7 +41,7 @@ class MinidumpGenerationITest : public ExternalMiniClusterITestBase {
 };
 
 void MinidumpGenerationITest::WaitForMinidumps(int expected, const string& dir) {
-  AssertEventually([&] {
+  ASSERT_EVENTUALLY([&] {
     vector<string> matches;
     ASSERT_OK(env_->Glob(JoinPathSegments(dir, "*.dmp"), &matches));
     ASSERT_EQ(expected, matches.size());

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/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 0f15d84..8d453b4 100644
--- a/src/kudu/integration-tests/raft_consensus-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest.cc
@@ -676,7 +676,7 @@ TEST_F(RaftConsensusITest, TestCatchupAfterOpsEvicted) {
 
   // Once the follower has caught up, all replicas should eventually GC the earlier
   // log segments that they were retaining.
-  AssertEventually([&]() {
+  ASSERT_EVENTUALLY([&]() {
       for (int i = 0; i < cluster_->num_tablet_servers(); i++) {
         SCOPED_TRACE(Substitute("TS $0", i));
         int num_wals = inspect_->CountFilesInWALDirForTS(i, tablet_id_, "wal-*");
@@ -1304,7 +1304,7 @@ TEST_F(RaftConsensusITest, TestLMPMismatchOnRestartedReplica) {
 
   // The COMMIT messages end up in the WAL asynchronously, so loop reading the
   // tablet server's WAL until it shows up.
-  AssertEventually([&]() {
+  ASSERT_EVENTUALLY([&]() {
       LogVerifier lv(cluster_.get());
       OpId commit;
       ASSERT_OK(lv.ScanForHighestCommittedOpIdInLog(replica_ets, tablet_id_, &commit));
@@ -1317,13 +1317,13 @@ TEST_F(RaftConsensusITest, TestLMPMismatchOnRestartedReplica) {
 
   // Send an operation 3.4 with preceding OpId 3.3.
   // We expect an LMP mismatch, since the replica has operation 2.3.
-  // We use 'AssertEventually' here because the replica
+  // We use 'ASSERT_EVENTUALLY' here because the replica
   // may need a few retries while it's in BOOTSTRAPPING state.
   req.set_caller_term(3);
   req.mutable_preceding_id()->CopyFrom(MakeOpId(3, 3));
   req.clear_ops();
   AddOp(MakeOpId(3, 4), &req);
-  AssertEventually([&]() {
+  ASSERT_EVENTUALLY([&]() {
       rpc.Reset();
       ASSERT_OK(c_proxy->UpdateConsensus(req, &resp, &rpc));
       ASSERT_EQ(resp.status().error().code(),
@@ -1390,9 +1390,9 @@ TEST_F(RaftConsensusITest, TestReplaceOperationStuckInPrepareQueue) {
   ASSERT_FALSE(resp.has_error()) << SecureDebugString(resp);
 
   // Ensure we can read the data.
-  // We need to AssertEventually here because otherwise it's possible to read the old value
+  // We need to ASSERT_EVENTUALLY here because otherwise it's possible to read the old value
   // of row '1', if the operation is still in flight.
-  AssertEventually([&]() {
+  ASSERT_EVENTUALLY([&]() {
       vector<string> results;
       NO_FATALS(WaitForRowCount(replica_ts->tserver_proxy.get(), 2, &results));
       ASSERT_EQ("(int32 key=1, int32 int_val=3, string string_val=\"term: 3 index: 4\")",

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/integration-tests/registration-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/registration-test.cc b/src/kudu/integration-tests/registration-test.cc
index fa91d7a..1290710 100644
--- a/src/kudu/integration-tests/registration-test.cc
+++ b/src/kudu/integration-tests/registration-test.cc
@@ -217,16 +217,16 @@ TEST_F(RegistrationTest, TestTabletReports) {
 // from the master.
 TEST_F(RegistrationTest, TestTSGetsSignedX509Certificate) {
   MiniTabletServer* ts = cluster_->mini_tablet_server(0);
-  AssertEventually([&](){
+  ASSERT_EVENTUALLY([&](){
       ASSERT_TRUE(ts->server()->tls_context().has_signed_cert());
-    }, MonoDelta::FromSeconds(10));
+    });
 }
 
 // Check that after the tablet server registers, it gets the list of valid
 // public token signing keys.
 TEST_F(RegistrationTest, TestTSGetsTskList) {
   MiniTabletServer* ts = cluster_->mini_tablet_server(0);
-  AssertEventually([&](){
+  ASSERT_EVENTUALLY([&](){
       ASSERT_FALSE(ts->server()->token_verifier().ExportKeys().empty());
     });
 }
@@ -253,7 +253,7 @@ TEST_F(RegistrationTest, TestExposeHttpsURLs) {
 
   // Need "eventually" here because the tserver may take a few seconds
   // to re-register while starting up.
-  AssertEventually([&](){
+  ASSERT_EVENTUALLY([&](){
       string contents;
       NO_FATALS(CheckTabletServersPage(&contents));
       ASSERT_STR_MATCHES(contents, expected_url_regex);

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/integration-tests/tablet_history_gc-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/tablet_history_gc-itest.cc b/src/kudu/integration-tests/tablet_history_gc-itest.cc
index 93e780a..4a40a6c 100644
--- a/src/kudu/integration-tests/tablet_history_gc-itest.cc
+++ b/src/kudu/integration-tests/tablet_history_gc-itest.cc
@@ -174,7 +174,7 @@ TEST_F(TabletHistoryGcITest, TestUndoDeltaBlockGc) {
   // no more undo deltas.
   HybridClock* c = down_cast<HybridClock*>(tablet->clock().get());
   AddTimeToHybridClock(c, MonoDelta::FromSeconds(FLAGS_tablet_history_max_age_sec));
-  AssertEventually([&] {
+  ASSERT_EVENTUALLY([&] {
     ASSERT_EQ(0, tablet->CountUndoDeltasForTests());
   });
 
@@ -199,7 +199,7 @@ TEST_F(TabletHistoryGcITest, TestUndoDeltaBlockGc) {
   ASSERT_EQ(kNumRows, num_rows_scanned);
 
   // Check that the tablet metrics have reasonable values.
-  AssertEventually([&] {
+  ASSERT_EVENTUALLY([&] {
     ASSERT_GT(tablet->metrics()->undo_delta_block_gc_init_duration->TotalCount(), 0);
     ASSERT_GT(tablet->metrics()->undo_delta_block_gc_delete_duration->TotalCount(), 0);
     ASSERT_GT(tablet->metrics()->undo_delta_block_gc_perform_duration->TotalCount(), 0);

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/integration-tests/token_signer-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/token_signer-itest.cc b/src/kudu/integration-tests/token_signer-itest.cc
index 4e4c828..37e829f 100644
--- a/src/kudu/integration-tests/token_signer-itest.cc
+++ b/src/kudu/integration-tests/token_signer-itest.cc
@@ -251,6 +251,7 @@ TEST_F(TokenSignerITest, AuthnTokenLifecycle) {
             VerifyTokenSignature(stoken, &token);
         ASSERT_EQ(VerificationResult::VALID, res);
     }, MonoDelta::FromMilliseconds(5L * FLAGS_heartbeat_interval_ms));
+    NO_PENDING_FATALS();
   }
 
   // Get closer to the very end of the TSK's activity interval and generate

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/integration-tests/ts_recovery-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/ts_recovery-itest.cc b/src/kudu/integration-tests/ts_recovery-itest.cc
index 1b60be1..bbc3c11 100644
--- a/src/kudu/integration-tests/ts_recovery-itest.cc
+++ b/src/kudu/integration-tests/ts_recovery-itest.cc
@@ -278,7 +278,7 @@ TEST_F(TsRecoveryITest, TestChangeMaxCellSize) {
   ASSERT_OK(itest::CreateTabletServerMap(cluster_->master_proxy().get(),
                                          cluster_->messenger(),
                                          &ts_map));
-  AssertEventually([&]() {
+  ASSERT_EVENTUALLY([&]() {
       vector<tserver::ListTabletsResponsePB::StatusAndSchemaPB> tablets;
       ASSERT_OK(ListTablets(ts_map[ts->uuid()], MonoDelta::FromSeconds(10), &tablets));
       ASSERT_EQ(1, tablets.size());
@@ -376,7 +376,7 @@ TEST_F(TsRecoveryITestDeathTest, TestRecoverFromOpIdOverflow) {
   ASSERT_OK(cluster_->Restart());
 
   OpId last_written_opid;
-  AssertEventually([&] {
+  ASSERT_EVENTUALLY([&] {
     // Tablet bootstrap should have converted the negative OpIds to positive ones.
     ASSERT_OK(itest::GetLastOpIdForReplica(tablet_id, ts, RECEIVED_OPID, MonoDelta::FromSeconds(5),
                                            &last_written_opid));
@@ -386,7 +386,6 @@ TEST_F(TsRecoveryITestDeathTest, TestRecoverFromOpIdOverflow) {
     expected_opid.set_index(static_cast<int64_t>(INT32_MAX) + kNumOverflowedEntriesToWrite);
     ASSERT_OPID_EQ(expected_opid, last_written_opid);
   });
-  NO_FATALS();
 
   // Now, write some records that will have a higher opid than INT32_MAX and
   // ensure they get written. This checks for overflows in the write path.
@@ -405,7 +404,7 @@ TEST_F(TsRecoveryITestDeathTest, TestRecoverFromOpIdOverflow) {
 
   // Validate.
   OpId prev_written_opid = last_written_opid;
-  AssertEventually([&] {
+  ASSERT_EVENTUALLY([&] {
     ASSERT_OK(itest::GetLastOpIdForReplica(tablet_id, ts, RECEIVED_OPID, MonoDelta::FromSeconds(5),
                                            &last_written_opid));
     ASSERT_TRUE(last_written_opid.IsInitialized());
@@ -415,7 +414,6 @@ TEST_F(TsRecoveryITestDeathTest, TestRecoverFromOpIdOverflow) {
     ASSERT_GT(last_written_opid.term(), prev_written_opid.term());
     ASSERT_GT(last_written_opid.index(), prev_written_opid.index());
   });
-  NO_FATALS();
   NO_FATALS(cluster_->AssertNoCrashes());
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/master/master-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index 3777513..143797e 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -1339,7 +1339,7 @@ TEST_F(MasterTest, TestConnectToMaster) {
 // Test that the master signs its on server certificate when it becomes the leader,
 // and also that it loads TSKs into the messenger's verifier.
 TEST_F(MasterTest, TestSignOwnCertAndLoadTSKs) {
-  AssertEventually([&]() {
+  ASSERT_EVENTUALLY([&]() {
       ASSERT_TRUE(master_->tls_context().has_signed_cert());
       ASSERT_GT(master_->messenger()->token_verifier().GetMaxKnownKeySequenceNumber(), -1);
     });

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/rpc/exactly_once_rpc-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/exactly_once_rpc-test.cc b/src/kudu/rpc/exactly_once_rpc-test.cc
index 792b0e6..388919d 100644
--- a/src/kudu/rpc/exactly_once_rpc-test.cc
+++ b/src/kudu/rpc/exactly_once_rpc-test.cc
@@ -581,6 +581,7 @@ TEST_F(ExactlyOnceRpcTest, TestExactlyOnceSemanticsGarbageCollectionStressTest)
   AssertEventually([&]() {
       ASSERT_EQ(0, mem_tracker_->consumption());
     }, MonoDelta::FromSeconds(5));
+  NO_PENDING_FATALS();
 }
 
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/tablet/diskrowset-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/diskrowset-test.cc b/src/kudu/tablet/diskrowset-test.cc
index c4e5ff7..3b68307 100644
--- a/src/kudu/tablet/diskrowset-test.cc
+++ b/src/kudu/tablet/diskrowset-test.cc
@@ -172,31 +172,26 @@ TEST_F(TestRowSet, TestRandomRead) {
   ASSERT_OK(OpenTestRowSet(&rs));
 
   // Read un-updated row.
-  VerifyRandomRead(*rs, "hello 000000000000050",
-                   R"((string key="hello 000000000000050", uint32 val=50))");
-  NO_FATALS();
+  NO_FATALS(VerifyRandomRead(*rs, "hello 000000000000050",
+                             R"((string key="hello 000000000000050", uint32 val=50))"));
 
   // Update the row.
   OperationResultPB result;
   ASSERT_OK(UpdateRow(rs.get(), 50, 12345, &result));
 
   // Read it again -- should see the updated value.
-  VerifyRandomRead(*rs, "hello 000000000000050",
-                   R"((string key="hello 000000000000050", uint32 val=12345))");
-  NO_FATALS();
+  NO_FATALS(VerifyRandomRead(*rs, "hello 000000000000050",
+                             R"((string key="hello 000000000000050", uint32 val=12345))"));
 
   // Try to read a row which comes before the first key.
   // This should return no rows.
-  VerifyRandomRead(*rs, "aaaaa", "");
-  NO_FATALS();
+  NO_FATALS(VerifyRandomRead(*rs, "aaaaa", ""));
 
   // Same with a row which falls between keys.
-  VerifyRandomRead(*rs, "hello 000000000000050_between_keys", "");
-  NO_FATALS();
+  NO_FATALS(VerifyRandomRead(*rs, "hello 000000000000050_between_keys", ""));
 
   // And a row which falls after the last key.
-  VerifyRandomRead(*rs, "hello 000000000000101", "");
-  NO_FATALS();
+  NO_FATALS(VerifyRandomRead(*rs, "hello 000000000000101", ""));
 }
 
 // Test Delete() support within a DiskRowSet.

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/tablet/tablet-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet-test.cc b/src/kudu/tablet/tablet-test.cc
index 564ca3e..d20862e 100644
--- a/src/kudu/tablet/tablet-test.cc
+++ b/src/kudu/tablet/tablet-test.cc
@@ -543,7 +543,7 @@ TYPED_TEST(TestTablet, TestRowIteratorOrdered) {
         new RunDuringDuplicatingRowSetPhase<decltype(RunScans)>(RunScans));
     this->tablet()->SetFlushCompactCommonHooksForTests(hooks_shared);
     ASSERT_OK(this->tablet()->Compact(Tablet::FORCE_COMPACT_ALL));
-    NO_FATALS();
+    NO_PENDING_FATALS();
   }
 
   {

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/tools/kudu-admin-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-admin-test.cc b/src/kudu/tools/kudu-admin-test.cc
index 04573d6..ba27ed2 100644
--- a/src/kudu/tools/kudu-admin-test.cc
+++ b/src/kudu/tools/kudu-admin-test.cc
@@ -1005,7 +1005,7 @@ TEST_F(AdminCliTest, TestLeaderStepDown) {
   ASSERT_TRUE(s.ok() || not_currently_leader);
   if (s.ok()) {
     int64 new_term;
-    AssertEventually([&]() {
+    ASSERT_EVENTUALLY([&]() {
         ASSERT_OK(GetTermFromConsensus(tservers, tablet_id_,
                                        &new_term));
         ASSERT_GT(new_term, current_term);

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/util/debug-util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/debug-util-test.cc b/src/kudu/util/debug-util-test.cc
index a07c9c2..3122adb 100644
--- a/src/kudu/util/debug-util-test.cc
+++ b/src/kudu/util/debug-util-test.cc
@@ -92,7 +92,7 @@ TEST_F(DebugUtilTest, TestSignalStackTrace) {
 
   // We have to loop a little bit because it takes a little while for the thread
   // to start up and actually call our function.
-  AssertEventually([&]() {
+  ASSERT_EVENTUALLY([&]() {
       ASSERT_STR_CONTAINS(DumpThreadStack(t->tid()), "SleeperThread")
     });
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/util/env-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env-test.cc b/src/kudu/util/env-test.cc
index ee0b42a..6e40891 100644
--- a/src/kudu/util/env-test.cc
+++ b/src/kudu/util/env-test.cc
@@ -680,6 +680,7 @@ TEST_F(TestEnv, TestGetFileModifiedTime) {
     ASSERT_OK(env_->GetFileModifiedTime(writer->filename(), &after_time));
     ASSERT_LT(initial_time, after_time);
   }, MonoDelta::FromSeconds(5));
+  NO_PENDING_FATALS();
 }
 
 TEST_F(TestEnv, TestRWFile) {
@@ -786,7 +787,7 @@ TEST_F(TestEnv, TestGetSpaceInfoFreeBytes) {
 
   // Loop in case there are concurrent tests running that are modifying the
   // filesystem.
-  AssertEventually([&] {
+  ASSERT_EVENTUALLY([&] {
     if (env_->FileExists(kTestFilePath)) {
       ASSERT_OK(env_->DeleteFile(kTestFilePath)); // Clean up the previous iteration.
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/util/env_util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util-test.cc b/src/kudu/util/env_util-test.cc
index cee0ffa..78bb006 100644
--- a/src/kudu/util/env_util-test.cc
+++ b/src/kudu/util/env_util-test.cc
@@ -63,7 +63,7 @@ TEST_F(EnvUtilTest, TestDiskSpaceCheck) {
 
   // Check 1% reservation logic. We loop this in case there are other FS
   // operations happening concurrent with this test.
-  AssertEventually([&] {
+  ASSERT_EVENTUALLY([&] {
     SpaceInfo space_info;
     ASSERT_OK(env_->GetSpaceInfo(test_dir_, &space_info));
     // Try for 1 less byte than 1% free. This request should be rejected.

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/util/file_cache-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/file_cache-test.cc b/src/kudu/util/file_cache-test.cc
index d61a0fd..28a0e3e 100644
--- a/src/kudu/util/file_cache-test.cc
+++ b/src/kudu/util/file_cache-test.cc
@@ -83,7 +83,7 @@ class FileCacheTest : public KuduTest {
     ASSERT_EQ(initial_open_fds_ + num_expected_fds, CountOpenFds(env_));
 
     // The expiry thread may take some time to run.
-    AssertEventually([&]() {
+    ASSERT_EVENTUALLY([&]() {
       ASSERT_EQ(num_expected_descriptors, cache_->NumDescriptorsForTests());
     });
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/util/maintenance_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/maintenance_manager-test.cc b/src/kudu/util/maintenance_manager-test.cc
index 98dc9ba..a16213e 100644
--- a/src/kudu/util/maintenance_manager-test.cc
+++ b/src/kudu/util/maintenance_manager-test.cc
@@ -199,7 +199,7 @@ TEST_F(MaintenanceManagerTest, TestRegisterUnregister) {
   CHECK_OK(Thread::Create(
       "TestThread", "TestRegisterUnregister",
       boost::bind(&TestMaintenanceOp::set_remaining_runs, &op1, 1), &thread));
-  AssertEventually([&]() {
+  ASSERT_EVENTUALLY([&]() {
       ASSERT_EQ(op1.DurationHistogram()->TotalCount(), 1);
     });
   manager_->UnregisterOp(&op1);
@@ -218,7 +218,7 @@ TEST_F(MaintenanceManagerTest, TestNewOpsDontGetScheduledDuringUnregister) {
   manager_->RegisterOp(&op1);
 
   // Wait until two instances of the ops start running, since we have two MM threads.
-  AssertEventually([&]() {
+  ASSERT_EVENTUALLY([&]() {
       ASSERT_EQ(op1.RunningGauge()->value(), 2);
     });
 
@@ -244,7 +244,7 @@ TEST_F(MaintenanceManagerTest, TestMemoryPressure) {
   // Fake that the server is under memory pressure.
   indicate_memory_pressure_ = true;
 
-  AssertEventually([&]() {
+  ASSERT_EVENTUALLY([&]() {
       ASSERT_EQ(op.DurationHistogram()->TotalCount(), 1);
     });
   manager_->UnregisterOp(&op);
@@ -303,7 +303,7 @@ TEST_F(MaintenanceManagerTest, TestCompletedOpsHistory) {
     op.set_ram_anchored(100);
     manager_->RegisterOp(&op);
 
-    AssertEventually([&]() {
+    ASSERT_EVENTUALLY([&]() {
         ASSERT_EQ(op.DurationHistogram()->TotalCount(), 1);
       });
     manager_->UnregisterOp(&op);

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/util/minidump-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/minidump-test.cc b/src/kudu/util/minidump-test.cc
index 2b39d97..f4c44e2 100644
--- a/src/kudu/util/minidump-test.cc
+++ b/src/kudu/util/minidump-test.cc
@@ -45,7 +45,7 @@ class MinidumpDeathTest : public KuduTest {
 };
 
 void MinidumpDeathTest::WaitForMinidumps(int expected, const string& dir) {
-  AssertEventually([&] {
+  ASSERT_EVENTUALLY([&] {
     vector<string> matches;
     ASSERT_OK(env_->Glob(JoinPathSegments(dir, "*.dmp"), &matches));
     ASSERT_EQ(expected, matches.size());

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/util/test_macros.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/test_macros.h b/src/kudu/util/test_macros.h
index 7da83e5..407de18 100644
--- a/src/kudu/util/test_macros.h
+++ b/src/kudu/util/test_macros.h
@@ -21,7 +21,13 @@
 #include <string>
 
 // ASSERT_NO_FATAL_FAILURE is just too long to type.
-#define NO_FATALS ASSERT_NO_FATAL_FAILURE
+#define NO_FATALS(expr) \
+  ASSERT_NO_FATAL_FAILURE(expr)
+
+// Detect fatals in the surrounding scope. NO_FATALS() only checks for fatals
+// in the expression passed to it.
+#define NO_PENDING_FATALS() \
+  if (testing::Test::HasFatalFailure()) { return; }
 
 #define ASSERT_OK(status) do { \
   const Status& _s = status;        \

http://git-wip-us.apache.org/repos/asf/kudu/blob/50b84ae6/src/kudu/util/test_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/test_util.h b/src/kudu/util/test_util.h
index 9ff9275..061fb3a 100644
--- a/src/kudu/util/test_util.h
+++ b/src/kudu/util/test_util.h
@@ -28,6 +28,11 @@
 #include "kudu/util/monotime.h"
 #include "kudu/util/test_macros.h"
 
+#define ASSERT_EVENTUALLY(expr) do { \
+  AssertEventually(expr); \
+  NO_PENDING_FATALS(); \
+} while (0)
+
 namespace kudu {
 
 extern const char* kInvalidPath;
@@ -94,6 +99,10 @@ std::string GetTestDataDirectory();
 //
 // The function is run in a loop with exponential backoff, capped at once
 // a second.
+//
+// To check whether AssertEventually() eventually succeeded, call
+// NO_PENDING_FATALS() afterward, or use ASSERT_EVENTUALLY() which performs
+// this check automatically.
 void AssertEventually(const std::function<void(void)>& f,
                       const MonoDelta& timeout = MonoDelta::FromSeconds(30));