You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2018/03/27 18:47:21 UTC

[7/7] impala git commit: IMPALA-6728: Always use Kudu based kinit if FLAGS_use_krpc=true

IMPALA-6728: Always use Kudu based kinit if FLAGS_use_krpc=true

We rely on the KPRC logic to do the Kerberos authentication
when KRPC is enabled. Therefore, when FLAGS_ues_krpc=true,
we must always call kudu::security::InitKerberosForServer()
to initialize the Kerberos related logic. This change makes
Impala ignore FLAGS_use_kudu_kinit=false when FLAGS_use_krpc=true.

Change-Id: Ia7086e5c9b460233e9e957f886141b3e6bba414b
Reviewed-on: http://gerrit.cloudera.org:8080/9797
Reviewed-by: Michael Ho <kw...@cloudera.com>
Tested-by: Impala Public Jenkins


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/3d52f8c9
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/3d52f8c9
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/3d52f8c9

Branch: refs/heads/2.x
Commit: 3d52f8c99955f01721cf5abdf3277490008dae60
Parents: 00f131c
Author: Michael Ho <kw...@cloudera.com>
Authored: Sat Mar 24 17:25:39 2018 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Tue Mar 27 11:01:40 2018 +0000

----------------------------------------------------------------------
 be/src/rpc/auth-provider.h            |  6 +--
 be/src/rpc/authentication.cc          | 13 ++++--
 be/src/rpc/rpc-mgr-kerberized-test.cc | 65 +++++++++++++++++++---------
 be/src/rpc/rpc-mgr-test-base.h        |  6 ---
 be/src/rpc/thrift-server-test.cc      | 69 +++++++++++++++++++-----------
 be/src/testutil/mini-kdc-wrapper.cc   | 59 +++++++++++--------------
 be/src/testutil/mini-kdc-wrapper.h    | 44 ++++++++++---------
 7 files changed, 149 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/3d52f8c9/be/src/rpc/auth-provider.h
----------------------------------------------------------------------
diff --git a/be/src/rpc/auth-provider.h b/be/src/rpc/auth-provider.h
index ee3bc5f..3e5517f 100644
--- a/be/src/rpc/auth-provider.h
+++ b/be/src/rpc/auth-provider.h
@@ -143,9 +143,9 @@ class SaslAuthProvider : public AuthProvider {
   /// function as a client.
   bool needs_kinit_;
 
-  /// Runs "RunKinit" below if needs_kinit_ is true and FLAGS_use_kudu_kinit is false.
-  /// Once started, this thread lives as long as the process does and periodically forks
-  /// impalad and execs the 'kinit' process.
+  /// Runs "RunKinit" below if needs_kinit_ is true and FLAGS_use_kudu_kinit is false
+  /// and FLAGS_use_krpc is false. Once started, this thread lives as long as the process
+  /// does and periodically forks impalad and execs the 'kinit' process.
   std::unique_ptr<Thread> kinit_thread_;
 
   /// Periodically (roughly once every FLAGS_kerberos_reinit_interval minutes) calls kinit

http://git-wip-us.apache.org/repos/asf/impala/blob/3d52f8c9/be/src/rpc/authentication.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/authentication.cc b/be/src/rpc/authentication.cc
index fc41723..4c3df50 100644
--- a/be/src/rpc/authentication.cc
+++ b/be/src/rpc/authentication.cc
@@ -67,6 +67,7 @@ using namespace apache::thrift;
 using namespace boost::filesystem;   // for is_regular()
 using namespace strings;
 
+DECLARE_bool(use_krpc);
 DECLARE_string(keytab_file);
 DECLARE_string(principal);
 DECLARE_string(be_principal);
@@ -107,11 +108,12 @@ DEFINE_string(internal_principals_whitelist, "hdfs", "(Advanced) Comma-separated
     "'hdfs' which is the system user that in certain deployments must access "
     "catalog server APIs.");
 
-// TODO: Remove this flag and the old kerberos code in a compatibility-breaking release.
+// TODO: Remove this flag and the old kerberos code once we remove 'use_krpc' flag.
 // (IMPALA-5893)
 DEFINE_bool(use_kudu_kinit, true, "If true, Impala will programatically perform kinit "
     "by calling into the libkrb5 library using the provided APIs. If false, it will fork "
-    "off a kinit process.");
+    "off a kinit process. If use_krpc=true, this flag is treated as true regardless of "
+    "what it's set to.");
 
 namespace impala {
 
@@ -840,7 +842,12 @@ Status SaslAuthProvider::Start() {
   if (needs_kinit_) {
     DCHECK(is_internal_);
     DCHECK(!principal_.empty());
-    if (FLAGS_use_kudu_kinit) {
+    if (FLAGS_use_kudu_kinit || FLAGS_use_krpc) {
+      // With KRPC enabled, we always rely on the Kudu library to carry out the Kerberos
+      // authentication during connection negotiation.
+      if (!FLAGS_use_kudu_kinit) {
+        LOG(INFO) << "Ignoring --use_kudu_kinit=false as KRPC and Kerberos are enabled";
+      }
       // Starts a thread that periodically does a 'kinit'. The thread lives as long as the
       // process does.
       KUDU_RETURN_IF_ERROR(kudu::security::InitKerberosForServer(principal_, keytab_file_,

http://git-wip-us.apache.org/repos/asf/impala/blob/3d52f8c9/be/src/rpc/rpc-mgr-kerberized-test.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/rpc-mgr-kerberized-test.cc b/be/src/rpc/rpc-mgr-kerberized-test.cc
index 6244c2d..141f359 100644
--- a/be/src/rpc/rpc-mgr-kerberized-test.cc
+++ b/be/src/rpc/rpc-mgr-kerberized-test.cc
@@ -18,45 +18,46 @@
 #include "rpc/rpc-mgr-test-base.h"
 #include "service/fe-support.h"
 
+DECLARE_bool(use_kudu_kinit);
+DECLARE_bool(use_krpc);
+
+DECLARE_string(be_principal);
+DECLARE_string(hostname);
+DECLARE_string(principal);
 DECLARE_string(ssl_client_ca_certificate);
 DECLARE_string(ssl_server_certificate);
 DECLARE_string(ssl_private_key);
 
-namespace impala {
+// The principal name and the realm used for creating the mini-KDC.
+// To be initialized at main().
+static string kdc_principal;
+static string kdc_realm;
 
-static int kdc_port = GetServerPort();
+namespace impala {
 
 class RpcMgrKerberizedTest :
     public RpcMgrTestBase<testing::TestWithParam<KerberosSwitch> > {
-  virtual void SetUp() override {
-    IpAddr ip;
-    ASSERT_OK(HostnameToIpAddr(FLAGS_hostname, &ip));
-    string spn = Substitute("impala-test/$0", ip);
 
-    kdc_wrapper_.reset(new MiniKdcWrapper(
-        std::move(spn), "KRBTEST.COM", "24h", "7d", kdc_port));
-    DCHECK(kdc_wrapper_.get() != nullptr);
-
-    ASSERT_OK(kdc_wrapper_->SetupAndStartMiniKDC(GetParam()));
+  virtual void SetUp() override {
+    KerberosSwitch k = GetParam();
+    FLAGS_use_krpc = true;
+    FLAGS_use_kudu_kinit = k == USE_KRPC_KUDU_KERBEROS;
+    FLAGS_principal = "dummy-service/host@realm";
+    FLAGS_be_principal = strings::Substitute("$0@$1", kdc_principal, kdc_realm);
     ASSERT_OK(InitAuth(CURRENT_EXECUTABLE_PATH));
-
     RpcMgrTestBase::SetUp();
   }
 
   virtual void TearDown() override {
-    ASSERT_OK(kdc_wrapper_->TearDownMiniKDC(GetParam()));
-    RpcMgrTestBase::TearDown();
+    FLAGS_principal.clear();
+    FLAGS_be_principal.clear();
   }
-
- private:
-  boost::scoped_ptr<MiniKdcWrapper> kdc_wrapper_;
 };
 
-// TODO: IMPALA-6477: This test breaks on CentOS 6.4. Re-enable after a fix.
 INSTANTIATE_TEST_CASE_P(KerberosOnAndOff,
                         RpcMgrKerberizedTest,
-                        ::testing::Values(USE_KUDU_KERBEROS,
-                                          USE_IMPALA_KERBEROS));
+                        ::testing::Values(USE_KRPC_IMPALA_KERBEROS,
+                                          USE_KRPC_KUDU_KERBEROS));
 
 TEST_P(RpcMgrKerberizedTest, MultipleServicesTls) {
   // TODO: We're starting a seperate RpcMgr here instead of configuring
@@ -81,12 +82,34 @@ TEST_P(RpcMgrKerberizedTest, MultipleServicesTls) {
 
 } // namespace impala
 
+using impala::Status;
+
 int main(int argc, char** argv) {
   ::testing::InitGoogleTest(&argc, argv);
   impala::InitCommonRuntime(argc, argv, true, impala::TestInfo::BE_TEST);
   impala::InitFeSupport();
 
+  // Set up and start KDC.
+  impala::IpAddr ip;
+  impala::Status status = impala::HostnameToIpAddr(FLAGS_hostname, &ip);
+  DCHECK(status.ok());
+  kdc_principal = Substitute("impala-test/$0", ip);
+  kdc_realm = "KRBTEST.COM";
+
+  int port = impala::FindUnusedEphemeralPort(nullptr);
+  std::unique_ptr<impala::MiniKdcWrapper> kdc;
+  status = impala::MiniKdcWrapper::SetupAndStartMiniKDC(
+      kdc_principal, kdc_realm, "24h", "7d", port, &kdc);
+  DCHECK(status.ok());
+
   // Fill in the path of the current binary for use by the tests.
   CURRENT_EXECUTABLE_PATH = argv[0];
-  return RUN_ALL_TESTS();
+  int retval = RUN_ALL_TESTS();
+
+  // Shutdown KDC.
+  status = kdc->TearDownMiniKDC();
+  DCHECK(status.ok());
+
+  return retval;
+
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/3d52f8c9/be/src/rpc/rpc-mgr-test-base.h
----------------------------------------------------------------------
diff --git a/be/src/rpc/rpc-mgr-test-base.h b/be/src/rpc/rpc-mgr-test-base.h
index ce063f8..f994fd8 100644
--- a/be/src/rpc/rpc-mgr-test-base.h
+++ b/be/src/rpc/rpc-mgr-test-base.h
@@ -67,12 +67,6 @@ namespace impala {
 
 static int32_t SERVICE_PORT = FindUnusedEphemeralPort(nullptr);
 
-int GetServerPort() {
-  int port = FindUnusedEphemeralPort(nullptr);
-  EXPECT_FALSE(port == -1);
-  return port;
-}
-
 const static string IMPALA_HOME(getenv("IMPALA_HOME"));
 const string& SERVER_CERT =
     Substitute("$0/be/src/testutil/server-cert.pem", IMPALA_HOME);

http://git-wip-us.apache.org/repos/asf/impala/blob/3d52f8c9/be/src/rpc/thrift-server-test.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-server-test.cc b/be/src/rpc/thrift-server-test.cc
index 8bd7275..f0a0bc5 100644
--- a/be/src/rpc/thrift-server-test.cc
+++ b/be/src/rpc/thrift-server-test.cc
@@ -35,6 +35,11 @@ using namespace strings;
 using namespace apache::thrift;
 using apache::thrift::transport::SSLProtocol;
 
+DECLARE_bool(use_kudu_kinit);
+DECLARE_bool(use_krpc);
+
+DECLARE_string(principal);
+DECLARE_string(be_principal);
 DECLARE_string(ssl_client_ca_certificate);
 DECLARE_string(ssl_cipher_list);
 DECLARE_string(ssl_minimum_version);
@@ -44,22 +49,26 @@ DECLARE_int32(state_store_port);
 DECLARE_int32(be_port);
 DECLARE_int32(beeswax_port);
 
-string IMPALA_HOME(getenv("IMPALA_HOME"));
-const string& SERVER_CERT =
+static string IMPALA_HOME(getenv("IMPALA_HOME"));
+static const string& SERVER_CERT =
     Substitute("$0/be/src/testutil/server-cert.pem", IMPALA_HOME);
-const string& PRIVATE_KEY =
+static const string& PRIVATE_KEY =
     Substitute("$0/be/src/testutil/server-key.pem", IMPALA_HOME);
-const string& BAD_SERVER_CERT =
+static const string& BAD_SERVER_CERT =
     Substitute("$0/be/src/testutil/bad-cert.pem", IMPALA_HOME);
-const string& BAD_PRIVATE_KEY =
+static const string& BAD_PRIVATE_KEY =
     Substitute("$0/be/src/testutil/bad-key.pem", IMPALA_HOME);
-const string& PASSWORD_PROTECTED_PRIVATE_KEY =
+static const string& PASSWORD_PROTECTED_PRIVATE_KEY =
     Substitute("$0/be/src/testutil/server-key-password.pem", IMPALA_HOME);
 
+// The principal name and the realm used for creating the mini-KDC.
+static const string kdc_principal = "impala/localhost";
+static const string kdc_realm = "KRBTEST.COM";
+
 // Only use TLSv1.0 compatible ciphers, as tests might run on machines with only TLSv1.0
 // support.
-const string TLS1_0_COMPATIBLE_CIPHER = "RC4-SHA";
-const string TLS1_0_COMPATIBLE_CIPHER_2 = "RC4-MD5";
+static const string TLS1_0_COMPATIBLE_CIPHER = "RC4-SHA";
+static const string TLS1_0_COMPATIBLE_CIPHER_2 = "RC4-MD5";
 
 /// Dummy server class (chosen because it has the smallest interface to implement) that
 /// tests can use to start Thrift servers.
@@ -81,8 +90,6 @@ int GetServerPort() {
   return port;
 }
 
-static int kdc_port = GetServerPort();
-
 template <class T> class ThriftTestBase : public T {
  protected:
   virtual void SetUp() {}
@@ -95,31 +102,33 @@ static string CURRENT_EXECUTABLE_PATH;
 
 class ThriftKerberizedParamsTest :
     public ThriftTestBase<testing::TestWithParam<KerberosSwitch> > {
-  virtual void SetUp() {
-    kdc_wrapper_.reset(new MiniKdcWrapper(
-        "impala/localhost", "KRBTEST.COM", "24h", "7d", kdc_port));
-    DCHECK(kdc_wrapper_.get() != nullptr);
 
-    ASSERT_OK(kdc_wrapper_->SetupAndStartMiniKDC(GetParam()));
+  virtual void SetUp() override {
+    KerberosSwitch k = GetParam();
+    FLAGS_use_krpc = false;
+    if (k == KERBEROS_OFF) {
+      FLAGS_principal.clear();
+      FLAGS_be_principal.clear();
+    } else {
+      FLAGS_use_kudu_kinit = k == USE_THRIFT_KUDU_KERBEROS;
+      FLAGS_principal = "dummy-service/host@realm";
+      FLAGS_be_principal = strings::Substitute("$0@$1", kdc_principal, kdc_realm);
+    }
     ASSERT_OK(InitAuth(CURRENT_EXECUTABLE_PATH));
-
     ThriftTestBase::SetUp();
   }
 
-  virtual void TearDown() {
-    ASSERT_OK(kdc_wrapper_->TearDownMiniKDC(GetParam()));
-    ThriftTestBase::TearDown();
+  virtual void TearDown() override {
+    FLAGS_principal.clear();
+    FLAGS_be_principal.clear();
   }
-
- private:
-  boost::scoped_ptr<MiniKdcWrapper> kdc_wrapper_;
 };
 
 INSTANTIATE_TEST_CASE_P(KerberosOnAndOff,
                         ThriftKerberizedParamsTest,
                         ::testing::Values(KERBEROS_OFF,
-                                          USE_KUDU_KERBEROS,
-                                          USE_IMPALA_KERBEROS));
+                                          USE_THRIFT_KUDU_KERBEROS,
+                                          USE_THRIFT_IMPALA_KERBEROS));
 
 TEST(ThriftTestBase, Connectivity) {
   int port = GetServerPort();
@@ -557,7 +566,17 @@ int main(int argc, char** argv) {
   ::testing::InitGoogleTest(&argc, argv);
   impala::InitCommonRuntime(argc, argv, false, impala::TestInfo::BE_TEST);
 
+  int port = impala::FindUnusedEphemeralPort(nullptr);
+  std::unique_ptr<impala::MiniKdcWrapper> kdc;
+  Status status = impala::MiniKdcWrapper::SetupAndStartMiniKDC(
+      kdc_principal, kdc_realm, "24h", "7d", port, &kdc);
+  DCHECK(status.ok());
+
   // Fill in the path of the current binary for use by the tests.
   CURRENT_EXECUTABLE_PATH = argv[0];
-  return RUN_ALL_TESTS();
+  int retval = RUN_ALL_TESTS();
+
+  status = kdc->TearDownMiniKDC();
+  DCHECK(status.ok());
+  return retval;
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/3d52f8c9/be/src/testutil/mini-kdc-wrapper.cc
----------------------------------------------------------------------
diff --git a/be/src/testutil/mini-kdc-wrapper.cc b/be/src/testutil/mini-kdc-wrapper.cc
index eb9d9f1..526e5b1 100644
--- a/be/src/testutil/mini-kdc-wrapper.cc
+++ b/be/src/testutil/mini-kdc-wrapper.cc
@@ -30,11 +30,7 @@ using namespace impala;
 namespace filesystem = boost::filesystem;
 using filesystem::path;
 
-DECLARE_bool(use_kudu_kinit);
-
 DECLARE_string(keytab_file);
-DECLARE_string(principal);
-DECLARE_string(be_principal);
 DECLARE_string(krb5_conf);
 
 Status MiniKdcWrapper::StartKdc(string keytab_dir) {
@@ -64,46 +60,41 @@ Status MiniKdcWrapper::CreateServiceKeytab(const string& spn, string* kt_path) {
   return Status::OK();
 }
 
-Status MiniKdcWrapper::SetupAndStartMiniKDC(KerberosSwitch k) {
-  if (k != KERBEROS_OFF) {
-    // Enable the workaround for MIT krb5 1.10 bugs from krb5_realm_override.cc.
-    setenv("KUDU_ENABLE_KRB5_REALM_FIX", "true", 0);
+Status MiniKdcWrapper::SetupAndStartMiniKDC(string spn, string realm,
+    string ticket_lifetime, string renew_lifetime, int kdc_port,
+    unique_ptr<MiniKdcWrapper>* kdc_ptr) {
+  std::unique_ptr<MiniKdcWrapper> kdc(new MiniKdcWrapper(
+      spn, realm, ticket_lifetime, renew_lifetime, kdc_port));
+  DCHECK(kdc.get() != nullptr);
 
-    FLAGS_use_kudu_kinit = k == USE_KUDU_KERBEROS;
+  // Enable the workaround for MIT krb5 1.10 bugs from krb5_realm_override.cc.
+  setenv("KUDU_ENABLE_KRB5_REALM_FIX", "true", 0);
 
-    // Check if the unique directory already exists, and create it if it doesn't.
-    RETURN_IF_ERROR(FileSystemUtil::RemoveAndCreateDirectory(unique_test_dir_.string()));
-    string keytab_dir = unique_test_dir_.string() + "/krb5kdc";
+  // Check if the unique directory already exists, and create it if it doesn't.
+  RETURN_IF_ERROR(FileSystemUtil::RemoveAndCreateDirectory(kdc->unique_test_dir_.string()));
+  string keytab_dir = kdc->unique_test_dir_.string() + "/krb5kdc";
 
-    RETURN_IF_ERROR(StartKdc(keytab_dir));
+  RETURN_IF_ERROR(kdc->StartKdc(keytab_dir));
 
-    string kt_path;
-    RETURN_IF_ERROR(CreateServiceKeytab(spn_, &kt_path));
+  string kt_path;
+  RETURN_IF_ERROR(kdc->CreateServiceKeytab(kdc->spn_, &kt_path));
 
-    // Set the appropriate flags based on how we've set up the kerberos environment.
-    FLAGS_krb5_conf = strings::Substitute("$0/$1", keytab_dir, "krb5.conf");
-    FLAGS_keytab_file = kt_path;
+  // Set the appropriate flags based on how we've set up the kerberos environment.
+  FLAGS_krb5_conf = strings::Substitute("$0/$1", keytab_dir, "krb5.conf");
+  FLAGS_keytab_file = kt_path;
 
-    // We explicitly set 'principal' and 'be_principal' even though 'principal' won't be
-    // used to test IMPALA-6256.
-    FLAGS_principal = "dummy-service/host@realm";
-    FLAGS_be_principal = strings::Substitute("$0@$1", spn_, realm_);
-  }
+  *kdc_ptr = std::move(kdc);
   return Status::OK();
 }
 
-Status MiniKdcWrapper::TearDownMiniKDC(KerberosSwitch k) {
-  if (k != KERBEROS_OFF) {
-    RETURN_IF_ERROR(StopKdc());
+Status MiniKdcWrapper::TearDownMiniKDC() {
+  RETURN_IF_ERROR(StopKdc());
 
-    // Clear the flags so we don't step on other tests that may run in the same process.
-    FLAGS_keytab_file.clear();
-    FLAGS_principal.clear();
-    FLAGS_be_principal.clear();
-    FLAGS_krb5_conf.clear();
+  // Clear the flags so we don't step on other tests that may run in the same process.
+  FLAGS_keytab_file.clear();
+  FLAGS_krb5_conf.clear();
 
-    // Remove test directory.
-    RETURN_IF_ERROR(FileSystemUtil::RemovePaths({unique_test_dir_.string()}));
-  }
+  // Remove test directory.
+  RETURN_IF_ERROR(FileSystemUtil::RemovePaths({unique_test_dir_.string()}));
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/3d52f8c9/be/src/testutil/mini-kdc-wrapper.h
----------------------------------------------------------------------
diff --git a/be/src/testutil/mini-kdc-wrapper.h b/be/src/testutil/mini-kdc-wrapper.h
index 1d5e0b8..17c174a 100644
--- a/be/src/testutil/mini-kdc-wrapper.h
+++ b/be/src/testutil/mini-kdc-wrapper.h
@@ -29,34 +29,26 @@ namespace impala {
 
 enum KerberosSwitch {
   KERBEROS_OFF,
-  USE_KUDU_KERBEROS,    // FLAGS_use_kudu_kinit = true
-  USE_IMPALA_KERBEROS   // FLAGS_use_kudu_kinit = false
+  USE_KRPC_KUDU_KERBEROS,    // FLAGS_use_kudu_kinit = true,  FLAGS_use_krpc = true
+  USE_KRPC_IMPALA_KERBEROS,  // FLAGS_use_kudu_kinit = false, FLAGS_use_krpc = true
+  USE_THRIFT_KUDU_KERBEROS,  // FLAGS_use_kudu_kinit = true,  FLAGS_use_krpc = false
+  USE_THRIFT_IMPALA_KERBEROS // FLAGS_use_kudu_kinit = false, FLAGS_use_krpc = false
 };
 
 /// This class allows tests to easily start and stop a KDC and configure Impala's auth
-/// layer.
-/// If the mode is USE_KUDU_KERBEROS or USE_IMPALA_KERBEROS, the MiniKdc which is a
-/// wrapper around the 'krb5kdc' process, is configured and started.
-/// If the mode is KERBEROS_OFF, Impala's auth layer is configured to use plain SASL and
-/// the KDC is not started.
+/// layer. A MiniKdc which is a wrapper around the 'krb5kdc' process, is configured and
+/// started.
 class MiniKdcWrapper {
  public:
-  MiniKdcWrapper(std::string spn, std::string realm, std::string ticket_lifetime,
-    std::string renew_lifetime,int kdc_port) :
-      spn_(spn),
-      realm_(realm),
-      ticket_lifetime_(ticket_lifetime),
-      renew_lifetime_(renew_lifetime),
-      kdc_port_(kdc_port) {
-  }
-
-  /// If 'k' is 'USE_KUDU_KERBEROS' or 'USE_IMPALA_KERBEROS', this function creates the
-  /// 'unique_test_dir_' path, starts the KDC and sets the appropriate flags that Impala
-  /// requires to run with Kerberos.
-  Status SetupAndStartMiniKDC(KerberosSwitch k);
+  /// This function creates the 'unique_test_dir_' path, starts the KDC and sets the
+  /// appropriate flags that Impala requires to run with Kerberos. The newly created
+  /// KDC is stored in 'kdc_ptr'. Return error status on failure.
+  static Status SetupAndStartMiniKDC(std::string spn, std::string realm,
+      std::string ticket_lifetime, std::string renew_lifetime, int kdc_port,
+      std::unique_ptr<MiniKdcWrapper>* kdc_ptr);
 
   /// Undoes everything done by SetupAndStartMiniKDC().
-  Status TearDownMiniKDC(KerberosSwitch k);
+  Status TearDownMiniKDC();
 
  private:
   boost::scoped_ptr<kudu::MiniKdc> kdc_;
@@ -79,6 +71,16 @@ class MiniKdcWrapper {
   /// Create a unique directory for this test to store its files in.
   boost::filesystem::path unique_test_dir_ = boost::filesystem::unique_path();
 
+  /// Called by SetupAndStartMiniKDC() only.
+  MiniKdcWrapper(std::string spn, std::string realm, std::string ticket_lifetime,
+    std::string renew_lifetime, int kdc_port) :
+      spn_(spn),
+      realm_(realm),
+      ticket_lifetime_(ticket_lifetime),
+      renew_lifetime_(renew_lifetime),
+      kdc_port_(kdc_port) {
+  }
+
   /// Starts the KDC and configures it to use 'keytab_dir' as the location to store the
   /// keytab. The 'keytab_dir' will not be cleaned up by this class.
   Status StartKdc(string keytab_dir);