You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2022/10/12 04:31:09 UTC

[kudu] 01/02: [test] make master_authz-itest more robust

This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 54c58fb8c521df6f220c097d15ca70c473beed7f
Author: kedeng <kd...@gmail.com>
AuthorDate: Thu Sep 29 11:45:08 2022 +0800

    [test] make master_authz-itest more robust
    
    I note the master_authz-itest unit-test may fail like this :
    `
    I20220929 10:56:08.093688   728 heartbeater.cc:466] Master 127.31.156.254:23975 was elected leader, sending a full tablet report...
    9月 29 10:56:16 10-120-18-85 krb5kdc[32375](info): TGS_REQ (2 etypes {17 16}) 127.0.0.1: ISSUE: authtime 1664420166, etypes {rep=17 tkt=17 ses=17}, kudu/127.31.156.254@KRBTEST.COM for HTTP/127.31.156.212@KRBTEST.COM
    I20220929 10:56:17.372444   532 master_service.cc:901] request to refresh authz privileges cache from {username='test-admin', principal='test-admin@KRBTEST.COM'} at 127.0.0.1:17302
    /data/github-kudu/kudu/src/kudu/integration-tests/master_authz-itest.cc:597: Failure
    Failed
    Bad status: Timed out: RefreshAuthzCache RPC to 127.31.156.254:23975 timed out after 10.000s (SENT)
    /data/github-kudu/kudu/src/kudu/integration-tests/master_authz-itest.cc:1083: Failure
    Expected: SetUpCluster(std::get<0>(GetParam())) doesn't generate new fatal failures in the current thread.
      Actual: it does.
    I20220929 10:56:27.373370 32371 external_mini_cluster-itest-base.cc:80] Found fatal failure
    `
    I have tested this case many times and found that all the failure is due to timeout.
    So I add retry policy for timeout to avoid direct failure in case of timeout.
    
    Change-Id: I2da13d18a8d7d6bcff5a41cdbad6d6ade6d528f3
    Reviewed-on: http://gerrit.cloudera.org:8080/19053
    Tested-by: Kudu Jenkins
    Reviewed-by: Yingchun Lai <ac...@gmail.com>
---
 src/kudu/integration-tests/master_authz-itest.cc | 214 +++++++++++------------
 1 file changed, 107 insertions(+), 107 deletions(-)

diff --git a/src/kudu/integration-tests/master_authz-itest.cc b/src/kudu/integration-tests/master_authz-itest.cc
index 16a5ef47b..cb8857e88 100644
--- a/src/kudu/integration-tests/master_authz-itest.cc
+++ b/src/kudu/integration-tests/master_authz-itest.cc
@@ -22,6 +22,7 @@
 #include <string>
 #include <thread>
 #include <tuple>
+#include <type_traits>
 #include <unordered_set>
 #include <utility>
 #include <vector>
@@ -33,14 +34,13 @@
 #include "kudu/client/schema.h"
 #include "kudu/client/shared_ptr.h" // IWYU pragma: keep
 #include "kudu/common/table_util.h"
-#include "kudu/common/wire_protocol.h"
 #include "kudu/gutil/stl_util.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/hms/hms_client.h"
 #include "kudu/integration-tests/cluster_itest_util.h"
 #include "kudu/integration-tests/external_mini_cluster-itest-base.h"
 #include "kudu/master/master.pb.h"
-#include "kudu/master/master.proxy.h"
+#include "kudu/master/master.proxy.h" // IWYU pragma: keep
 #include "kudu/mini-cluster/external_mini_cluster.h"
 #include "kudu/ranger/mini_ranger.h"
 #include "kudu/ranger/ranger.pb.h"
@@ -163,16 +163,16 @@ class MasterAuthzITestHarness {
                                     table_id, &table_locations);
   }
 
-  static Status RefreshAuthzPolicies(const unique_ptr<ExternalMiniCluster>& cluster) {
+  static void RefreshAuthzPolicies(const unique_ptr<ExternalMiniCluster>& cluster) {
     RefreshAuthzCacheRequestPB req;
     RefreshAuthzCacheResponsePB resp;
-    RpcController rpc;
-    rpc.set_timeout(MonoDelta::FromSeconds(10));
-    RETURN_NOT_OK(cluster->master_proxy()->RefreshAuthzCache(req, &resp, &rpc));
-    if (resp.has_error()) {
-      return StatusFromPB(resp.error().status());
-    }
-    return Status::OK();
+
+    ASSERT_EVENTUALLY([&] {
+      RpcController rpc;
+      rpc.set_timeout(MonoDelta::FromSeconds(10));
+      ASSERT_OK(cluster->master_proxy()->RefreshAuthzCache(req, &resp, &rpc));
+      ASSERT_FALSE(resp.has_error());
+    });
   }
 
   static Status IsCreateTableDone(const OperationParams& p,
@@ -332,30 +332,30 @@ class MasterAuthzITestHarness {
     return opts;
   }
 
-  virtual Status GrantCreateTablePrivilege(const PrivilegeParams& p,
-                                           const unique_ptr<ExternalMiniCluster>& cluster) = 0;
-  virtual Status GrantDropTablePrivilege(const PrivilegeParams& p,
+  virtual void GrantCreateTablePrivilege(const PrivilegeParams& p,
                                          const unique_ptr<ExternalMiniCluster>& cluster) = 0;
-  virtual Status GrantAlterTablePrivilege(const PrivilegeParams& p,
-                                          const unique_ptr<ExternalMiniCluster>& cluster) = 0;
-  virtual Status GrantAlterWithGrantTablePrivilege(
+  virtual void GrantDropTablePrivilege(const PrivilegeParams& p,
+                                       const unique_ptr<ExternalMiniCluster>& cluster) = 0;
+  virtual void GrantAlterTablePrivilege(const PrivilegeParams& p,
+                                        const unique_ptr<ExternalMiniCluster>& cluster) = 0;
+  virtual void GrantAlterWithGrantTablePrivilege(
       const PrivilegeParams& p,
       const unique_ptr<ExternalMiniCluster>& cluster) = 0;
-  virtual Status GrantRenameTablePrivilege(const PrivilegeParams& p,
-                                           const unique_ptr<ExternalMiniCluster>& cluster) = 0;
-  virtual Status GrantGetMetadataTablePrivilege(const PrivilegeParams& p,
-                                                const unique_ptr<ExternalMiniCluster>& cluster) = 0;
-  virtual Status GrantGetMetadataDatabasePrivilege(const PrivilegeParams& p,
-                                                   const unique_ptr<ExternalMiniCluster>& cluster)
-    = 0;
-  virtual Status GrantAllTablePrivilege(const PrivilegeParams& p,
-                                        const unique_ptr<ExternalMiniCluster>& cluster) = 0;
-  virtual Status GrantAllDatabasePrivilege(const PrivilegeParams& p,
-                                           const unique_ptr<ExternalMiniCluster>& cluster) = 0;
-  virtual Status GrantAllWithGrantTablePrivilege(const PrivilegeParams& p,
+  virtual void GrantRenameTablePrivilege(const PrivilegeParams& p,
+                                         const unique_ptr<ExternalMiniCluster>& cluster) = 0;
+  virtual void GrantGetMetadataTablePrivilege(const PrivilegeParams& p,
+                                              const unique_ptr<ExternalMiniCluster>& cluster) = 0;
+  virtual void GrantGetMetadataDatabasePrivilege(const PrivilegeParams& p,
                                                  const unique_ptr<ExternalMiniCluster>& cluster)
     = 0;
-  virtual Status GrantAllWithGrantDatabasePrivilege(
+  virtual void GrantAllTablePrivilege(const PrivilegeParams& p,
+                                      const unique_ptr<ExternalMiniCluster>& cluster) = 0;
+  virtual void GrantAllDatabasePrivilege(const PrivilegeParams& p,
+                                         const unique_ptr<ExternalMiniCluster>& cluster) = 0;
+  virtual void GrantAllWithGrantTablePrivilege(const PrivilegeParams& p,
+                                               const unique_ptr<ExternalMiniCluster>& cluster)
+    = 0;
+  virtual void GrantAllWithGrantDatabasePrivilege(
       const PrivilegeParams& p,
       const unique_ptr<ExternalMiniCluster>& cluster) = 0;
   virtual Status CreateTable(const OperationParams& p,
@@ -386,25 +386,25 @@ class RangerITestHarness : public MasterAuthzITestHarness {
  public:
   static constexpr int kSleepAfterNewPolicyMs = 1400;
 
-  Status GrantCreateTablePrivilege(const PrivilegeParams& p,
-                                   const unique_ptr<ExternalMiniCluster>& cluster) override {
+  void GrantCreateTablePrivilege(const PrivilegeParams& p,
+                                 const unique_ptr<ExternalMiniCluster>& cluster) override {
     AuthorizationPolicy policy;
     policy.databases.emplace_back(p.db_name);
     // IsCreateTableDone() requires METADATA on the table level.
     policy.tables.emplace_back("*");
     policy.items.emplace_back(PolicyItem({p.user_name}, {ActionPB::CREATE}, false));
-    RETURN_NOT_OK(ranger_->AddPolicy(move(policy)));
-    return RefreshAuthzPolicies(cluster);
+    ASSERT_OK(ranger_->AddPolicy(move(policy)));
+    NO_FATALS(RefreshAuthzPolicies(cluster));
   }
 
-  Status GrantDropTablePrivilege(const PrivilegeParams& p,
-                                 const unique_ptr<ExternalMiniCluster>& cluster) override {
+  void GrantDropTablePrivilege(const PrivilegeParams& p,
+                               const unique_ptr<ExternalMiniCluster>& cluster) override {
     AuthorizationPolicy policy;
     policy.databases.emplace_back(p.db_name);
     policy.tables.emplace_back(p.table_name);
     policy.items.emplace_back(PolicyItem({p.user_name}, {ActionPB::DROP}, false));
-    RETURN_NOT_OK(ranger_->AddPolicy(move(policy)));
-    return RefreshAuthzPolicies(cluster);
+    ASSERT_OK(ranger_->AddPolicy(move(policy)));
+    NO_FATALS(RefreshAuthzPolicies(cluster));
   }
 
   void CheckTableDoesNotExist(const string& database_name, const string& table_name,
@@ -414,111 +414,111 @@ class RangerITestHarness : public MasterAuthzITestHarness {
     ASSERT_TRUE(s.IsNotFound()) << s.ToString();
   }
 
-  Status GrantAlterTablePrivilege(const PrivilegeParams& p,
-                                  const unique_ptr<ExternalMiniCluster>& cluster) override {
+  void GrantAlterTablePrivilege(const PrivilegeParams& p,
+                                const unique_ptr<ExternalMiniCluster>& cluster) override {
     AuthorizationPolicy policy;
     policy.databases.emplace_back(p.db_name);
     policy.tables.emplace_back(p.table_name);
     policy.items.emplace_back(PolicyItem({p.user_name}, {ActionPB::ALTER}, false));
-    RETURN_NOT_OK(ranger_->AddPolicy(move(policy)));
-    return RefreshAuthzPolicies(cluster);
+    ASSERT_OK(ranger_->AddPolicy(move(policy)));
+    NO_FATALS(RefreshAuthzPolicies(cluster));
   }
 
-  Status GrantAlterWithGrantTablePrivilege(
+  void GrantAlterWithGrantTablePrivilege(
       const PrivilegeParams& p,
       const unique_ptr<ExternalMiniCluster>& cluster) override {
     AuthorizationPolicy policy;
     policy.databases.emplace_back(p.db_name);
     policy.tables.emplace_back(p.table_name);
     policy.items.emplace_back(PolicyItem({p.user_name}, {ActionPB::ALTER}, true));
-    RETURN_NOT_OK(ranger_->AddPolicy(move(policy)));
-    return RefreshAuthzPolicies(cluster);
+    ASSERT_OK(ranger_->AddPolicy(move(policy)));
+    NO_FATALS(RefreshAuthzPolicies(cluster));
   }
 
-  Status GrantRenameTablePrivilege(const PrivilegeParams& p,
-                                   const unique_ptr<ExternalMiniCluster>& cluster) override {
+  void GrantRenameTablePrivilege(const PrivilegeParams& p,
+                                 const unique_ptr<ExternalMiniCluster>& cluster) override {
     AuthorizationPolicy policy_new_table;
     policy_new_table.databases.emplace_back(p.db_name);
     // IsCreateTableDone() requires METADATA on the table level.
     policy_new_table.tables.emplace_back("*");
     policy_new_table.items.emplace_back(PolicyItem({p.user_name}, {ActionPB::CREATE}, false));
-    RETURN_NOT_OK(ranger_->AddPolicy(move(policy_new_table)));
+    ASSERT_OK(ranger_->AddPolicy(move(policy_new_table)));
 
     AuthorizationPolicy policy;
     policy.databases.emplace_back(p.db_name);
     policy.tables.emplace_back(p.table_name);
     policy.items.emplace_back(PolicyItem({p.user_name}, {ActionPB::ALL}, false));
-    RETURN_NOT_OK(ranger_->AddPolicy(move(policy)));
-    return RefreshAuthzPolicies(cluster);
+    ASSERT_OK(ranger_->AddPolicy(move(policy)));
+    NO_FATALS(RefreshAuthzPolicies(cluster));
   }
 
-  Status GrantGetMetadataDatabasePrivilege(const PrivilegeParams& p,
-                                           const unique_ptr<ExternalMiniCluster>& cluster)
+  void GrantGetMetadataDatabasePrivilege(const PrivilegeParams& p,
+                                         const unique_ptr<ExternalMiniCluster>& cluster)
         override {
     AuthorizationPolicy policy;
     policy.databases.emplace_back(p.db_name);
     policy.items.emplace_back(PolicyItem({p.user_name}, {ActionPB::METADATA}, false));
-    RETURN_NOT_OK(ranger_->AddPolicy(move(policy)));
-    return RefreshAuthzPolicies(cluster);
+    ASSERT_OK(ranger_->AddPolicy(move(policy)));
+    NO_FATALS(RefreshAuthzPolicies(cluster));
   }
 
-  Status GrantGetMetadataTablePrivilege(const PrivilegeParams& p,
-                                        const unique_ptr<ExternalMiniCluster>& cluster) override {
+  void GrantGetMetadataTablePrivilege(const PrivilegeParams& p,
+                                      const unique_ptr<ExternalMiniCluster>& cluster) override {
     AuthorizationPolicy policy;
     policy.databases.emplace_back(p.db_name);
     policy.tables.emplace_back(p.table_name);
     policy.items.emplace_back(PolicyItem({p.user_name}, {ActionPB::METADATA}, false));
-    RETURN_NOT_OK(ranger_->AddPolicy(move(policy)));
-    return RefreshAuthzPolicies(cluster);
+    ASSERT_OK(ranger_->AddPolicy(move(policy)));
+    NO_FATALS(RefreshAuthzPolicies(cluster));
   }
 
-  Status GrantAllTablePrivilege(const PrivilegeParams& p,
-                                const unique_ptr<ExternalMiniCluster>& cluster) override {
+  void GrantAllTablePrivilege(const PrivilegeParams& p,
+                              const unique_ptr<ExternalMiniCluster>& cluster) override {
     AuthorizationPolicy policy;
     policy.databases.emplace_back(p.db_name);
     policy.tables.emplace_back(p.table_name);
     policy.items.emplace_back(PolicyItem({p.user_name}, {ActionPB::ALL}, false));
-    RETURN_NOT_OK(ranger_->AddPolicy(move(policy)));
-    return RefreshAuthzPolicies(cluster);
+    ASSERT_OK(ranger_->AddPolicy(move(policy)));
+    NO_FATALS(RefreshAuthzPolicies(cluster));
   }
 
-  Status GrantAllDatabasePrivilege(const PrivilegeParams& p,
-                                   const unique_ptr<ExternalMiniCluster>& cluster) override {
+  void GrantAllDatabasePrivilege(const PrivilegeParams& p,
+                                 const unique_ptr<ExternalMiniCluster>& cluster) override {
     AuthorizationPolicy db_policy;
     db_policy.databases.emplace_back(p.db_name);
     db_policy.items.emplace_back(PolicyItem({p.user_name}, {ActionPB::ALL}, false));
-    RETURN_NOT_OK(ranger_->AddPolicy(move(db_policy)));
+    ASSERT_OK(ranger_->AddPolicy(move(db_policy)));
     AuthorizationPolicy tbl_policy;
     tbl_policy.databases.emplace_back(p.db_name);
     tbl_policy.tables.emplace_back("*");
     tbl_policy.items.emplace_back(PolicyItem({p.user_name}, {ActionPB::ALL}, false));
-    RETURN_NOT_OK(ranger_->AddPolicy(move(tbl_policy)));
-    return RefreshAuthzPolicies(cluster);
+    ASSERT_OK(ranger_->AddPolicy(move(tbl_policy)));
+    NO_FATALS(RefreshAuthzPolicies(cluster));
   }
 
-  Status GrantAllWithGrantTablePrivilege(const PrivilegeParams& p,
-                                         const unique_ptr<ExternalMiniCluster>& cluster) override {
+  void GrantAllWithGrantTablePrivilege(const PrivilegeParams& p,
+                                       const unique_ptr<ExternalMiniCluster>& cluster) override {
     AuthorizationPolicy policy;
     policy.databases.emplace_back(p.db_name);
     policy.tables.emplace_back(p.table_name);
     policy.items.emplace_back(PolicyItem({p.user_name}, {ActionPB::ALL}, true));
-    RETURN_NOT_OK(ranger_->AddPolicy(move(policy)));
-    return RefreshAuthzPolicies(cluster);
+    ASSERT_OK(ranger_->AddPolicy(move(policy)));
+    NO_FATALS(RefreshAuthzPolicies(cluster));
   }
 
-  Status GrantAllWithGrantDatabasePrivilege(
+  void GrantAllWithGrantDatabasePrivilege(
       const PrivilegeParams& p,
       const unique_ptr<ExternalMiniCluster>& cluster) override {
     AuthorizationPolicy db_policy;
     db_policy.databases.emplace_back(p.db_name);
     db_policy.items.emplace_back(PolicyItem({p.user_name}, {ActionPB::ALL}, true));
-    RETURN_NOT_OK(ranger_->AddPolicy(move(db_policy)));
+    ASSERT_OK(ranger_->AddPolicy(move(db_policy)));
     AuthorizationPolicy tbl_policy;
     tbl_policy.databases.emplace_back(p.db_name);
     tbl_policy.tables.emplace_back("*");
     tbl_policy.items.emplace_back(PolicyItem({p.user_name}, {ActionPB::ALL}, true));
-    RETURN_NOT_OK(ranger_->AddPolicy(move(tbl_policy)));
-    return RefreshAuthzPolicies(cluster);
+    ASSERT_OK(ranger_->AddPolicy(move(tbl_policy)));
+    NO_FATALS(RefreshAuthzPolicies(cluster));
   }
 
   Status CreateTable(const OperationParams& p,
@@ -590,7 +590,7 @@ class MasterAuthzITestBase : public ExternalMiniClusterITestBase {
 
     ASSERT_OK(harness_->SetUpExternalServiceClients(cluster_));
     ASSERT_OK(harness_->SetUpCredentials());
-    ASSERT_OK(harness_->RefreshAuthzPolicies(cluster_));
+    NO_FATALS(harness_->RefreshAuthzPolicies(cluster_));
     ASSERT_OK(harness_->SetUpTables(cluster_, client_));
 
     // Log in as 'test-user' and reset the client to pick up the change in user.
@@ -608,48 +608,48 @@ class MasterAuthzITestBase : public ExternalMiniClusterITestBase {
     return harness_->GetTableLocationsWithTableId(table_name, table_id, cluster_);
   }
 
-  Status GrantCreateTablePrivilege(const PrivilegeParams& p) {
-    return harness_->GrantCreateTablePrivilege(p, cluster_);
+  void GrantCreateTablePrivilege(const PrivilegeParams& p) {
+    NO_FATALS(harness_->GrantCreateTablePrivilege(p, cluster_));
   }
 
-  Status GrantDropTablePrivilege(const PrivilegeParams& p) {
-    return harness_->GrantDropTablePrivilege(p, cluster_);
+  void GrantDropTablePrivilege(const PrivilegeParams& p) {
+    NO_FATALS(harness_->GrantDropTablePrivilege(p, cluster_));
   }
 
-  Status GrantAlterTablePrivilege(const PrivilegeParams& p) {
-    return harness_->GrantAlterTablePrivilege(p, cluster_);
+  void GrantAlterTablePrivilege(const PrivilegeParams& p) {
+    NO_FATALS(harness_->GrantAlterTablePrivilege(p, cluster_));
   }
 
-  Status GrantRenameTablePrivilege(const PrivilegeParams& p) {
-    return harness_->GrantRenameTablePrivilege(p, cluster_);
+  void GrantRenameTablePrivilege(const PrivilegeParams& p) {
+    NO_FATALS(harness_->GrantRenameTablePrivilege(p, cluster_));
   }
 
-  Status GrantGetMetadataTablePrivilege(const PrivilegeParams& p) {
-    return harness_->GrantGetMetadataTablePrivilege(p, cluster_);
+  void GrantGetMetadataTablePrivilege(const PrivilegeParams& p) {
+    NO_FATALS(harness_->GrantGetMetadataTablePrivilege(p, cluster_));
   }
 
-  Status GrantAlterWithGrantTablePrivilege(const PrivilegeParams& p) {
-    return harness_->GrantAlterWithGrantTablePrivilege(p, cluster_);
+  void GrantAlterWithGrantTablePrivilege(const PrivilegeParams& p) {
+    NO_FATALS(harness_->GrantAlterWithGrantTablePrivilege(p, cluster_));
   }
 
-  Status GrantAllTablePrivilege(const PrivilegeParams& p) {
-    return harness_->GrantAllTablePrivilege(p, cluster_);
+  void GrantAllTablePrivilege(const PrivilegeParams& p) {
+    NO_FATALS(harness_->GrantAllTablePrivilege(p, cluster_));
   }
 
-  Status GrantAllDatabasePrivilege(const PrivilegeParams& p) {
-    return harness_->GrantAllDatabasePrivilege(p, cluster_);
+  void GrantAllDatabasePrivilege(const PrivilegeParams& p) {
+    NO_FATALS(harness_->GrantAllDatabasePrivilege(p, cluster_));
   }
 
-  Status GrantAllWithGrantTablePrivilege(const PrivilegeParams& p) {
-    return harness_->GrantAllWithGrantTablePrivilege(p, cluster_);
+  void GrantAllWithGrantTablePrivilege(const PrivilegeParams& p) {
+    NO_FATALS(harness_->GrantAllWithGrantTablePrivilege(p, cluster_));
   }
 
-  Status GrantAllWithGrantDatabasePrivilege(const PrivilegeParams& p) {
-    return harness_->GrantAllWithGrantDatabasePrivilege(p, cluster_);
+  void GrantAllWithGrantDatabasePrivilege(const PrivilegeParams& p) {
+    NO_FATALS(harness_->GrantAllWithGrantDatabasePrivilege(p, cluster_));
   }
 
-  Status GrantGetMetadataDatabasePrivilege(const PrivilegeParams& p) {
-    return harness_->GrantGetMetadataDatabasePrivilege(p, cluster_);
+  void GrantGetMetadataDatabasePrivilege(const PrivilegeParams& p) {
+    NO_FATALS(harness_->GrantGetMetadataDatabasePrivilege(p, cluster_));
   }
 
   Status CreateTable(const OperationParams& p) {
@@ -878,12 +878,12 @@ TEST_P(MasterAuthzITest, TestAuthzListTables) {
 
   // Listing tables only shows the tables which user has proper privileges on.
   tables.clear();
-  ASSERT_OK(this->GrantGetMetadataTablePrivilege({ kDatabaseName, kTableName }));
+  NO_FATALS(this->GrantGetMetadataTablePrivilege({ kDatabaseName, kTableName }));
   ASSERT_OK(this->client_->ListTables(&tables));
   ASSERT_EQ(vector<string>({ table_name }), tables);
 
   tables.clear();
-  ASSERT_OK(this->GrantGetMetadataTablePrivilege(
+  NO_FATALS(this->GrantGetMetadataTablePrivilege(
       {kDatabaseName, kSecondTable, "{OWNER}"}));
   ASSERT_OK(this->client_->ListTables(&tables));
   unordered_set<string> tables_set(tables.begin(), tables.end());
@@ -897,8 +897,8 @@ TEST_P(MasterAuthzITest, TestAuthzListTablesConcurrentRename) {
       "catalog_manager_inject_latency_list_authz_ms", "3000"));;
   const auto table_name = Substitute("$0.$1", kDatabaseName, kTableName);
   const auto sec_table_name = Substitute("$0.$1", kDatabaseName, kSecondTable);
-  ASSERT_OK(this->GrantGetMetadataTablePrivilege({ kDatabaseName, kTableName }));
-  ASSERT_OK(this->GrantRenameTablePrivilege({ kDatabaseName, kTableName }));
+  NO_FATALS(this->GrantGetMetadataTablePrivilege({ kDatabaseName, kTableName }));
+  NO_FATALS(this->GrantRenameTablePrivilege({ kDatabaseName, kTableName }));
 
   // List the tables while injecting latency.
   vector<string> tables;
@@ -921,7 +921,7 @@ TEST_P(MasterAuthzITest, TestAuthzGiveAwayOwnership) {
   // We need to grant metadata permissions to the user, otherwise the ownership
   // change would time out due to lack of privileges when checking the alter
   // table progress.
-  this->GrantGetMetadataTablePrivilege({ kDatabaseName, kTableName, kTestUser });
+  NO_FATALS(this->GrantGetMetadataTablePrivilege({ kDatabaseName, kTableName, kTestUser }));
   const string table_name = Substitute("$0.$1", kDatabaseName, kTableName);
 
   // Change table owner.
@@ -1024,12 +1024,12 @@ TEST_P(MasterAuthzOwnerITest, TestMismatchedTable) {
   ASSERT_TRUE(s.IsNotAuthorized()) << s.ToString();
   ASSERT_STR_MATCHES(s.ToString(), "[Uu]nauthorized action");
 
-  ASSERT_OK(this->GrantGetMetadataTablePrivilege({ kDatabaseName, kTableName, kUsername }));
+  NO_FATALS(this->GrantGetMetadataTablePrivilege({ kDatabaseName, kTableName, kUsername }));
   s = this->GetTableLocationsWithTableId(table_name_b, table_id_a);
   ASSERT_TRUE(s.IsNotAuthorized());
   ASSERT_STR_MATCHES(s.ToString(), "[Uu]nauthorized action");
 
-  ASSERT_OK(this->GrantGetMetadataTablePrivilege({ kDatabaseName, kSecondTable, kUsername }));
+  NO_FATALS(this->GrantGetMetadataTablePrivilege({ kDatabaseName, kSecondTable, kUsername }));
   s = this->GetTableLocationsWithTableId(table_name_b, table_id_a);
   ASSERT_TRUE(s.IsNotFound());
   ASSERT_STR_CONTAINS(s.ToString(), "the table ID refers to a different table");
@@ -1042,7 +1042,7 @@ typedef function<Status(MasterAuthzITestBase*, const OperationParams&)> Operator
 // Functor that grants the required permission for an operation (e.g Create,
 // Rename) on a table given the database the table belongs to and the name of
 // the table, if applicable.
-typedef function<Status(MasterAuthzITestBase*, const PrivilegeParams&)> PrivilegeFunc;
+typedef function<void(MasterAuthzITestBase*, const PrivilegeParams&)> PrivilegeFunc;
 
 // A description of the operation function that describes a particular action
 // on a table a user can perform, as well as the privilege granting function
@@ -1101,7 +1101,7 @@ TEST_P(TestAuthzTable, TestAuthorizeTable) {
   ASSERT_STR_MATCHES(s.ToString(), "[Uu]nauthorized action");
 
   // User 'test-user' can operate on the table after obtaining proper privileges.
-  ASSERT_OK(desc.funcs.grant_privileges(this, privilege_params));
+  NO_FATALS(desc.funcs.grant_privileges(this, privilege_params));
   ASSERT_OK(desc.funcs.do_action(this, action_params));
 
   // Ensure that operating on a table while the Authz service is unreachable fails.
@@ -1263,7 +1263,7 @@ TEST_P(AuthzErrorHandlingTest, TestNonExistentTable) {
     ASSERT_OK(StartAuthzProvider());
   }
   ASSERT_EVENTUALLY([&] {
-    ASSERT_OK(funcs.grant_privileges(this, privilege_params));
+    NO_FATALS(funcs.grant_privileges(this, privilege_params));
   });
   s = funcs.do_action(this, action_params);
   ASSERT_TRUE(s.IsNotFound()) << s.ToString();