You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by aw...@apache.org on 2020/04/13 07:13:38 UTC

[kudu] branch master updated: KUDU-3078: parameterize ts_sentry-itest to use Ranger

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

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


The following commit(s) were added to refs/heads/master by this push:
     new e6c742e  KUDU-3078: parameterize ts_sentry-itest to use Ranger
e6c742e is described below

commit e6c742ec72d5da84554f1ab5f0e5c5603ab6307b
Author: Andrew Wong <aw...@cloudera.com>
AuthorDate: Fri Apr 10 20:43:21 2020 -0700

    KUDU-3078: parameterize ts_sentry-itest to use Ranger
    
    Following the example set in dacb8203f05ca2d873d524501214bfe5144ac9bb,
    this updates the test harness in ts_sentry-itest (now ts_authz-itest) to
    be parameterized by authz provider.
    
    Change-Id: I2232cfe0295cce83154cd01b5104c9d1b6ed955d
    Reviewed-on: http://gerrit.cloudera.org:8080/15714
    Reviewed-by: Hao Hao <ha...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/integration-tests/CMakeLists.txt          |   2 +-
 .../{ts_sentry-itest.cc => ts_authz-itest.cc}      | 337 +++++++++++++++------
 2 files changed, 247 insertions(+), 92 deletions(-)

diff --git a/src/kudu/integration-tests/CMakeLists.txt b/src/kudu/integration-tests/CMakeLists.txt
index ea63c21..75f2fb1 100644
--- a/src/kudu/integration-tests/CMakeLists.txt
+++ b/src/kudu/integration-tests/CMakeLists.txt
@@ -132,8 +132,8 @@ ADD_KUDU_TEST(tombstoned_voting-stress-test RUN_SERIAL true)
 ADD_KUDU_TEST(token_signer-itest)
 ADD_KUDU_TEST(location_assignment-itest
   DATA_FILES ../scripts/assign-location.py)
+ADD_KUDU_TEST(ts_authz-itest NUM_SHARDS 2)
 ADD_KUDU_TEST(ts_recovery-itest PROCESSORS 4)
-ADD_KUDU_TEST(ts_sentry-itest NUM_SHARDS 2)
 ADD_KUDU_TEST(ts_tablet_manager-itest)
 ADD_KUDU_TEST(update_scan_delta_compact-test RUN_SERIAL true)
 ADD_KUDU_TEST(webserver-crawl-itest LABELS no_dist_test)
diff --git a/src/kudu/integration-tests/ts_sentry-itest.cc b/src/kudu/integration-tests/ts_authz-itest.cc
similarity index 68%
rename from src/kudu/integration-tests/ts_sentry-itest.cc
rename to src/kudu/integration-tests/ts_authz-itest.cc
index 772aac3..2641010 100644
--- a/src/kudu/integration-tests/ts_sentry-itest.cc
+++ b/src/kudu/integration-tests/ts_authz-itest.cc
@@ -17,6 +17,7 @@
 
 #include <cstdlib>
 #include <memory>
+#include <ostream>
 #include <string>
 #include <thread>
 #include <type_traits>
@@ -39,13 +40,13 @@
 #include "kudu/gutil/stl_util.h"
 #include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
-#include "kudu/hms/hms_client.h"
-#include "kudu/hms/mini_hms.h"
 #include "kudu/integration-tests/data_gen_util.h"
 #include "kudu/integration-tests/external_mini_cluster-itest-base.h"
 #include "kudu/integration-tests/hms_itest-base.h"
 #include "kudu/master/sentry_authz_provider-test-base.h"
 #include "kudu/mini-cluster/external_mini_cluster.h"
+#include "kudu/ranger/mini_ranger.h"
+#include "kudu/ranger/ranger.pb.h"
 #include "kudu/security/test/mini_kdc.h"
 #include "kudu/sentry/mini_sentry.h"
 #include "kudu/sentry/sentry_client.h"
@@ -58,6 +59,7 @@
 #include "kudu/util/random_util.h"
 #include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/status.h"
+#include "kudu/util/string_case.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
@@ -83,6 +85,10 @@ using kudu::master::CreateRoleAndAddToGroups;
 using kudu::master::GetColumnPrivilege;
 using kudu::master::GetDatabasePrivilege;
 using kudu::master::GetTablePrivilege;
+using kudu::ranger::ActionPB;
+using kudu::ranger::AuthorizationPolicy;
+using kudu::ranger::PolicyItem;
+using kudu::ranger::MiniRanger;
 using kudu::sentry::SentryClient;
 using kudu::tablet::WritePrivileges;
 using kudu::tablet::WritePrivilegeType;
@@ -247,6 +253,7 @@ Status PerformAction(const RWPrivileges& privileges,
 // mini_sentry.cc). We expect Sentry to be aware of users "user[0-2]".
 constexpr int kNumUsers = 3;
 constexpr const char* kAdminGroup = "admin";
+constexpr const char* kAdminUser = "test-admin";
 
 constexpr int kNumTables = 3;
 constexpr int kNumColsPerTable = 3;
@@ -259,21 +266,18 @@ constexpr int kAuthzCacheTTLMultiplier = 3;
 
 } // anonymous namespace
 
-// These tests will use the HMS and Sentry, and thus, are very slow.
-// SKIP_IF_SLOW_NOT_ALLOWED() should be the very first thing called in the body
-// of every test based on this test class.
-class TSSentryITestHarness : public HmsITestHarness {
+// Test harness used to set up a cluster and grant privileges, that is agnostic
+// the underlying authorization service (e.g. Sentry, Ranger).
+class TSAuthzITestHarness {
  public:
+  virtual ~TSAuthzITestHarness() {}
+
   ExternalMiniClusterOptions GetClusterOpts() {
     ExternalMiniClusterOptions opts;
     opts.enable_kerberos = true;
-    opts.enable_sentry = true;
-    opts.hms_mode = HmsMode::ENABLE_METASTORE_INTEGRATION;
     // Set a low token timeout so we can ensure retries are working properly.
     opts.extra_master_flags.emplace_back(Substitute("--authz_token_validity_seconds=$0",
                                                     kAuthzTokenTTLSecs));
-    opts.extra_master_flags.emplace_back(Substitute("--sentry_privileges_cache_ttl_factor=$0",
-                                                    kAuthzCacheTTLMultiplier));
     // In addition to our users, we will be using the "kudu" user to perform
     // administrative tasks like creating tables.
     opts.extra_master_flags.emplace_back(
@@ -285,43 +289,24 @@ class TSSentryITestHarness : public HmsITestHarness {
     return opts;
   }
 
-  Status SetUpExternalServiceClients(const unique_ptr<ExternalMiniCluster>& cluster) {
-    // Set up the HMS client so we can set up a database.
-    thrift::ClientOptions hms_opts;
-    hms_opts.enable_kerberos = true;
-    hms_opts.service_principal = "hive";
-    hms_client_.reset(new hms::HmsClient(cluster->hms()->address(), hms_opts));
-    RETURN_NOT_OK(hms_client_->Start());
+  // Sets up the external services (e.g. Sentry, Ranger).
+  virtual void SetUpExternalMiniServiceOpts(ExternalMiniClusterOptions* opts) = 0;
+  virtual Status SetUpExternalServiceClients(const unique_ptr<ExternalMiniCluster>& cluster) = 0;
 
-    // Set up the Sentry client so we can set up privileges.
-    thrift::ClientOptions sentry_opts;
-    sentry_opts.enable_kerberos = true;
-    sentry_opts.service_principal = "sentry";
-    sentry_client_.reset(new SentryClient(cluster->sentry()->address(), sentry_opts));
-    RETURN_NOT_OK(sentry_client_->Start());
-
-    return Status::OK();
-  }
-
-  Status SetUpCredentials() {
-    RETURN_NOT_OK(CreateRoleAndAddToGroups(sentry_client_.get(), kAdminRole, kAdminGroup));
-    RETURN_NOT_OK(AlterRoleGrantPrivilege(sentry_client_.get(), kAdminRole,
-        GetDatabasePrivilege(kDb, "ALL", TSentryGrantOption::DISABLED)));
-
-    return Status::OK();
-  }
-
-  Status SetUpTables() {
-    // Create the database in the HMS.
-    RETURN_NOT_OK(CreateDatabase(kDb));
+  // Sets up roles and privileges so admins have full privileges to the
+  // database. Also set up any user mappings necessary.
+  virtual Status SetUpCredentials() = 0;
 
-    // Finally populate a set of column names to use for our tables.
-    for (int i = 0; i < kNumColsPerTable; i++) {
-      cols_.emplace_back(Substitute("col$0", i));
-    }
+  // Sets up table-related metadata (e.g. HMS database, columns).
+  virtual Status SetUpTables() = 0;
 
-    return Status::OK();
-  }
+  // Grants privileges on the given table/column. These abide by Sentry's
+  // hierarchical definition of privileges, so if granting privileges on a
+  // table, these grant privileges on the table and all columns in that table.
+  virtual Status GrantTablePrivilege(const string& db, const string& tbl, const string& user,
+                                     const string& action) = 0;
+  virtual Status GrantColumnPrivilege(const string& db, const string& tbl, const string& col,
+                                      const string& user, const string& action) = 0;
 
   // Creates a table named 'table_ident' with 'kNumColsPerTable' columns.
   Status CreateTable(const string& table_ident, const shared_ptr<KuduClient>& client) {
@@ -347,9 +332,6 @@ class TSSentryITestHarness : public HmsITestHarness {
     }
   }
 
-  // A Sentry client with which to grant privileges.
-  unique_ptr<SentryClient> sentry_client_;
-
   // A list of users that may try to do things.
   vector<string> users_;
 
@@ -357,28 +339,200 @@ class TSSentryITestHarness : public HmsITestHarness {
   vector<string> cols_;
 };
 
-class TSSentryITest : public ExternalMiniClusterITestBase {
+class SentryITestHarness : public TSAuthzITestHarness,
+                           public HmsITestHarness {
+ public:
+  void SetUpExternalMiniServiceOpts(ExternalMiniClusterOptions* opts) override {
+    opts->hms_mode = HmsMode::ENABLE_METASTORE_INTEGRATION;
+    opts->enable_sentry = true;
+    opts->extra_master_flags.emplace_back(Substitute("--sentry_privileges_cache_ttl_factor=$0",
+                                                    kAuthzCacheTTLMultiplier));
+  }
+
+  Status SetUpExternalServiceClients(const unique_ptr<ExternalMiniCluster>& cluster) override {
+    // Set up the HMS client so we can set up a database.
+    thrift::ClientOptions hms_opts;
+    hms_opts.enable_kerberos = true;
+    hms_opts.service_principal = "hive";
+    RETURN_NOT_OK(RestartHmsClient(cluster, hms_opts));
+
+    // Set up the Sentry client so we can set up privileges.
+    thrift::ClientOptions sentry_opts;
+    sentry_opts.enable_kerberos = true;
+    sentry_opts.service_principal = "sentry";
+    sentry_client_.reset(new SentryClient(cluster->sentry()->address(), sentry_opts));
+    return sentry_client_->Start();
+  }
+
+  Status SetUpCredentials() override {
+    RETURN_NOT_OK(CreateRoleAndAddToGroups(sentry_client_.get(), kAdminRole, kAdminGroup));
+    RETURN_NOT_OK(AlterRoleGrantPrivilege(sentry_client_.get(), kAdminRole,
+        GetDatabasePrivilege(kDb, "ALL", TSentryGrantOption::DISABLED)));
+
+    // Set up our users such that userN --> groupN --> roleN.
+    // NOTE: The user-group mappings are specified in mini_sentry.cc.
+    for (int i = 0; i < kNumUsers; i++) {
+      const string user = Substitute("user$0", i);
+      const string role = Substitute("role$0", i);
+      const string group = Substitute("group$0", i);
+      RETURN_NOT_OK(CreateRoleAndAddToGroups(sentry_client_.get(), role, group));
+      EmplaceOrDie(&role_by_user_, user, role);
+    }
+    return Status::OK();
+  }
+  Status SetUpTables() override {
+    // Create the database in the HMS.
+    RETURN_NOT_OK(CreateDatabase(kDb));
+    // Finally populate a set of column names to use for our tables.
+    for (int i = 0; i < kNumColsPerTable; i++) {
+      cols_.emplace_back(Substitute("col$0", i));
+    }
+    return Status::OK();
+  }
+  Status GrantTablePrivilege(const string& db, const string& tbl, const string& user,
+                             const string& action) override {
+    const auto& role = FindOrDie(role_by_user_, user);
+    return AlterRoleGrantPrivilege(sentry_client_.get(), role,
+                                   GetTablePrivilege(db, tbl, action));
+  }
+  Status GrantColumnPrivilege(const string& db, const string& tbl, const string& col,
+                              const string& user, const string& action) override {
+    const auto& role = FindOrDie(role_by_user_, user);
+    return AlterRoleGrantPrivilege(sentry_client_.get(), role,
+                                   GetColumnPrivilege(db, tbl, col, action));
+  }
+ private:
+  // A Sentry client with which to grant privileges.
+  unique_ptr<SentryClient> sentry_client_;
+
+  // Maintain a mapping between users and roles.
+  unordered_map<string, string> role_by_user_;
+};
+
+namespace {
+ActionPB StringToActionPB(string s) {
+  ToLowerCase(&s);
+  if (s == "select") return ActionPB::SELECT;
+  if (s == "insert") return ActionPB::INSERT;
+  if (s == "update") return ActionPB::UPDATE;
+  if (s == "delete") return ActionPB::DELETE;
+  if (s == "alter") return ActionPB::ALTER;
+  if (s == "create") return ActionPB::CREATE;
+  if (s == "drop") return ActionPB::DROP;
+  if (s == "all") return ActionPB::ALL;
+  if (s == "metadata") return ActionPB::METADATA;
+  LOG(FATAL) << "unknown ActionPB string";
+}
+} // anonymous namespace
+
+class RangerITestHarness : public TSAuthzITestHarness {
+ public:
+  static constexpr int kSleepAfterNewPolicyMs = 1200;
+  void SetUpExternalMiniServiceOpts(ExternalMiniClusterOptions* opts) override {
+    opts->enable_ranger = true;
+  }
+  Status SetUpExternalServiceClients(const unique_ptr<ExternalMiniCluster>& cluster) override {
+    ranger_ = cluster->ranger();
+    return Status::OK();
+  }
+  Status SetUpCredentials() override {
+    AuthorizationPolicy policy;
+    policy.databases = { kDb };
+    policy.tables = { "*" };
+    policy.columns = { "*" };
+    policy.items.emplace_back(PolicyItem({ kAdminUser }, { ActionPB::ALL }));
+    RETURN_NOT_OK(ranger_->AddPolicy(std::move(policy)));
+    SleepFor(MonoDelta::FromMilliseconds(kSleepAfterNewPolicyMs));
+    return Status::OK();
+  }
+  Status SetUpTables() override {
+    // Finally populate a set of column names to use for our tables.
+    for (int i = 0; i < kNumColsPerTable; i++) {
+      cols_.emplace_back(Substitute("col$0", i));
+    }
+    return Status::OK();
+  }
+  Status GrantTablePrivilege(const string& db, const string& tbl, const string& user,
+                             const string& action) override {
+    AuthorizationPolicy policy;
+    policy.databases = { db };
+    policy.tables = { tbl };
+    policy.columns = { "*" };
+    policy.items.emplace_back(PolicyItem({ user }, { StringToActionPB(action) }));
+    RETURN_NOT_OK(ranger_->AddPolicy(std::move(policy)));
+    SleepFor(MonoDelta::FromMilliseconds(kSleepAfterNewPolicyMs));
+    return Status::OK();
+  }
+  Status GrantColumnPrivilege(const string& db, const string& tbl, const string& col,
+                              const string& user, const string& action) override {
+    AuthorizationPolicy policy;
+    policy.databases = { db };
+    policy.tables = { tbl };
+    policy.columns = { col };
+    policy.items.emplace_back(PolicyItem({ user }, { StringToActionPB(action) }));
+    RETURN_NOT_OK(ranger_->AddPolicy(std::move(policy)));
+    SleepFor(MonoDelta::FromMilliseconds(kSleepAfterNewPolicyMs));
+    return Status::OK();
+  }
+ private:
+  MiniRanger* ranger_;
+};
+
+// TODO(awong): refactor so we can share code between master_authz-itest.
+enum HarnessEnum {
+  kSentry,
+  kRanger,
+};
+string HarnessEnumToString(HarnessEnum h) {
+  switch (h) {
+    case kSentry:
+      return "Sentry";
+    case kRanger:
+      return "Ranger";
+  }
+  return "";
+}
+
+// These tests will use the HMS and Sentry, and thus, are very slow.
+// SKIP_IF_SLOW_NOT_ALLOWED() should be the very first thing called in the body
+// of every test based on this test class.
+class TSAuthzITest : public ExternalMiniClusterITestBase,
+                     public ::testing::WithParamInterface<HarnessEnum> {
  public:
   void SetUp() override {
     SKIP_IF_SLOW_NOT_ALLOWED();
+    switch (GetParam()) {
+      case kSentry:
+        harness_.reset(new SentryITestHarness());
+        break;
+      case kRanger:
+        harness_.reset(new RangerITestHarness());
+        break;
+      default:
+        LOG(FATAL) << "unknown harness";
+    }
     ExternalMiniClusterITestBase::SetUp();
-    harness_.AddUsers();
-    ExternalMiniClusterOptions opts = harness_.GetClusterOpts();
+    harness_->AddUsers();
+    ExternalMiniClusterOptions opts = harness_->GetClusterOpts();
+    harness_->SetUpExternalMiniServiceOpts(&opts);
     NO_FATALS(StartClusterWithOpts(std::move(opts)));
 
     ASSERT_OK(cluster_->kdc()->CreateUserPrincipal("kudu"));
     ASSERT_OK(cluster_->kdc()->Kinit("kudu"));
 
-    ASSERT_OK(harness_.SetUpExternalServiceClients(cluster_));
-    ASSERT_OK(harness_.SetUpCredentials());
-    ASSERT_OK(harness_.SetUpTables());
+    ASSERT_OK(harness_->SetUpExternalServiceClients(cluster_));
+    ASSERT_OK(harness_->SetUpCredentials());
+    ASSERT_OK(harness_->SetUpTables());
 
-    // Create a client as the "kudu" user, who now has admin privileges.
-    ASSERT_OK(cluster_->CreateClient(nullptr, &admin_client_));
+    // Create a client as the admin user, who now has admin privileges on the
+    // database.
+    ASSERT_OK(cluster_->kdc()->CreateUserPrincipal(kAdminUser));
+    ASSERT_OK(cluster_->kdc()->Kinit(kAdminUser));
+    ASSERT_OK(cluster_->CreateClient(/*builder*/nullptr, &client_));
   }
 
   Status CreateTable(const string& table_ident) {
-    return harness_.CreateTable(table_ident, client_);
+    return harness_->CreateTable(table_ident, client_);
   }
 
   void TearDown() override {
@@ -387,15 +541,12 @@ class TSSentryITest : public ExternalMiniClusterITestBase {
   }
 
  protected:
-  // Kudu client with which to perform admin operations.
-  shared_ptr<KuduClient> admin_client_;
-
-  TSSentryITestHarness harness_;
+  unique_ptr<TSAuthzITestHarness> harness_;
 };
 
 // Tests authorizing read and write operations coming from multiple concurrent
 // users for multiple tables.
-TEST_F(TSSentryITest, TestReadsAndWrites) {
+TEST_P(TSAuthzITest, TestReadsAndWrites) {
   SKIP_IF_SLOW_NOT_ALLOWED();
 
   // First, set up the tables.
@@ -415,17 +566,13 @@ TEST_F(TSSentryITest, TestReadsAndWrites) {
   // Set up a bunch of clients for each user.
   unordered_map<string, vector<shared_ptr<KuduClient>>> user_to_clients;
   ThreadSafeRandom prng(SeedRandom());
-  unordered_set<string> cols(harness_.cols_.begin(), harness_.cols_.end());
+  unordered_set<string> cols(harness_->cols_.begin(), harness_->cols_.end());
   static constexpr int kNumClientsPerUser = 4;
   for (int i = 0; i < kNumUsers; i++) {
-    const string& user = harness_.users_[i];
-    // Register the user with the KDC, and add a role to the user's group
-    // (provided to MiniSentry in mini_sentry.cc).
+    const string& user = harness_->users_[i];
+    // Register the user with the KDC.
     ASSERT_OK(cluster_->kdc()->CreateUserPrincipal(user));
     ASSERT_OK(cluster_->kdc()->Kinit(user));
-    const string role = Substitute("role$0", i);
-    ASSERT_OK(CreateRoleAndAddToGroups(harness_.sentry_client_.get(),
-                                       role, Substitute("group$0", i)));
 
     // Set up multiple clients for each user.
     vector<shared_ptr<KuduClient>> clients;
@@ -437,17 +584,15 @@ TEST_F(TSSentryITest, TestReadsAndWrites) {
     EmplaceOrDie(&user_to_clients, user, std::move(clients));
 
     // Generate privileges for each user for every table, and grant the
-    // appropriate Sentry privileges.
+    // appropriate privileges.
     TableNameToPrivileges table_to_privileges;
     for (const string& table_name : tables) {
       RWPrivileges granted_privileges = GeneratePrivileges(cols, &prng);
       for (const auto& wp : granted_privileges.table_write_privileges) {
-        ASSERT_OK(AlterRoleGrantPrivilege(harness_.sentry_client_.get(), role,
-                  GetTablePrivilege(kDb, table_name, WritePrivilegeToString(wp))));
+        ASSERT_OK(harness_->GrantTablePrivilege(kDb, table_name, user, WritePrivilegeToString(wp)));
       }
       for (const auto& col : granted_privileges.column_scan_privileges) {
-        ASSERT_OK(AlterRoleGrantPrivilege(harness_.sentry_client_.get(), role,
-                  GetColumnPrivilege(kDb, table_name, col, "SELECT")));
+        ASSERT_OK(harness_->GrantColumnPrivilege(kDb, table_name, col, user, "SELECT"));
       }
       RWPrivileges not_granted_privileges = ComplementaryPrivileges(cols, granted_privileges);
       InsertOrDie(&table_to_privileges, table_name,
@@ -468,7 +613,7 @@ TEST_F(TSSentryITest, TestReadsAndWrites) {
       t.join();
     }
   });
-  for (const string& user : harness_.users_) {
+  for (const string& user : harness_->users_) {
     // Start a thread for every user that performs a bunch of operations.
     const auto* const table_to_privileges = FindOrNull(user_to_privileges, user);
     for (const auto& client_sp : FindOrDie(user_to_clients, user)) {
@@ -507,7 +652,7 @@ TEST_F(TSSentryITest, TestReadsAndWrites) {
 }
 
 // Test for a couple of scenarios related to alter tables.
-TEST_F(TSSentryITest, TestAlters) {
+TEST_P(TSAuthzITest, TestAlters) {
   SKIP_IF_SLOW_NOT_ALLOWED();
 
   static const string kTableName = "table";
@@ -517,8 +662,6 @@ TEST_F(TSSentryITest, TestAlters) {
   const string user = "user0";
   ASSERT_OK(cluster_->kdc()->CreateUserPrincipal(user));
   ASSERT_OK(cluster_->kdc()->Kinit(user));
-  const string role = "role0";
-  ASSERT_OK(CreateRoleAndAddToGroups(harness_.sentry_client_.get(), role, "group0"));
 
   shared_ptr<KuduClient> user_client;
   ASSERT_OK(cluster_->CreateClient(nullptr, &user_client));
@@ -526,14 +669,12 @@ TEST_F(TSSentryITest, TestAlters) {
   // Note: we only need privileges on the metadata for OpenTable() calls.
   // METADATA isn't a first-class Sentry privilege and won't get carried over
   // on table rename, so we just grant INSERT privileges.
-  ASSERT_OK(AlterRoleGrantPrivilege(harness_.sentry_client_.get(), role,
-            GetTablePrivilege(kDb, kTableName, "INSERT")));
+  ASSERT_OK(harness_->GrantTablePrivilege(kDb, kTableName, user, "INSERT"));
 
   // First, grant privileges on a new column that doesn't yet exist. Once that
   // column is created, we should be able to scan it immediately.
   const string new_column = Substitute("col$0", kNumColsPerTable);
-  ASSERT_OK(AlterRoleGrantPrivilege(harness_.sentry_client_.get(), role,
-            GetColumnPrivilege(kDb, kTableName, new_column, "SELECT")));
+  ASSERT_OK(harness_->GrantColumnPrivilege(kDb, kTableName, new_column, user, "SELECT"));
   {
     unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(table_ident));
     table_alterer->AddColumn(new_column)->Type(KuduColumnSchema::INT32);
@@ -544,31 +685,39 @@ TEST_F(TSSentryITest, TestAlters) {
   ASSERT_OK(PerformScan({ new_column }, /*prng=*/nullptr, table.get()));
 
   // Now create another column and grant the user privileges for that column.
-  // Since privileges are cached, even though we've granted privileges, clients
-  // will use the cached privilege and not be authorized for a bit.
   const string another_column = Substitute("col$0", kNumColsPerTable + 1);
-  ASSERT_OK(AlterRoleGrantPrivilege(harness_.sentry_client_.get(), role,
-            GetColumnPrivilege(kDb, kTableName, another_column, "SELECT")));
+  ASSERT_OK(harness_->GrantColumnPrivilege(kDb, kTableName, another_column, user, "SELECT"));
   {
     unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(table_ident));
     table_alterer->AddColumn(another_column)->Type(KuduColumnSchema::INT32);
     ASSERT_OK(table_alterer->Alter());
   }
+  // Since privileges are cached in Sentry, even though we've granted
+  // privileges, clients will use the cached privilege and not be authorized
+  // for a bit. Provided we waited a bit after granting each privilege, no such
+  // behavior exists in Ranger.
   ASSERT_OK(user_client->OpenTable(table_ident, &table));
-  Status s = PerformScan({ another_column }, /*prng=*/nullptr, table.get());
-  ASSERT_TRUE(s.IsRemoteError()) << s.ToString();
-  ASSERT_STR_CONTAINS(s.ToString(), "not authorized");
+  if (GetParam() == kSentry) {
+    Status s = PerformScan({ another_column }, /*prng=*/nullptr, table.get());
+    ASSERT_TRUE(s.IsRemoteError()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "not authorized");
+  }
 
   // Wait the full duration of the cache TTL, and an additional full token TTL.
   // This ensures that the client's token will expire we will get a new one
-  // with the most up-to-date privileges from Sentry.
+  // with the most up-to-date privileges from the authorization provider.
   SleepFor(MonoDelta::FromSeconds(kAuthzTokenTTLSecs * (1 + kAuthzCacheTTLMultiplier)));
   ASSERT_OK(PerformScan({ another_column }, /*prng=*/nullptr, table.get()));
 
   // Now rename the table to something else. There shouldn't be any privileges
   // cached for the newly-renamed table, so we should immediately be able to
-  // scan it.
-  const string new_table_ident = Substitute("$0.$1", kDb, "newtable");
+  // scan it. Sentry automatically renames privileges -- Ranger doesn't, so we
+  // need to explicitly grant privileges on the new table name.
+  const string kNewTableName = "newtable";
+  const string new_table_ident = Substitute("$0.$1", kDb, kNewTableName);
+  if (GetParam() != kSentry) {
+    ASSERT_OK(harness_->GrantTablePrivilege(kDb, kNewTableName, user, "SELECT"));
+  }
   {
     unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(table_ident));
     table_alterer->RenameTo(new_table_ident);
@@ -578,4 +727,10 @@ TEST_F(TSSentryITest, TestAlters) {
   ASSERT_OK(PerformScan({ another_column }, nullptr, table.get()));
 }
 
+INSTANTIATE_TEST_CASE_P(AuthzProviders, TSAuthzITest,
+    ::testing::Values(kSentry, kRanger),
+    [] (const testing::TestParamInfo<TSAuthzITest::ParamType>& info) {
+      return HarnessEnumToString(info.param);
+    });
+
 } // namespace kudu