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

[kudu] 01/04: KUDU-3081 Add Kerberos support to MiniRanger

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

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

commit 897eb377ce91a19614626ac177f4315c5c5dac69
Author: Attila Bukor <ab...@apache.org>
AuthorDate: Tue Mar 31 00:25:46 2020 +0200

    KUDU-3081 Add Kerberos support to MiniRanger
    
    As integration tests using Sentry authz provider all depend on a
    Kerberized mini cluster we also need to add Kerberos support to
    MiniRanger to be able to parameterize the integration tests.
    
    As adding Kerberos support proved to be trickier than expected this
    patch contains several other improvements needed for this to work.
    
    After configuring Kerberos for both MiniRanger and the Ranger subprocess
    correctly I still ran into an error:
    
    GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos credentails)
    
    This turned out to be due to a mismatch in forward and reverse lookup of
    MiniRanger's hostname. To fix this I switched to using IP addresses in
    the configuration instead of FQDNs.
    
    This patch also changes the default "java" binary in Ranger client to be
    the one in $JAVA_HOME/bin instead of the one in $PATH as dist-test has
    mismatching versions and it wouldn't start otherwise. To use the one in
    $PATH a user can still simply set it to "java", or provide a full path.
    
    Change-Id: I32118780ad912791fe5e371004345428b6459549
    Reviewed-on: http://gerrit.cloudera.org:8080/15601
    Tested-by: Kudu Jenkins
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
---
 src/kudu/integration-tests/master_sentry-itest.cc | 137 +++++++++----
 src/kudu/master/ranger_authz_provider.cc          |   4 +-
 src/kudu/mini-cluster/external_mini_cluster.cc    |  33 +++
 src/kudu/postgres/mini_postgres.cc                |   2 +-
 src/kudu/postgres/mini_postgres.h                 |   2 +-
 src/kudu/ranger/mini_ranger.cc                    |  64 ++++--
 src/kudu/ranger/mini_ranger.h                     |  34 +++-
 src/kudu/ranger/mini_ranger_configs.h             | 234 +++++++++++++++-------
 src/kudu/ranger/ranger_client.cc                  |  35 +++-
 src/kudu/ranger/ranger_client.h                   |   6 +
 src/kudu/security/init.cc                         |   9 +
 src/kudu/security/init.h                          |   8 +
 12 files changed, 429 insertions(+), 139 deletions(-)

diff --git a/src/kudu/integration-tests/master_sentry-itest.cc b/src/kudu/integration-tests/master_sentry-itest.cc
index 2fbbaac..b184af7 100644
--- a/src/kudu/integration-tests/master_sentry-itest.cc
+++ b/src/kudu/integration-tests/master_sentry-itest.cc
@@ -48,6 +48,8 @@
 #include "kudu/master/master.proxy.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/rpc/rpc_controller.h"
 #include "kudu/rpc/rpc_header.pb.h"
 #include "kudu/rpc/user_credentials.h"
@@ -71,8 +73,10 @@ using kudu::client::KuduColumnSchema;
 using kudu::client::KuduScanToken;
 using kudu::client::KuduScanTokenBuilder;
 using kudu::client::KuduSchema;
+using kudu::client::KuduSchemaBuilder;
 using kudu::client::KuduTable;
 using kudu::client::KuduTableAlterer;
+using kudu::client::KuduTableCreator;
 using kudu::client::sp::shared_ptr;
 using kudu::hms::HmsClient;
 using kudu::master::AlterRoleGrantPrivilege;
@@ -85,6 +89,10 @@ using kudu::master::MasterServiceProxy;
 using kudu::master::ResetAuthzCacheRequestPB;
 using kudu::master::ResetAuthzCacheResponsePB;
 using kudu::master::VOTER_REPLICA;
+using kudu::ranger::ActionPB;
+using kudu::ranger::AuthorizationPolicy;
+using kudu::ranger::MiniRanger;
+using kudu::ranger::PolicyItem;
 using kudu::rpc::RpcController;
 using kudu::rpc::UserCredentials;
 using kudu::sentry::SentryClient;
@@ -92,6 +100,7 @@ using sentry::TSentryGrantOption;
 using sentry::TSentryPrivilege;
 using std::atomic;
 using std::function;
+using std::move;
 using std::ostream;
 using std::string;
 using std::thread;
@@ -100,6 +109,19 @@ using std::unordered_set;
 using std::vector;
 using strings::Substitute;
 
+namespace {
+const char* const kAdminGroup = "admin";
+const char* const kAdminUser = "test-admin";
+const char* const kUserGroup = "user";
+const char* const kTestUser = "test-user";
+const char* const kImpalaUser = "impala";
+const char* const kDevRole = "developer";
+const char* const kAdminRole = "ad";
+const char* const kDatabaseName = "db";
+const char* const kTableName = "table";
+const char* const kSecondTable = "second_table";
+} // namespace
+
 namespace kudu {
 
 // Parameters for the operator functor (see below for OperationFunc).
@@ -130,17 +152,6 @@ struct PrivilegeParams {
 // integration enabled.
 class SentryITestBase : public HmsITestBase {
  public:
-  static const char* const kAdminGroup;
-  static const char* const kAdminUser;
-  static const char* const kUserGroup;
-  static const char* const kTestUser;
-  static const char* const kImpalaUser;
-  static const char* const kDevRole;
-  static const char* const kAdminRole;
-  static const char* const kDatabaseName;
-  static const char* const kTableName;
-  static const char* const kSecondTable;
-
   Status StopSentry() {
     RETURN_NOT_OK(sentry_client_->Stop());
     RETURN_NOT_OK(cluster_->sentry()->Stop());
@@ -359,16 +370,28 @@ class SentryITestBase : public HmsITestBase {
   unique_ptr<SentryClient> sentry_client_;
 };
 
-const char* const SentryITestBase::kAdminGroup = "admin";
-const char* const SentryITestBase::kAdminUser = "test-admin";
-const char* const SentryITestBase::kUserGroup = "user";
-const char* const SentryITestBase::kTestUser = "test-user";
-const char* const SentryITestBase::kImpalaUser = "impala";
-const char* const SentryITestBase::kDevRole = "developer";
-const char* const SentryITestBase::kAdminRole = "ad";
-const char* const SentryITestBase::kDatabaseName = "db";
-const char* const SentryITestBase::kTableName = "table";
-const char* const SentryITestBase::kSecondTable = "second_table";
+class RangerITestBase : public ExternalMiniClusterITestBase {
+ public:
+  void SetUp() override {
+    ExternalMiniClusterITestBase::SetUp();
+
+    cluster::ExternalMiniClusterOptions opts;
+    opts.enable_ranger = true;
+    opts.enable_kerberos = true;
+
+    StartClusterWithOpts(std::move(opts));
+    ranger_ = this->cluster_->ranger();
+
+    AuthorizationPolicy policy;
+    policy.databases.emplace_back(kDatabaseName);
+    policy.tables.emplace_back("*");
+    policy.items.emplace_back(PolicyItem({kAdminUser}, {ActionPB::ALL}));
+    ASSERT_OK(ranger_->AddPolicy(move(policy)));
+    SleepFor(MonoDelta::FromMilliseconds(1500));
+  }
+ private:
+  MiniRanger* ranger_;
+};
 
 // Functor that performs a certain operation (e.g. Create, Rename) on a table
 // given its name and its desired new name, if necessary (only used for Rename).
@@ -409,6 +432,44 @@ ostream& operator <<(ostream& out, const AuthzDescriptor& d) {
 // enabled.
 class MasterSentryTest : public SentryITestBase {};
 
+class MasterRangerTest : public RangerITestBase {};
+
+TEST_F(MasterRangerTest, TestCreateTableAuthorized) {
+  ASSERT_OK(cluster_->kdc()->Kinit(kAdminUser));
+  ASSERT_OK(cluster_->CreateClient(nullptr, &client_));
+
+  KuduSchemaBuilder b;
+  b.AddColumn("key")->Type(KuduColumnSchema::INT32)->NotNull()
+                    ->PrimaryKey();
+  KuduSchema schema;
+  ASSERT_OK(b.Build(&schema));
+  unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+  ASSERT_OK(table_creator->table_name(Substitute("$0.$1",
+                                      kDatabaseName, kTableName))
+    .schema(&schema)
+    .num_replicas(1)
+    .set_range_partition_columns({"key"})
+    .Create());
+}
+
+TEST_F(MasterRangerTest, TestCreateTableUnauthorized) {
+  ASSERT_OK(cluster_->kdc()->Kinit(kTestUser));
+  ASSERT_OK(cluster_->CreateClient(nullptr, &client_));
+
+  KuduSchemaBuilder b;
+  b.AddColumn("key")->Type(KuduColumnSchema::INT32)->NotNull()
+                    ->PrimaryKey();
+  KuduSchema schema;
+  ASSERT_OK(b.Build(&schema));
+  unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+  ASSERT_TRUE(table_creator->table_name(Substitute("$0.$1",
+                                        kDatabaseName, kTableName))
+    .schema(&schema)
+    .num_replicas(1)
+    .set_range_partition_columns({"key"})
+    .Create().IsNotAuthorized());
+}
+
 // Test that the trusted user can access the cluster without being authorized.
 TEST_F(MasterSentryTest, TestTrustedUserAcl) {
   // Log in as 'impala' and reset the client to pick up the change in user.
@@ -552,7 +613,7 @@ static const AuthzDescriptor kAuthzCombinations[] = {
         &SentryITestBase::GrantCreateTablePrivilege,
         "CreateTable",
       },
-      SentryITestBase::kDatabaseName,
+      kDatabaseName,
       "new_table",
       ""
     },
@@ -562,8 +623,8 @@ static const AuthzDescriptor kAuthzCombinations[] = {
         &SentryITestBase::GrantDropTablePrivilege,
         "DropTable",
       },
-      SentryITestBase::kDatabaseName,
-      SentryITestBase::kTableName,
+      kDatabaseName,
+      kTableName,
       ""
     },
     {
@@ -572,8 +633,8 @@ static const AuthzDescriptor kAuthzCombinations[] = {
         &SentryITestBase::GrantAlterTablePrivilege,
         "AlterTable",
       },
-      SentryITestBase::kDatabaseName,
-      SentryITestBase::kTableName,
+      kDatabaseName,
+      kTableName,
       ""
     },
     {
@@ -582,8 +643,8 @@ static const AuthzDescriptor kAuthzCombinations[] = {
         &SentryITestBase::GrantRenameTablePrivilege,
         "RenameTable",
       },
-      SentryITestBase::kDatabaseName,
-      SentryITestBase::kTableName,
+      kDatabaseName,
+      kTableName,
       "new_table"
     },
     {
@@ -592,8 +653,8 @@ static const AuthzDescriptor kAuthzCombinations[] = {
         &SentryITestBase::GrantGetMetadataTablePrivilege,
         "GetTableSchema",
       },
-      SentryITestBase::kDatabaseName,
-      SentryITestBase::kTableName,
+      kDatabaseName,
+      kTableName,
       ""
     },
     {
@@ -602,8 +663,8 @@ static const AuthzDescriptor kAuthzCombinations[] = {
         &SentryITestBase::GrantGetMetadataTablePrivilege,
         "GetTableLocations",
       },
-      SentryITestBase::kDatabaseName,
-      SentryITestBase::kTableName,
+      kDatabaseName,
+      kTableName,
       ""
     },
     {
@@ -612,8 +673,8 @@ static const AuthzDescriptor kAuthzCombinations[] = {
         &SentryITestBase::GrantGetMetadataTablePrivilege,
         "GetTabletLocations",
       },
-      SentryITestBase::kDatabaseName,
-      SentryITestBase::kTableName,
+      kDatabaseName,
+      kTableName,
       ""
     },
     {
@@ -622,8 +683,8 @@ static const AuthzDescriptor kAuthzCombinations[] = {
         &SentryITestBase::GrantGetMetadataTablePrivilege,
         "IsCreateTableDone",
       },
-      SentryITestBase::kDatabaseName,
-      SentryITestBase::kTableName,
+      kDatabaseName,
+      kTableName,
       ""
     },
     {
@@ -632,8 +693,8 @@ static const AuthzDescriptor kAuthzCombinations[] = {
         &SentryITestBase::GrantGetMetadataTablePrivilege,
         "IsAlterTableDone",
       },
-      SentryITestBase::kDatabaseName,
-      SentryITestBase::kTableName,
+      kDatabaseName,
+      kTableName,
       ""
     },
 };
diff --git a/src/kudu/master/ranger_authz_provider.cc b/src/kudu/master/ranger_authz_provider.cc
index 8cbb49f..8fe4e70 100644
--- a/src/kudu/master/ranger_authz_provider.cc
+++ b/src/kudu/master/ranger_authz_provider.cc
@@ -32,8 +32,8 @@ DECLARE_string(ranger_config_path);
 
 namespace kudu {
 
-class MetricEntity;
 class Env;
+class MetricEntity;
 
 namespace master {
 
@@ -46,7 +46,7 @@ using std::string;
 using std::unordered_set;
 
 RangerAuthzProvider::RangerAuthzProvider(Env* env,
-    const scoped_refptr<MetricEntity>& metric_entity) :
+                                         const scoped_refptr<MetricEntity>& metric_entity) :
   client_(env, metric_entity) {}
 
 Status RangerAuthzProvider::Start() {
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc b/src/kudu/mini-cluster/external_mini_cluster.cc
index 32fce60..d3cb42c 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -336,7 +336,36 @@ Status ExternalMiniCluster::Start() {
 
   if (opts_.enable_ranger) {
     ranger_.reset(new ranger::MiniRanger(cluster_root()));
+    string host = "127.0.0.1";
+    if (opts_.enable_kerberos) {
+
+      // The SPNs match the ones defined in mini_ranger_configs.h.
+      string admin_keytab;
+      RETURN_NOT_OK_PREPEND(kdc_->CreateServiceKeytab(
+            Substitute("rangeradmin/$0@KRBTEST.COM", host),
+            &admin_keytab),
+          "could not create rangeradmin keytab");
+
+      string lookup_keytab;
+      RETURN_NOT_OK_PREPEND(kdc_->CreateServiceKeytab(
+            Substitute("rangerlookup/$0@KRBTEST.COM", host),
+            &lookup_keytab),
+          "could not create rangerlookup keytab");
+
+      string spnego_keytab;
+      RETURN_NOT_OK_PREPEND(kdc_->CreateServiceKeytab(
+            Substitute("HTTP/$0@KRBTEST.COM", host),
+            &spnego_keytab),
+          "could not create ranger HTTP keytab");
+
+      ranger_->EnableKerberos(kdc_->GetEnvVars()["KRB5_CONFIG"], admin_keytab,
+                              lookup_keytab, spnego_keytab);
+    }
+
     RETURN_NOT_OK_PREPEND(ranger_->Start(), "Failed to start the Ranger service");
+    RETURN_NOT_OK_PREPEND(ranger_->CreateClientConfig(JoinPathSegments(cluster_root(),
+                                                                       "ranger-client")),
+                          "Failed to write Ranger client config");
   }
 
   // Start the HMS.
@@ -609,6 +638,10 @@ Status ExternalMiniCluster::StartMasters() {
       if (!opts_.enable_kerberos) {
         opts.extra_flags.emplace_back("--sentry_service_security_mode=none");
       }
+    } else if (opts_.enable_ranger) {
+      opts.extra_flags.emplace_back(Substitute("--ranger_config_path=$0",
+                                               JoinPathSegments(cluster_root(),
+                                                                "ranger-client")));
     }
     opts.logtostderr = opts_.logtostderr;
 
diff --git a/src/kudu/postgres/mini_postgres.cc b/src/kudu/postgres/mini_postgres.cc
index 1471db2..9fe2c22 100644
--- a/src/kudu/postgres/mini_postgres.cc
+++ b/src/kudu/postgres/mini_postgres.cc
@@ -132,7 +132,7 @@ Status MiniPostgres::CreateConfigs() {
   string config_file = JoinPathSegments(pg_root(), "postgresql.conf");
   faststring config;
   ReadFileToString(env, config_file, &config);
-  config.append(Substitute("\nport=$0\n", port_));
+  config.append(Substitute("\nport = $0\n", port_));
   unique_ptr<WritableFile> file;
   RETURN_NOT_OK(env->NewWritableFile(config_file, &file));
   RETURN_NOT_OK(file->Append(config));
diff --git a/src/kudu/postgres/mini_postgres.h b/src/kudu/postgres/mini_postgres.h
index da8a0fd..7a7a136 100644
--- a/src/kudu/postgres/mini_postgres.h
+++ b/src/kudu/postgres/mini_postgres.h
@@ -26,8 +26,8 @@
 #include "kudu/util/env.h"
 #include "kudu/util/path_util.h"
 #include "kudu/util/status.h"
-#include "kudu/util/test_util.h"
 #include "kudu/util/subprocess.h" // IWYU pragma: keep
+#include "kudu/util/test_util.h"
 
 namespace kudu {
 namespace postgres {
diff --git a/src/kudu/ranger/mini_ranger.cc b/src/kudu/ranger/mini_ranger.cc
index 3351222..7c2f0a0 100644
--- a/src/kudu/ranger/mini_ranger.cc
+++ b/src/kudu/ranger/mini_ranger.cc
@@ -85,29 +85,31 @@ Status MiniRanger::InitRanger(string admin_home, bool* fresh_install) {
   return Status::OK();
 }
 
-Status MiniRanger::CreateConfigs(const string& fqdn) {
+Status MiniRanger::CreateConfigs() {
   // Ranger listens on 2 ports:
   //
-  // - ranger_port_ is the RPC port (REST API) that the Ranger subprocess and
+  // - port_ is the RPC port (REST API) that the Ranger subprocess and
   //   EasyCurl can talk to
   // - ranger_shutdown_port is the port which Ranger listens on for a shutdown
   //   command. We're not using this shutdown port as we simply send a SIGTERM,
   //   but it's necessary to set it to a random value to avoid collisions in
   //   parallel testing.
-  RETURN_NOT_OK(GetRandomPort(&ranger_port_));
+  RETURN_NOT_OK(GetRandomPort(&port_));
   uint16_t ranger_shutdown_port;
   RETURN_NOT_OK(GetRandomPort(&ranger_shutdown_port));
   string admin_home = ranger_admin_home();
 
-  ranger_admin_url_ = Substitute("http://$0:$1", fqdn, ranger_port_);
+  ranger_admin_url_ = Substitute("http://127.0.0.1:$0", port_);
 
   // Write config files
   RETURN_NOT_OK(WriteStringToFile(
-      env_, GetRangerInstallProperties(bin_dir(), mini_pg_.bound_port()),
+      env_, GetRangerInstallProperties(bin_dir(), "127.0.0.1", mini_pg_.bound_port()),
       JoinPathSegments(admin_home, "install.properties")));
 
   RETURN_NOT_OK(WriteStringToFile(
-      env_, GetRangerAdminSiteXml(ranger_port_, mini_pg_.bound_port()),
+      env_, GetRangerAdminSiteXml("127.0.0.1", port_, "127.0.0.1", mini_pg_.bound_port(),
+                                  admin_ktpath_, lookup_ktpath_,
+                                  spnego_ktpath_),
       JoinPathSegments(admin_home, "ranger-admin-site.xml")));
 
   RETURN_NOT_OK(WriteStringToFile(
@@ -116,7 +118,7 @@ Status MiniRanger::CreateConfigs(const string& fqdn) {
         ranger_shutdown_port),
       JoinPathSegments(admin_home, "ranger-admin-default-site.xml")));
 
-  RETURN_NOT_OK(WriteStringToFile(env_, GetRangerCoreSiteXml(/*secure=*/ false),
+  RETURN_NOT_OK(WriteStringToFile(env_, GetRangerCoreSiteXml(kerberos_),
                                   JoinPathSegments(admin_home, "core-site.xml")));
 
   RETURN_NOT_OK(WriteStringToFile(env_, GetRangerLog4jProperties("info"),
@@ -144,7 +146,7 @@ Status MiniRanger::DbSetup(const string& admin_home, const string& ews_dir,
       { "RANGER_ADMIN_HOME", ranger_home_ },
       { "RANGER_ADMIN_CONF", admin_home },
       { "XAPOLICYMGR_DIR", admin_home },
-      { "RANGER_PID_DIR_PATH", ranger_home_ },
+      { "RANGER_PID_DIR_PATH", admin_home },
       });
   db_setup.SetCurrentDir(admin_home);
   RETURN_NOT_OK(db_setup.Start());
@@ -173,9 +175,7 @@ Status MiniRanger::StartRanger() {
 
     LOG(INFO) << "Starting Ranger out of " << kAdminHome;
 
-    string fqdn;
-    RETURN_NOT_OK(GetFQDN(&fqdn));
-    RETURN_NOT_OK(CreateConfigs(fqdn));
+    RETURN_NOT_OK(CreateConfigs());
 
     if (fresh_install) {
       RETURN_NOT_OK(DbSetup(kAdminHome, kEwsDir, kWebAppDir));
@@ -188,21 +188,26 @@ Status MiniRanger::StartRanger() {
 
     LOG(INFO) << "Using Ranger class path: " << classpath;
 
-    LOG(INFO) << "Using FQDN: " << fqdn;
-    process_.reset(new Subprocess({
+    std::vector<string> args({
         JoinPathSegments(java_home_, "bin/java"),
         "-Dproc_rangeradmin",
-        Substitute("-Dhostname=$0", fqdn),
+        "-Dhostname=127.0.0.1",
         Substitute("-Dlog4j.configuration=file:$0",
                    JoinPathSegments(kAdminHome, "log4j.properties")),
         "-Duser=miniranger",
-        Substitute("-Dranger.service.host=$0", fqdn),
+        "-Dranger.service.host=127.0.0.1",
         "-Dservername=miniranger",
         Substitute("-Dcatalina.base=$0", kEwsDir),
         Substitute("-Dlogdir=$0", JoinPathSegments(kAdminHome, "logs")),
         "-Dranger.audit.solr.bootstrap.enabled=false",
-        "-cp", classpath, "org.apache.ranger.server.tomcat.EmbeddedServer"
-    }));
+    });
+    if (kerberos_) {
+      args.emplace_back(Substitute("-Djava.security.krb5.conf=$0", krb5_config_));
+    }
+    args.emplace_back("-cp");
+    args.emplace_back(classpath);
+    args.emplace_back("org.apache.ranger.server.tomcat.EmbeddedServer");
+    process_.reset(new Subprocess(args));
     process_->SetEnvVars({
         { "XAPOLICYMGR_DIR", kAdminHome },
         { "XAPOLICYMGR_EWS_DIR", kEwsDir },
@@ -211,6 +216,7 @@ Status MiniRanger::StartRanger() {
         { "JAVA_HOME", java_home_ },
         { "RANGER_PID_DIR_PATH", JoinPathSegments(data_root_, "tmppid") },
         { "RANGER_ADMIN_PID_NAME", "rangeradmin.pid" },
+        { "RANGER_ADMIN_CONF_DIR", kAdminHome },
         { "RANGER_USER", "miniranger" },
     });
     RETURN_NOT_OK(process_->Start());
@@ -232,8 +238,13 @@ Status MiniRanger::CreateKuduService() {
   EasyJson service;
   service.Set("name", "kudu");
   service.Set("type", "kudu");
+  // The below config authorizes "kudu" to download the list of authorized users
+  // for policies and tags respectively.
+  EasyJson configs = service.Set("configs", EasyJson::kObject);
+  configs.Set("policy.download.auth.users", "kudu");
+  configs.Set("tag.download.auth.users", "kudu");
 
-  RETURN_NOT_OK_PREPEND(PostToRanger("service/plugins/services", std::move(service)),
+  RETURN_NOT_OK_PREPEND(PostToRanger("service/plugins/services", service),
                         "Failed to create Kudu service");
   LOG(INFO) << "Created Kudu service";
   return Status::OK();
@@ -302,5 +313,22 @@ Status MiniRanger::PostToRanger(string url, EasyJson payload) {
   return Status::OK();
 }
 
+Status MiniRanger::CreateClientConfig(const string& client_config_path) {
+  auto policy_cache = JoinPathSegments(client_config_path, "policy-cache");
+  if (!env_->FileExists(client_config_path)) {
+    RETURN_NOT_OK(env_->CreateDir(client_config_path));
+    RETURN_NOT_OK(env_->CreateDir(policy_cache));
+  }
+
+  RETURN_NOT_OK(WriteStringToFile(env_, GetRangerCoreSiteXml(kerberos_),
+                                  JoinPathSegments(client_config_path, "core-site.xml")));
+  RETURN_NOT_OK(WriteStringToFile(env_, GetRangerKuduSecurityXml(policy_cache, "kudu",
+                                                                 ranger_admin_url_,
+                                                                 policy_poll_interval_ms_),
+                                  JoinPathSegments(client_config_path,
+                                                   "ranger-kudu-security.xml")));
+  return Status::OK();
+}
+
 } // namespace ranger
 } // namespace kudu
diff --git a/src/kudu/ranger/mini_ranger.h b/src/kudu/ranger/mini_ranger.h
index ff8947d..b1d009b 100644
--- a/src/kudu/ranger/mini_ranger.h
+++ b/src/kudu/ranger/mini_ranger.h
@@ -71,6 +71,7 @@ class MiniRanger {
   explicit MiniRanger(std::string data_root)
     : data_root_(std::move(data_root)),
       mini_pg_(data_root_),
+      kerberos_(false),
       env_(Env::Default()) {
         curl_.set_auth(CurlAuthType::BASIC, "admin", "admin");
       }
@@ -84,6 +85,24 @@ class MiniRanger {
   // Adds a new policy to Ranger.
   Status AddPolicy(AuthorizationPolicy policy) WARN_UNUSED_RESULT;
 
+  // Creates the client configs files in client_config_path.
+  Status CreateClientConfig(const std::string& client_config_path) WARN_UNUSED_RESULT;
+
+  void EnableKerberos(std::string krb5_config,
+                      std::string admin_ktpath,
+                      std::string lookup_ktpath,
+                      std::string spnego_ktpath) {
+    kerberos_ = true;
+    krb5_config_ = std::move(krb5_config);
+    admin_ktpath_ = std::move(admin_ktpath);
+    lookup_ktpath_ = std::move(lookup_ktpath);
+    spnego_ktpath_ = std::move(spnego_ktpath);
+  }
+
+  void set_policy_poll_interval_ms(uint32_t policy_poll_interval_ms) {
+    policy_poll_interval_ms_ = policy_poll_interval_ms;
+  }
+
  private:
   // Starts the Ranger service.
   Status StartRanger() WARN_UNUSED_RESULT;
@@ -95,7 +114,7 @@ class MiniRanger {
     WARN_UNUSED_RESULT;
 
   // Creates configuration files.
-  Status CreateConfigs(const std::string& fqdn) WARN_UNUSED_RESULT;
+  Status CreateConfigs() WARN_UNUSED_RESULT;
 
   // Initializes Ranger's database.
   Status DbSetup(const std::string& admin_home, const std::string& ews_dir,
@@ -144,10 +163,21 @@ class MiniRanger {
   std::string ranger_home_;
   std::string java_home_;
 
+  bool kerberos_;
+  std::string admin_ktpath_;
+  std::string lookup_ktpath_;
+  std::string spnego_ktpath_;
+  std::string krb5_config_;
+
   Env* env_;
   EasyCurl curl_;
 
-  uint16_t ranger_port_;
+  uint16_t port_ = 0;
+
+  // Determines how frequently clients fetch policies from the server. The
+  // default is 200ms so that tests don't have to wait too long until freshly
+  // created policies can be used.
+  uint32_t policy_poll_interval_ms_ = 200;
 };
 
 } // namespace ranger
diff --git a/src/kudu/ranger/mini_ranger_configs.h b/src/kudu/ranger/mini_ranger_configs.h
index 1998a1c..2dcf19d 100644
--- a/src/kudu/ranger/mini_ranger_configs.h
+++ b/src/kudu/ranger/mini_ranger_configs.h
@@ -24,16 +24,22 @@
 namespace kudu {
 namespace ranger {
 
-// Taken and modified from:
-// https://github.com/apache/ranger/blob/master/security-admin/scripts/install.properties
-//
-// $0: directory containing postgresql.jar
-// $1: postgres port
-static const char* kInstallProperties = R"(
+// Gets the contents of the install.properties file used by the db_setup.py
+// script.
+inline std::string GetRangerInstallProperties(const std::string& bin_dir,
+                                              const std::string& pg_host,
+                                              uint16_t pg_port) {
+  // Taken and modified from:
+  // https://github.com/apache/ranger/blob/master/security-admin/scripts/install.properties
+  //
+  // $0: directory containing postgresql.jar
+  // $1: postgres host
+  // $2: postgres port
+  const char* kInstallProperties = R"(
 DB_FLAVOR=POSTGRES
 SQL_CONNECTOR_JAR=$0/postgresql.jar
 db_root_user=postgres
-db_host=localhost:$1
+db_host=$1:$2
 db_root_password=
 db_ssl_enabled=false
 db_ssl_required=false
@@ -51,18 +57,34 @@ postgres_core_file=db/postgres/optimized/current/ranger_core_db_postgres.sql
 sqlserver_core_file=db/sqlserver/optimized/current/ranger_core_db_sqlserver.sql
 sqlanywhere_core_file=db/sqlanywhere/optimized/current/ranger_core_db_sqlanywhere.sql
 )";
+  return strings::Substitute(kInstallProperties, bin_dir, pg_host, pg_port);
+}
 
-// For port info, see:
-// https://docs.cloudera.com/HDPDocuments/HDP2/HDP-2.6.5/bk_reference/content/ranger-ports.html
-//
-// postgres DB hardcoded as "ranger"
-// ranger jdbc user: miniranger
-// ranger jdbc pw: miniranger
-// hardcoded auth NONE
-//
-// $0: postgres port
-// $1: admin port/RPC (REST API) port
-const char* kRangerAdminSiteTemplate = R"(
+// Gets the contents of the ranger-admin-site.xml config that has most of the
+// configuration needed to start Ranger.
+inline std::string GetRangerAdminSiteXml(const std::string& admin_host,
+                                         uint16_t admin_port,
+                                         const std::string& pg_host,
+                                         uint16_t pg_port,
+                                         const std::string& admin_keytab,
+                                         const std::string& lookup_keytab,
+                                         const std::string& spnego_keytab) {
+  // For port info, see:
+  // https://docs.cloudera.com/HDPDocuments/HDP2/HDP-2.6.5/bk_reference/content/ranger-ports.html
+  //
+  // postgres DB hardcoded as "ranger"
+  // ranger jdbc user: miniranger
+  // ranger jdbc pw: miniranger
+  // hardcoded auth NONE
+  //
+  // $0: postgres host
+  // $1: postgres port
+  // $2: admin host
+  // $3: admin port/RPC (REST API) port
+  // $4: admin keytab
+  // $5: lookup keytab
+  // $6: spnego keytab
+  const char* kRangerAdminSiteTemplate = R"(
 <configuration>
 
   <!-- DB config -->
@@ -74,7 +96,7 @@ const char* kRangerAdminSiteTemplate = R"(
   </property>
   <property>
     <name>ranger.jpa.jdbc.url</name>
-    <value>jdbc:postgresql://localhost:$0/ranger</value>
+    <value>jdbc:postgresql://$0:$1/ranger</value>
     <description/>
   </property>
   <property>
@@ -92,7 +114,7 @@ const char* kRangerAdminSiteTemplate = R"(
 
   <property>
     <name>ranger.externalurl</name>
-    <value>http://localhost:$1</value>
+    <value>http://$2:$3</value>
     <description/>
   </property>
   <property>
@@ -107,27 +129,65 @@ const char* kRangerAdminSiteTemplate = R"(
   </property>
   <property>
     <name>ranger.service.host</name>
-    <value>localhost</value>
+    <value>$2</value>
   </property>
   <property>
     <name>ranger.service.http.port</name>
-    <value>$1</value>
+    <value>$3</value>
   </property>
   <property>
     <name>ranger.admin.cookie.name</name>
     <value>RANGERADMINSESSIONID</value>
   </property>
+  <property>
+    <name>ranger.plugins.kudu.serviceuser</name>
+    <value>kudu</value>
+  </property>
+
+  <!-- Kerberos config -->
+  <property>
+    <name>ranger.admin.kerberos.keytab</name>
+    <value>$4</value>
+  </property>
+  <property>
+    <name>ranger.admin.kerberos.principal</name>
+    <value>rangeradmin/_HOST@KRBTEST.COM</value>
+  </property>
+  <property>
+    <name>ranger.lookup.kerberos.keytab</name>
+    <value>$5</value>
+  </property>
+  <property>
+    <name>ranger.lookup.kerberos.principal</name>
+    <value>rangerlookup/_HOST@KRBTEST.COM</value>
+  </property>
+  <property>
+    <name>ranger.spnego.kerberos.keytab</name>
+    <value>$6</value>
+  </property>
+  <property>
+    <name>ranger.spnego.kerberos.principal</name>
+    <value>HTTP/_HOST@KRBTEST.COM</value>
+  </property>
 </configuration>
 )";
+  return strings::Substitute(kRangerAdminSiteTemplate, pg_host, pg_port,
+                             admin_host, admin_port, admin_keytab,
+                             lookup_keytab, spnego_keytab);
+}
 
-// ranger-admin-default-site.xml
-// - postgres JDBC driver path
-// - RANGER_HOME (needed for jceks/KMS), impala says this is ranger-home, but the
-//   conf/jcsks directory doesn't exist for us.
-//
-// $0: postgres JDBC driver path
-// $1: ranger shutdown port
-const char* kRangerAdminDefaultSiteTemplate = R"(
+// Gets the ranger-admin-default-site.xml that has some additional configuration
+// needed to start Ranger. It's unclear why this has to be a separate file.
+inline std::string GetRangerAdminDefaultSiteXml(const std::string& pg_driver,
+                                                uint16_t shutdown_port) {
+  // ranger-admin-default-site.xml
+  // - postgres JDBC driver path
+  // - RANGER_HOME (needed for jceks/KMS), impala says this is ranger-home, but the
+  //   conf/jcsks directory doesn't exist for us.
+  //
+  // $0: postgres JDBC driver path
+  // $1: ranger shutdown port
+  const char* kRangerAdminDefaultSiteTemplate = R"(
 <configuration>
 
 <!-- Actual config we need -->
@@ -236,14 +296,21 @@ const char* kRangerAdminDefaultSiteTemplate = R"(
   </property>
 </configuration>
 )";
+  return strings::Substitute(kRangerAdminDefaultSiteTemplate, pg_driver,
+                             shutdown_port);
+}
 
-// log4j.properties file.
-//
-// This is the default log4j.properties with the only difference that rootLogger
-// is made configurable if it's needed for debugging.
-//
-// $0: log level
-const char *kLog4jPropertiesTemplate = R"(
+// Gets the contents of the log4j.properties file which is used to set up the
+// logging in Ranger. The only modification to the default log4j.properties is
+// the configurable log level.
+inline std::string GetRangerLog4jProperties(const std::string& log_level) {
+  // log4j.properties file.
+  //
+  // This is the default log4j.properties with the only difference that rootLogger
+  // is made configurable if it's needed for debugging.
+  //
+  // $0: log level
+  const char *kLog4jPropertiesTemplate = R"(
 log4j.rootLogger = $0,xa_log_appender
 
 
@@ -300,49 +367,78 @@ log4j.additivity.jdbc.resultset=false
 log4j.category.jdbc.connection=fatal,sql_appender
 log4j.additivity.jdbc.connection=false
 )";
+  return strings::Substitute(kLog4jPropertiesTemplate, log_level);
+}
 
-// core-site.xml containing authentication method.
-//
-// $0: authn method (simple or kerberos)
-const char* kCoreSiteTemplate = R"(
+// Gets the core-site.xml that configures authentication.
+inline std::string GetRangerCoreSiteXml(bool secure) {
+  // core-site.xml containing authentication method.
+  //
+  // $0: authn method (simple or kerberos)
+  const char* kCoreSiteTemplate = R"(
 <configuration>
   <property>
     <name>hadoop.security.authentication</name>
     <value>$0</value>
   </property>
+  <property>
+    <name>hadoop.security.group.mapping</name>
+    <value>org.apache.hadoop.security.NullGroupsMapping</value>
+  </property>
 </configuration>
 )";
+  if (secure) {
+    return strings::Substitute(kCoreSiteTemplate, "kerberos", "true");
+  }
 
-// Gets the contents of the install.properties file used by the db_setup.py
-// script.
-std::string GetRangerInstallProperties(std::string bin_dir, uint16_t pg_port) {
-  return strings::Substitute(kInstallProperties, bin_dir, pg_port);
-}
-
-// Gets the contents of the ranger-admin-site.xml config that has most of the
-// configuration needed to start Ranger.
-std::string GetRangerAdminSiteXml(uint16_t port, uint16_t pg_port) {
-  return strings::Substitute(kRangerAdminSiteTemplate, pg_port, port);
-}
-
-// Gets the ranger-admin-default-site.xml that has some additional configuration
-// needed to start Ranger. It's unclear why this has to be a separate file.
-std::string GetRangerAdminDefaultSiteXml(std::string pg_driver,
-                                         uint16_t shutdown_port) {
-  return strings::Substitute(kRangerAdminDefaultSiteTemplate, pg_driver,
-                             shutdown_port);
-}
-
-// Gets the contents of the log4j.properties file which is used to set up the
-// logging in Ranger. The only modification to the default log4j.properties is
-// the configurable log level.
-std::string GetRangerLog4jProperties(std::string log_level) {
-  return strings::Substitute(kLog4jPropertiesTemplate, log_level);
+  return strings::Substitute(kCoreSiteTemplate, "simple", "false");
 }
 
-// Gets the core-site.xml that configures authentication.
-std::string GetRangerCoreSiteXml(bool secure) {
-  return strings::Substitute(kCoreSiteTemplate, secure ? "kerberos" : "simple");
+// Gets the contents of ranger-kudu-security.xml for configuring the client.
+inline std::string GetRangerKuduSecurityXml(const std::string& policy_cache_dir,
+                                            const std::string& service_name,
+                                            const std::string& admin_url,
+                                            uint32_t policy_poll_interval_ms ) {
+  // ranger-kudu-security.xml (client configuration file).
+  //
+  // $0: Range policy cache dir
+  // $1: Kudu service name
+  // $2: Ranger admin URL
+  // $3: Policy poll interval (ms)
+  const char* kRangerKuduSecurity = R"(
+<configuration>
+  <property>
+    <name>ranger.plugin.kudu.policy.cache.dir</name>
+    <value>$0</value>
+  </property>
+  <property>
+    <name>ranger.plugin.kudu.service.name</name>
+    <value>$1</value>
+  </property>
+  <property>
+    <name>ranger.plugin.kudu.policy.rest.url</name>
+    <value>$2</value>
+  </property>
+  <property>
+    <name>ranger.plugin.kudu.policy.source.impl</name>
+    <value>org.apache.ranger.admin.client.RangerAdminRESTClient</value>
+  </property>
+  <property>
+    <name>ranger.plugin.kudu.policy.rest.ssl.config.file</name>
+    <value></value>
+  </property>
+  <property>
+    <name>ranger.plugin.kudu.policy.pollIntervalMs</name>
+    <value>$3</value>
+  </property>
+  <property>
+    <name>ranger.plugin.kudu.access.cluster.name</name>
+    <value>Cluster 1</value>
+  </property>
+</configuration>
+)";
+  return strings::Substitute(kRangerKuduSecurity, policy_cache_dir,
+                             service_name, admin_url, policy_poll_interval_ms);
 }
 
 } // namespace ranger
diff --git a/src/kudu/ranger/ranger_client.cc b/src/kudu/ranger/ranger_client.cc
index 2f0ba21..be40dd1 100644
--- a/src/kudu/ranger/ranger_client.cc
+++ b/src/kudu/ranger/ranger_client.cc
@@ -18,6 +18,7 @@
 #include "kudu/ranger/ranger_client.h"
 
 #include <cstdint>
+#include <cstdlib>
 #include <ostream>
 #include <string>
 #include <utility>
@@ -43,10 +44,12 @@
 #include "kudu/util/status.h"
 #include "kudu/util/subprocess.h"
 
-DEFINE_string(ranger_java_path, "java",
+DEFINE_string(ranger_java_path, "",
               "The path where the Java binary was installed. If "
-              "the value isn't an absolute path, it will be evaluated "
-              "using the Kudu user's PATH.");
+              "the value isn't an absolute path (e.g. 'java'), it will be "
+              "evaluated using the Kudu user's PATH. Empty string means "
+              "$JAVA_HOME/bin/java is used. If $JAVA_HOME is not found, Kudu "
+              "will attempt to find 'java' in $PATH.");
 TAG_FLAG(ranger_java_path, experimental);
 
 DEFINE_string(ranger_config_path, "",
@@ -133,6 +136,7 @@ DECLARE_string(principal);
 namespace kudu {
 namespace ranger {
 
+using kudu::security::GetKrb5ConfigFile;
 using kudu::subprocess::SubprocessMetrics;
 using std::move;
 using std::string;
@@ -162,7 +166,6 @@ static string GetJavaClasspath() {
   return Substitute("$0:$1", GetRangerJarPath(), FLAGS_ranger_config_path);
 }
 
-
 static string ranger_fifo_base() {
   DCHECK(!FLAGS_ranger_config_path.empty());
   const string& fifo_dir = FLAGS_ranger_receiver_fifo_dir.empty() ?
@@ -170,6 +173,17 @@ static string ranger_fifo_base() {
   return JoinPathSegments(fifo_dir, "ranger_receiever_fifo");
 }
 
+static string java_path() {
+  if (FLAGS_ranger_java_path.empty()) {
+    auto java_home = getenv("JAVA_HOME");
+    if (!java_home) {
+      return "java";
+    }
+    return JoinPathSegments(java_home, "bin/java");
+  }
+  return FLAGS_ranger_java_path;
+}
+
 // Builds the arguments to start the Ranger subprocess with the given receiver
 // fifo path. Specifically pass the principal and keytab file that the Ranger
 // subprocess will log in with if Kerberos is enabled. 'args' has the final
@@ -178,7 +192,12 @@ static Status BuildArgv(const string& fifo_path, vector<string>* argv) {
   DCHECK(argv);
   DCHECK(!FLAGS_ranger_config_path.empty());
   // Pass the required arguments to run the Ranger subprocess.
-  vector<string> ret = { FLAGS_ranger_java_path, "-cp", GetJavaClasspath(), kMainClass };
+  vector<string> ret = { java_path() };
+
+  ret.emplace_back(Substitute("-Djava.security.krb5.conf=$0", GetKrb5ConfigFile()));
+  ret.emplace_back("-cp");
+  ret.emplace_back(GetJavaClasspath());
+  ret.emplace_back(kMainClass);
   // When Kerberos is enabled in Kudu, pass both Kudu principal and keytab file
   // to the Ranger subprocess.
   if (!FLAGS_keytab_file.empty()) {
@@ -199,14 +218,14 @@ static Status BuildArgv(const string& fifo_path, vector<string>* argv) {
 static bool ValidateRangerConfiguration() {
   if (!FLAGS_ranger_config_path.empty()) {
     // First, check the specified path.
-    if (!Env::Default()->FileExists(FLAGS_ranger_java_path)) {
+    if (!Env::Default()->FileExists(java_path())) {
       // Otherwise, since the specified path is not absolute, check if
       // the Java binary is on the PATH.
       string p;
-      Status s = Subprocess::Call({ "which", FLAGS_ranger_java_path }, "", &p);
+      Status s = Subprocess::Call({ "which", java_path() }, "", &p);
       if (!s.ok()) {
         LOG(ERROR) << Substitute("--ranger_java_path has invalid java binary path: $0",
-                                 FLAGS_ranger_java_path);
+                                 java_path());
         return false;
       }
     }
diff --git a/src/kudu/ranger/ranger_client.h b/src/kudu/ranger/ranger_client.h
index 4578f6c..79ab45a 100644
--- a/src/kudu/ranger/ranger_client.h
+++ b/src/kudu/ranger/ranger_client.h
@@ -51,6 +51,12 @@ typedef subprocess::SubprocessProxy<RangerRequestListPB, RangerResponseListPB,
                                     RangerSubprocessMetrics> RangerSubprocess;
 
 // A client for the Ranger service that communicates with a Java subprocess.
+//
+// The Ranger subprocess itself is configured using xml files, like
+// core-site.xml and ranger-kudu-security.xml. The only configuration that is
+// coming from this class are environmental, like Java binary location, location
+// of config files, and krb5.conf file (setting it doesn't enable Kerberos, that
+// depends on core-site.xml).
 class RangerClient {
  public:
   // Similar to SentryAuthorizableScope scope which indicates the
diff --git a/src/kudu/security/init.cc b/src/kudu/security/init.cc
index 235a24a..5431865 100644
--- a/src/kudu/security/init.cc
+++ b/src/kudu/security/init.cc
@@ -507,6 +507,15 @@ Status InitKerberosForServer(const std::string& raw_principal, const std::string
   return Status::OK();
 }
 
+string GetKrb5ConfigFile() {
+  const char* config_file = getenv("KRB5_CONFIG");
+  if (!config_file) {
+    return "/etc/krb5.conf";
+  }
+
+  return string(config_file);
+}
+
 } // namespace security
 } // namespace kudu
 
diff --git a/src/kudu/security/init.h b/src/kudu/security/init.h
index 31dba47..8b76e94 100644
--- a/src/kudu/security/init.h
+++ b/src/kudu/security/init.h
@@ -92,5 +92,13 @@ Status MapPrincipalToLocalName(const std::string& principal, std::string* local_
 // with which one may Kinit.
 Status GetConfiguredPrincipal(const std::string& in_principal, std::string* out_principal);
 
+// Get the Kerberos config file location. It defaults to /etc/krb5.conf and it
+// can be overridden by the KRB5_CONFIG environment variable. As the Kerberos
+// libraries use the environment variable directly, this is not required
+// normally, but it can be useful if the file needs to be accessed directly
+// (e.g. when starting a Java subprocess, as Java doesn't respect the
+// environment variable).
+std::string GetKrb5ConfigFile();
+
 } // namespace security
 } // namespace kudu