You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by he...@apache.org on 2017/08/15 00:46:44 UTC

[5/6] incubator-impala git commit: IMPALA-4669: [SECURITY] Import Kudu security library from kudu@314c9d8

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/crypto.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/crypto.h b/be/src/kudu/security/crypto.h
new file mode 100644
index 0000000..619be4d
--- /dev/null
+++ b/be/src/kudu/security/crypto.h
@@ -0,0 +1,94 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <string>
+
+#include "kudu/security/openssl_util.h"
+
+// Forward declarations for the OpenSSL typedefs.
+typedef struct rsa_st RSA;
+typedef struct bio_st BIO;
+
+namespace kudu {
+
+class Status;
+
+namespace security {
+
+extern const size_t kNonceSize;
+
+// Supported message digests for data signing and signature verification.
+enum class DigestType {
+  SHA256,
+  SHA512,
+};
+
+// A class with generic public key interface, but actually it represents
+// an RSA key.
+class PublicKey : public RawDataWrapper<EVP_PKEY> {
+ public:
+  ~PublicKey() {}
+
+  Status FromString(const std::string& data, DataFormat format) WARN_UNUSED_RESULT;
+  Status ToString(std::string* data, DataFormat format) const WARN_UNUSED_RESULT;
+  Status FromFile(const std::string& fpath, DataFormat format) WARN_UNUSED_RESULT;
+
+  Status FromBIO(BIO* bio, DataFormat format) WARN_UNUSED_RESULT;
+
+  // Using the key, verify data signature using the specified message
+  // digest algorithm for signature verification.
+  // The input signature should be in in raw format (i.e. no base64 encoding).
+  Status VerifySignature(DigestType digest,
+                         const std::string& data,
+                         const std::string& signature) const WARN_UNUSED_RESULT;
+
+  // Sets 'equals' to true if the other public key equals this.
+  Status Equals(const PublicKey& other, bool* equals) const WARN_UNUSED_RESULT;
+};
+
+// A class with generic private key interface, but actually it represents
+// an RSA private key. It's important to have PrivateKey and PublicKey
+// be different types to avoid accidental leakage of private keys.
+class PrivateKey : public RawDataWrapper<EVP_PKEY> {
+ public:
+  ~PrivateKey() {}
+
+  Status FromString(const std::string& data, DataFormat format) WARN_UNUSED_RESULT;
+  Status ToString(std::string* data, DataFormat format) const WARN_UNUSED_RESULT;
+  Status FromFile(const std::string& fpath, DataFormat format) WARN_UNUSED_RESULT;
+
+  // Output the public part of the keypair into the specified placeholder.
+  Status GetPublicKey(PublicKey* public_key) const WARN_UNUSED_RESULT;
+
+  // Using the key, generate data signature using the specified
+  // message digest algorithm. The result signature is in raw format
+  // (i.e. no base64 encoding).
+  Status MakeSignature(DigestType digest,
+                       const std::string& data,
+                       std::string* signature) const WARN_UNUSED_RESULT;
+};
+
+// Utility method to generate private keys.
+Status GeneratePrivateKey(int num_bits, PrivateKey* ret) WARN_UNUSED_RESULT;
+
+// Generates a nonce of size kNonceSize, and writes it to the provided string.
+Status GenerateNonce(std::string* s) WARN_UNUSED_RESULT;
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/init.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/init.cc b/be/src/kudu/security/init.cc
new file mode 100644
index 0000000..dfdb5cd
--- /dev/null
+++ b/be/src/kudu/security/init.cc
@@ -0,0 +1,473 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/security/init.h"
+
+#include <krb5/krb5.h>
+
+#include <algorithm>
+#include <functional>
+#include <memory>
+#include <mutex>
+#include <random>
+#include <string>
+#include <vector>
+
+#include <boost/optional.hpp>
+
+#include "kudu/gutil/strings/util.h"
+#include "kudu/util/flags.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/rw_mutex.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/thread.h"
+
+DEFINE_string(keytab_file, "",
+              "Path to the Kerberos Keytab file for this server. Specifying a "
+              "keytab file will cause the server to kinit, and enable Kerberos "
+              "to be used to authenticate RPC connections.");
+TAG_FLAG(keytab_file, stable);
+
+DEFINE_string(principal, "kudu/_HOST",
+              "Kerberos principal that this daemon will log in as. The special token "
+              "_HOST will be replaced with the FQDN of the local host.");
+TAG_FLAG(principal, experimental);
+// This is currently tagged as unsafe because there is no way for users to configure
+// clients to expect a non-default principal. As such, configuring a server to login
+// as a different one would end up with a cluster that can't be connected to.
+// See KUDU-1884.
+TAG_FLAG(principal, unsafe);
+
+using std::mt19937;
+using std::random_device;
+using std::string;
+using std::uniform_int_distribution;
+using std::uniform_real_distribution;
+using std::vector;
+
+namespace kudu {
+namespace security {
+
+namespace {
+
+class KinitContext;
+
+// Global context for usage of the Krb5 library.
+krb5_context g_krb5_ctx;
+
+// Global instance of the context used by the kinit/renewal thread.
+KinitContext* g_kinit_ctx;
+
+// This lock is used to avoid a race while renewing the kerberos ticket.
+// The race can occur between the time we reinitialize the cache and the
+// time when we actually store the renewed credential back in the cache.
+RWMutex* g_kerberos_reinit_lock;
+
+class KinitContext {
+ public:
+  KinitContext();
+
+  // Equivalent implementation of 'kinit -kt <keytab path> <principal>'.
+  //
+  // This logs in from the given keytab as the given principal, returning
+  // RuntimeError if any part of this process fails.
+  //
+  // If the log-in is successful, then the default ticket cache is overwritten
+  // with the credentials of the newly logged-in principal.
+  Status Kinit(const string& keytab_path, const string& principal);
+
+  // Acquires a new Ticket Granting Ticket (TGT).
+  //
+  // Renews the existing ticket if possible, or acquires a new Ticket Granting
+  // Ticket (TGT).
+  Status DoRenewal();
+
+  // Calculates the next sleep interval based on the 'ticket_end_timestamp_' and
+  // adds some jitter so that all the nodes do not hit the KDC at the same time.
+  //
+  // If 'num_retries' > 0, it calls GetBackedOffRenewInterval() to return a backed
+  // off interval.
+  int32_t GetNextRenewInterval(uint32_t num_retries);
+
+  // Returns a value based on 'time_remaining' that increases exponentially with
+  // 'num_retries', with a random jitter of +/- 0%-50% of that value.
+  int32_t GetBackedOffRenewInterval(int32_t time_remaining, uint32_t num_retries);
+
+  const string& principal_str() const { return principal_str_; }
+  const string& username_str() const { return username_str_; }
+
+ private:
+  krb5_principal principal_;
+  krb5_keytab keytab_;
+  krb5_ccache ccache_;
+  krb5_get_init_creds_opt* opts_;
+
+  // The stringified principal and username that we are logged in as.
+  string principal_str_, username_str_;
+
+  // This is the time that the current TGT in use expires.
+  int32_t ticket_end_timestamp_;
+};
+
+Status Krb5CallToStatus(krb5_context ctx, krb5_error_code code) {
+  if (code == 0) return Status::OK();
+
+  std::unique_ptr<const char, std::function<void(const char*)>> err_msg(
+      krb5_get_error_message(ctx, code),
+      std::bind(krb5_free_error_message, ctx, std::placeholders::_1));
+  return Status::RuntimeError(err_msg.get());
+}
+#define KRB5_RETURN_NOT_OK_PREPEND(call, prepend) \
+  RETURN_NOT_OK_PREPEND(Krb5CallToStatus(g_krb5_ctx, (call)), (prepend))
+
+
+void InitKrb5Ctx() {
+  static std::once_flag once;
+  std::call_once(once, [&]() {
+      CHECK_EQ(krb5_init_context(&g_krb5_ctx), 0);
+    });
+}
+
+KinitContext::KinitContext() {}
+
+// Port of the data_eq() implementation from krb5/k5-int.h
+inline int data_eq(krb5_data d1, krb5_data d2) {
+    return (d1.length == d2.length && !memcmp(d1.data, d2.data, d1.length));
+}
+
+// Port of the data_eq_string() implementation from krb5/k5-int.h
+inline int data_eq_string(krb5_data d, const char *s) {
+    return (d.length == strlen(s) && !memcmp(d.data, s, d.length));
+}
+
+Status Krb5UnparseName(krb5_principal princ, string* name) {
+  char* c_name;
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_unparse_name(g_krb5_ctx, princ, &c_name),
+                             "krb5_unparse_name");
+  auto cleanup_name = MakeScopedCleanup([&]() {
+      krb5_free_unparsed_name(g_krb5_ctx, c_name);
+    });
+  *name = c_name;
+  return Status::OK();
+}
+
+// Periodically calls DoRenewal().
+void RenewThread() {
+  uint32_t failure_retries = 0;
+  while (true) {
+    // This thread is run immediately after the first Kinit, so sleep first.
+    SleepFor(MonoDelta::FromSeconds(g_kinit_ctx->GetNextRenewInterval(failure_retries)));
+
+    Status s = g_kinit_ctx->DoRenewal();
+    WARN_NOT_OK(s, "Kerberos reacquire error: ");
+    if (!s.ok()) {
+      ++failure_retries;
+    } else {
+      failure_retries = 0;
+    }
+  }
+}
+
+int32_t KinitContext::GetNextRenewInterval(uint32_t num_retries) {
+  int32_t time_remaining = ticket_end_timestamp_ - time(nullptr);
+
+  // If the last ticket renewal was a failure, we back off our retry attempts exponentially.
+  if (num_retries > 0) return GetBackedOffRenewInterval(time_remaining, num_retries);
+
+  // If the time remaining between now and ticket expiry is:
+  // * > 10 minutes:   We attempt to renew the ticket between 5 seconds and 5 minutes before the
+  //                   ticket expires.
+  // * 5 - 10 minutes: We attempt to renew the ticket betwen 5 seconds and 1 minute before the
+  //                   ticket expires.
+  // * < 5 minutes:    Attempt to renew the ticket every 'time_remaining'.
+  // The jitter is added to make sure that every server doesn't flood the KDC at the same time.
+  random_device rd;
+  mt19937 generator(rd());
+  if (time_remaining > 600) {
+    uniform_int_distribution<> dist(5, 300);
+    return time_remaining - dist(generator);
+  } else if (time_remaining > 300) {
+    uniform_int_distribution<> dist(5, 60);
+    return time_remaining - dist(generator);
+  }
+  return time_remaining;
+}
+
+int32_t KinitContext::GetBackedOffRenewInterval(int32_t time_remaining, uint32_t num_retries) {
+  // The minimum sleep interval after a failure will be 60 seconds.
+  int32_t next_interval = std::max(time_remaining, 60);
+  int32_t base_time = std::min(next_interval * (1 << num_retries), INT32_MAX);
+  random_device rd;
+  mt19937 generator(rd());
+  uniform_real_distribution<> dist(0.5, 1.5);
+  return static_cast<int32_t>(base_time * dist(generator));
+}
+
+Status KinitContext::DoRenewal() {
+
+  krb5_cc_cursor cursor;
+  // Setup a cursor to iterate through the credential cache.
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_start_seq_get(g_krb5_ctx, ccache_, &cursor),
+                             "Failed to peek into ccache");
+  auto cleanup_cursor = MakeScopedCleanup([&]() {
+      krb5_cc_end_seq_get(g_krb5_ctx, ccache_, &cursor); });
+
+  krb5_creds creds;
+  memset(&creds, 0, sizeof(krb5_creds));
+
+  krb5_error_code rc;
+  // Iterate through the credential cache.
+  while (!(rc = krb5_cc_next_cred(g_krb5_ctx, ccache_, &cursor, &creds))) {
+    auto cleanup_creds = MakeScopedCleanup([&]() {
+        krb5_free_cred_contents(g_krb5_ctx, &creds); });
+    if (krb5_is_config_principal(g_krb5_ctx, creds.server)) continue;
+
+    // We only want to renew the TGT (Ticket Granting Ticket). Ignore all other tickets.
+    // This follows the same format as is_local_tgt() from krb5:src/clients/klist/klist.c
+    if (creds.server->length != 2 ||
+        data_eq(creds.server->data[1], principal_->realm) == 0 ||
+        data_eq_string(creds.server->data[0], KRB5_TGS_NAME) == 0 ||
+        data_eq(creds.server->realm, principal_->realm) == 0) {
+      continue;
+    }
+
+    time_t now = time(nullptr);
+    time_t ticket_expiry = creds.times.endtime;
+    time_t renew_till = creds.times.renew_till;
+    time_t renew_deadline = renew_till - 30;
+
+    krb5_creds new_creds;
+    memset(&new_creds, 0, sizeof(krb5_creds));
+    auto cleanup_new_creds = MakeScopedCleanup([&]() {
+        krb5_free_cred_contents(g_krb5_ctx, &new_creds); });
+    // If the ticket has already expired or if there's only a short period before which the
+    // renew window closes, we acquire a new ticket.
+    if (ticket_expiry < now || renew_deadline < now) {
+      // Acquire a new ticket using the keytab. This ticket will automatically be put into the
+      // credential cache.
+      {
+        std::lock_guard<RWMutex> l(*g_kerberos_reinit_lock);
+        KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_keytab(g_krb5_ctx, &new_creds, principal_,
+                                                              keytab_, 0 /* valid from now */,
+                                                              nullptr /* TKT service name */,
+                                                              opts_),
+                                   "Reacquire error: unable to login from keytab");
+#ifdef __APPLE__
+        // Heimdal krb5 doesn't have the 'krb5_get_init_creds_opt_set_out_ccache' option,
+        // so use this alternate route.
+        KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_initialize(g_krb5_ctx, ccache_, principal_),
+                                   "Reacquire error: could not init ccache");
+
+        KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_store_cred(g_krb5_ctx, ccache_, &creds),
+                                   "Reacquire error: could not store creds in cache");
+#endif
+      }
+      LOG(INFO) << "Successfully reacquired a new kerberos TGT";
+    } else {
+      // Renew existing ticket.
+      KRB5_RETURN_NOT_OK_PREPEND(krb5_get_renewed_creds(g_krb5_ctx, &new_creds, principal_,
+                                                        ccache_, nullptr),
+                                 "Failed to renew ticket");
+
+      {
+        // Take the write lock here so that any connections undergoing negotiation have to wait
+        // until the new credentials are placed in the cache.
+        std::lock_guard<RWMutex> l(*g_kerberos_reinit_lock);
+        // Clear existing credentials in cache.
+        KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_initialize(g_krb5_ctx, ccache_, principal_),
+                                   "Failed to re-initialize ccache");
+
+        // Store the new credentials in the cache.
+        KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_store_cred(g_krb5_ctx, ccache_, &new_creds),
+                                   "Failed to store credentials in ccache");
+      }
+      LOG(INFO) << "Successfully renewed kerberos TGT";
+    }
+    ticket_end_timestamp_ = new_creds.times.endtime;
+    break;
+  }
+  return Status::OK();
+}
+
+Status KinitContext::Kinit(const string& keytab_path, const string& principal) {
+  InitKrb5Ctx();
+
+  // Parse the principal
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_parse_name(g_krb5_ctx, principal.c_str(), &principal_),
+                             "could not parse principal");
+
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_kt_resolve(g_krb5_ctx, keytab_path.c_str(), &keytab_),
+                             "unable to resolve keytab");
+
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_default(g_krb5_ctx, &ccache_),
+                             "unable to get default credentials cache");
+
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_opt_alloc(g_krb5_ctx, &opts_),
+                             "unable to allocate get_init_creds_opt struct");
+
+#ifndef __APPLE__
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_opt_set_out_ccache(g_krb5_ctx, opts_, ccache_),
+                             "unable to set init_creds options");
+#endif
+
+  krb5_creds creds;
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_keytab(g_krb5_ctx, &creds, principal_, keytab_,
+                                                        0 /* valid from now */,
+                                                        nullptr /* TKT service name */, opts_),
+                             "unable to login from keytab");
+  auto cleanup_creds = MakeScopedCleanup([&]() {
+      krb5_free_cred_contents(g_krb5_ctx, &creds); });
+
+  ticket_end_timestamp_ = creds.times.endtime;
+
+#ifdef __APPLE__
+  // Heimdal krb5 doesn't have the 'krb5_get_init_creds_opt_set_out_ccache' option,
+  // so use this alternate route.
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_initialize(g_krb5_ctx, ccache_, principal_),
+                             "could not init ccache");
+
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_store_cred(g_krb5_ctx, ccache_, &creds),
+                             "could not store creds in cache");
+#endif
+
+  // Convert the logged-in principal back to a string. This may be different than
+  // 'principal', since the default realm will be filled in based on the Kerberos
+  // configuration if not originally specified.
+  RETURN_NOT_OK_PREPEND(Krb5UnparseName(principal_, &principal_str_),
+                        "could not stringify the logged-in principal");
+  RETURN_NOT_OK_PREPEND(MapPrincipalToLocalName(principal_str_, &username_str_),
+                        "could not map own logged-in principal to a short username");
+
+  LOG(INFO) << "Logged in from keytab as " << principal_str_
+            << " (short username " << username_str_ << ")";
+
+  return Status::OK();
+}
+
+Status GetConfiguredPrincipal(string* principal) {
+  string p = FLAGS_principal;
+  string hostname;
+  // Try to fill in either the FQDN or hostname.
+  if (!GetFQDN(&hostname).ok()) {
+    RETURN_NOT_OK(GetHostname(&hostname));
+  }
+  GlobalReplaceSubstring("_HOST", hostname, &p);
+  *principal = p;
+  return Status::OK();
+}
+} // anonymous namespace
+
+
+RWMutex* KerberosReinitLock() {
+  return g_kerberos_reinit_lock;
+}
+
+Status CanonicalizeKrb5Principal(std::string* principal) {
+  InitKrb5Ctx();
+  krb5_principal princ;
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_parse_name(g_krb5_ctx, principal->c_str(), &princ),
+                             "could not parse principal");
+  auto cleanup = MakeScopedCleanup([&]() {
+      krb5_free_principal(g_krb5_ctx, princ);
+    });
+  RETURN_NOT_OK_PREPEND(Krb5UnparseName(princ, principal),
+                        "failed to convert principal back to string");
+  return Status::OK();
+}
+
+Status MapPrincipalToLocalName(const std::string& principal, std::string* local_name) {
+  InitKrb5Ctx();
+  krb5_principal princ;
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_parse_name(g_krb5_ctx, principal.c_str(), &princ),
+                             "could not parse principal");
+  auto cleanup = MakeScopedCleanup([&]() {
+      krb5_free_principal(g_krb5_ctx, princ);
+    });
+  char buf[1024];
+  krb5_error_code rc;
+#ifndef __APPLE__
+  rc = krb5_aname_to_localname(g_krb5_ctx, princ, arraysize(buf), buf);
+#else
+  // macOS's Heimdal library has a no-op implementation of
+  // krb5_aname_to_localname, so instead we fall down to below and grab the
+  // first component of the principal.
+  rc = KRB5_LNAME_NOTRANS;
+#endif
+  if (rc == KRB5_LNAME_NOTRANS) {
+    // No name mapping specified. We fall back to simply taking the first component
+    // of the principal, for compatibility with the default behavior of Hadoop.
+    // TODO(todd): we should support custom configured auth-to-local mapping, since
+    // most Hadoop ecosystem components do not load them from krb5.conf.
+    if (princ->length > 0) {
+      local_name->assign(princ->data[0].data, princ->data[0].length);
+      return Status::OK();
+    }
+    return Status::NotFound("unable to find first component of principal");
+  }
+  if (rc == KRB5_CONFIG_NOTENUFSPACE) {
+    return Status::InvalidArgument("mapped username too large");
+  }
+  KRB5_RETURN_NOT_OK_PREPEND(rc, "krb5_aname_to_localname");
+  if (strlen(buf) == 0) {
+    return Status::InvalidArgument("principal mapped to empty username");
+  }
+  local_name->assign(buf);
+  return Status::OK();
+}
+
+boost::optional<string> GetLoggedInPrincipalFromKeytab() {
+  if (!g_kinit_ctx) return boost::none;
+  return g_kinit_ctx->principal_str();
+}
+
+boost::optional<string> GetLoggedInUsernameFromKeytab() {
+  if (!g_kinit_ctx) return boost::none;
+  return g_kinit_ctx->username_str();
+}
+
+Status InitKerberosForServer() {
+  if (FLAGS_keytab_file.empty()) return Status::OK();
+
+  // Have the daemons use an in-memory ticket cache, so they don't accidentally
+  // pick up credentials from test cases or any other daemon.
+  // TODO(todd): extract these krb5 env vars into some constants since they're
+  // typo-prone.
+  setenv("KRB5CCNAME", "MEMORY:kudu", 1);
+  setenv("KRB5_KTNAME", FLAGS_keytab_file.c_str(), 1);
+
+  // KUDU-1897: disable the Kerberos replay cache. The KRPC protocol includes a
+  // per-connection server-generated nonce to protect against replay attacks
+  // when authenticating via Kerberos. The replay cache has many performance and
+  // implementation issues.
+  setenv("KRB5RCACHETYPE", "none", 1);
+
+  g_kinit_ctx = new KinitContext();
+  string principal;
+  RETURN_NOT_OK(GetConfiguredPrincipal(&principal));
+  RETURN_NOT_OK_PREPEND(g_kinit_ctx->Kinit(FLAGS_keytab_file, principal), "unable to kinit");
+
+  g_kerberos_reinit_lock = new RWMutex(RWMutex::Priority::PREFER_WRITING);
+  scoped_refptr<Thread> renew_thread;
+  // Start the renewal thread.
+  RETURN_NOT_OK(Thread::Create("kerberos", "renewal thread", &RenewThread, &renew_thread));
+
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/init.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/init.h b/be/src/kudu/security/init.h
new file mode 100644
index 0000000..61c9577
--- /dev/null
+++ b/be/src/kudu/security/init.h
@@ -0,0 +1,66 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+#pragma once
+
+#include <string>
+
+#include <boost/optional/optional_fwd.hpp>
+
+namespace kudu {
+
+class RWMutex;
+class Status;
+
+namespace security {
+
+// Initializes Kerberos for a server. In particular, this processes
+// the '--keytab_file' command line flag.
+Status InitKerberosForServer();
+
+// Returns the process lock 'kerberos_reinit_lock'
+// This lock is acquired in write mode while the ticket is being renewed, and
+// acquired in read mode before using the SASL library which might require a ticket.
+RWMutex* KerberosReinitLock();
+
+// Return the full principal (user/host@REALM) that the server has used to
+// log in from the keytab.
+//
+// If the server has not logged in from a keytab, returns boost::none.
+boost::optional<std::string> GetLoggedInPrincipalFromKeytab();
+
+// Same, but returns the mapped short username.
+boost::optional<std::string> GetLoggedInUsernameFromKeytab();
+
+// Canonicalize the given principal name by adding '@DEFAULT_REALM' in the case that
+// the principal has no realm.
+//
+// TODO(todd): move to kerberos_util.h in the later patch in this series (the file doesn't
+// exist yet, and trying to avoid rebase pain).
+Status CanonicalizeKrb5Principal(std::string* principal);
+
+// Map the given Kerberos principal 'principal' to a short username (i.e. with no realm or
+// host component).
+//
+// This respects the "auth-to-local" mappings from the system krb5.conf. However, if no such
+// mapping can be found, we fall back to simply taking the first component of the principal.
+//
+// TODO(todd): move to kerberos_util.h in the later patch in this series (the file doesn't
+// exist yet, and trying to avoid rebase pain).
+Status MapPrincipalToLocalName(const std::string& principal, std::string* local_name);
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/kerberos_util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/kerberos_util.cc b/be/src/kudu/security/kerberos_util.cc
new file mode 100644
index 0000000..09f1b82
--- /dev/null
+++ b/be/src/kudu/security/kerberos_util.cc
@@ -0,0 +1,36 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/security/kerberos_util.h"
+#include "kudu/gutil/strings/split.h"
+
+#include <array>
+#include <utility>
+
+namespace kudu {
+namespace security {
+
+std::array<StringPiece, 3> SplitKerberosPrincipal(StringPiece principal) {
+
+  std::pair<StringPiece, StringPiece> user_realm = strings::Split(principal, "@");
+  std::pair<StringPiece, StringPiece> princ_host = strings::Split(user_realm.first, "/");
+  return {{princ_host.first, princ_host.second, user_realm.second}};
+}
+
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/kerberos_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/kerberos_util.h b/be/src/kudu/security/kerberos_util.h
new file mode 100644
index 0000000..4c27a86
--- /dev/null
+++ b/be/src/kudu/security/kerberos_util.h
@@ -0,0 +1,29 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+#pragma once
+
+#include <array>
+
+class StringPiece;
+
+namespace kudu {
+namespace security {
+
+std::array<StringPiece, 3> SplitKerberosPrincipal(StringPiece principal);
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/krb5_realm_override.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/krb5_realm_override.cc b/be/src/kudu/security/krb5_realm_override.cc
new file mode 100644
index 0000000..3f875f1
--- /dev/null
+++ b/be/src/kudu/security/krb5_realm_override.cc
@@ -0,0 +1,104 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+// This file provides a workaround for tests running with Kerberos 1.11 or earlier.
+// These versions of Kerberos are missing a fix which allows service principals
+// to use IP addresses in their host component:
+//
+//   http://krbdev.mit.edu/rt/Ticket/Display.html?id=7603
+//
+// We use such principals in external minicluster tests, where servers have IP addresses
+// like 127.x.y.z that have no corresponding reverse DNS.
+//
+// The file contains an implementation of krb5_get_host_realm which wraps the one
+// in the Kerberos library. It detects the return code that indicates the
+// above problem and falls back to the default realm/
+//
+// The wrapper is injected via linking it into tests as well as the
+// "security" library. The linkage invocation uses the '-Wl,--undefined'
+// linker flag to force linking even though no symbol here is explicitly
+// referenced.
+
+#include <dlfcn.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include <krb5/krb5.h>
+#include <glog/logging.h>
+
+extern "C" {
+
+// This symbol is exported from the static library so that other static-linked binaries
+// can reference it and force this compilation unit to be linked. Otherwise the linker
+// thinks it's unused and doesn't link it.
+int krb5_realm_override_loaded = 1;
+
+// Save the original function from the Kerberos library itself.
+// We use dlsym() to load all of them, since this file gets linked into
+// some test binaries that themselves may not link against libkrb5.so at all.
+static void* g_orig_krb5_get_host_realm;
+static void* g_orig_krb5_get_default_realm;
+static void* g_orig_krb5_free_default_realm;
+
+// We only enable our workaround if this environment variable is set.
+constexpr static const char* kEnvVar = "KUDU_ENABLE_KRB5_REALM_FIX";
+
+#define CALL_ORIG(func_name, ...) \
+  ((decltype(&func_name))g_orig_ ## func_name)(__VA_ARGS__)
+
+__attribute__((constructor))
+static void init_orig_func() {
+  g_orig_krb5_get_host_realm = dlsym(RTLD_NEXT, "krb5_get_host_realm");
+  g_orig_krb5_get_default_realm = dlsym(RTLD_NEXT, "krb5_get_default_realm");
+  g_orig_krb5_free_default_realm = dlsym(RTLD_NEXT, "krb5_free_default_realm");
+}
+
+krb5_error_code krb5_get_host_realm(krb5_context context, const char* host, char*** realmsp) {
+  CHECK(g_orig_krb5_get_host_realm);
+  CHECK(g_orig_krb5_get_default_realm);
+  CHECK(g_orig_krb5_free_default_realm);
+
+  krb5_error_code rc = CALL_ORIG(krb5_get_host_realm, context, host, realmsp);
+  if (rc != KRB5_ERR_NUMERIC_REALM || getenv(kEnvVar) == nullptr) {
+    return rc;
+  }
+  // If we get KRB5_ERR_NUMERIC_REALM, this is indicative of a Kerberos version
+  // which has not provided support for numeric addresses as service host names
+  // So, we fill in the default realm instead.
+  char* default_realm;
+  rc = CALL_ORIG(krb5_get_default_realm, context, &default_realm);
+  if (rc != 0) {
+    return rc;
+  }
+
+  char** ret_realms;
+  ret_realms = static_cast<char**>(malloc(2 * sizeof(*ret_realms)));
+  if (ret_realms == nullptr) return ENOMEM;
+  ret_realms[0] = strdup(default_realm);
+  if (ret_realms[0] == nullptr) {
+    free(ret_realms);
+    return ENOMEM;
+  }
+  ret_realms[1] = 0;
+  *realmsp = ret_realms;
+
+  CALL_ORIG(krb5_free_default_realm, context, default_realm);
+  return 0;
+}
+
+} // extern "C"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/openssl_util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/openssl_util.cc b/be/src/kudu/security/openssl_util.cc
new file mode 100644
index 0000000..f5ce822
--- /dev/null
+++ b/be/src/kudu/security/openssl_util.cc
@@ -0,0 +1,211 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/security/openssl_util.h"
+
+#include <cstdio>
+#include <cstdlib>
+#include <mutex>
+#include <sstream>
+#include <string>
+
+#include <glog/logging.h>
+#include <openssl/err.h>
+#include <openssl/rand.h>
+#include <openssl/ssl.h>
+
+#include "kudu/util/debug/leakcheck_disabler.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+
+using std::ostringstream;
+using std::string;
+
+namespace kudu {
+namespace security {
+
+namespace {
+
+// Determine whether initialization was ever called.
+//
+// Thread safety:
+// - written by DoInitializeOpenSSL (single-threaded, due to std::call_once)
+// - read by DisableOpenSSLInitialization (must not be concurrent with above)
+bool g_ssl_is_initialized = false;
+
+// If true, then we expect someone else has initialized SSL.
+//
+// Thread safety:
+// - read by DoInitializeOpenSSL (single-threaded, due to std::call_once)
+// - written by DisableOpenSSLInitialization (must not be concurrent with above)
+bool g_disable_ssl_init = false;
+
+// Array of locks used by OpenSSL.
+// We use an intentionally-leaked C-style array here to avoid non-POD static data.
+Mutex* kCryptoLocks = nullptr;
+
+// Lock/Unlock the nth lock. Only to be used by OpenSSL.
+void LockingCB(int mode, int type, const char* /*file*/, int /*line*/) {
+  DCHECK(kCryptoLocks);
+  Mutex* m = &kCryptoLocks[type];
+  if (mode & CRYPTO_LOCK) {
+    m->lock();
+  } else {
+    m->unlock();
+  }
+}
+
+Status CheckOpenSSLInitialized() {
+  if (!CRYPTO_get_locking_callback()) {
+    return Status::RuntimeError("Locking callback not initialized");
+  }
+  auto ctx = ssl_make_unique(SSL_CTX_new(SSLv23_method()));
+  if (!ctx) {
+    ERR_clear_error();
+    return Status::RuntimeError("SSL library appears uninitialized (cannot create SSL_CTX)");
+  }
+  return Status::OK();
+}
+
+void DoInitializeOpenSSL() {
+  // In case the user's thread has left some error around, clear it.
+  ERR_clear_error();
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  if (g_disable_ssl_init) {
+    VLOG(2) << "Not initializing OpenSSL (disabled by application)";
+    return;
+  }
+
+  // Check that OpenSSL isn't already initialized. If it is, it's likely
+  // we are embedded in (or embedding) another application/library which
+  // initializes OpenSSL, and we risk installing conflicting callbacks
+  // or crashing due to concurrent initialization attempts. In that case,
+  // log a warning.
+  auto ctx = ssl_make_unique(SSL_CTX_new(SSLv23_method()));
+  if (ctx) {
+    LOG(WARNING) << "It appears that OpenSSL has been previously initialized by "
+                 << "code outside of Kudu. Please use kudu::client::DisableOpenSSLInitialization() "
+                 << "to avoid potential crashes due to conflicting initialization.";
+    // Continue anyway; all of the below is idempotent, except for the locking callback,
+    // which we check before overriding. They aren't thread-safe, however -- that's why
+    // we try to get embedding applications to do the right thing here rather than risk a
+    // potential initialization race.
+  } else {
+    // As expected, SSL is not initialized, so SSL_CTX_new() failed. Make sure
+    // it didn't leave anything in our error queue.
+    ERR_clear_error();
+  }
+
+  SSL_load_error_strings();
+  SSL_library_init();
+  OpenSSL_add_all_algorithms();
+  RAND_poll();
+
+  if (!CRYPTO_get_locking_callback()) {
+    // Initialize the OpenSSL mutexes. We intentionally leak these, so ignore
+    // LSAN warnings.
+    debug::ScopedLeakCheckDisabler d;
+    int num_locks = CRYPTO_num_locks();
+    CHECK(!kCryptoLocks);
+    kCryptoLocks = new Mutex[num_locks];
+
+    // Callbacks used by OpenSSL required in a multi-threaded setting.
+    CRYPTO_set_locking_callback(LockingCB);
+  }
+
+  g_ssl_is_initialized = true;
+}
+
+} // anonymous namespace
+
+Status DisableOpenSSLInitialization() {
+  if (g_disable_ssl_init) return Status::OK();
+  if (g_ssl_is_initialized) {
+    return Status::IllegalState("SSL already initialized. Initialization can only be disabled "
+                                "before first usage.");
+  }
+  RETURN_NOT_OK(CheckOpenSSLInitialized());
+  g_disable_ssl_init = true;
+  return Status::OK();
+}
+
+void InitializeOpenSSL() {
+  static std::once_flag ssl_once;
+  std::call_once(ssl_once, DoInitializeOpenSSL);
+}
+
+string GetOpenSSLErrors() {
+  ostringstream serr;
+  uint32_t l;
+  int line, flags;
+  const char *file, *data;
+  bool is_first = true;
+  while ((l = ERR_get_error_line_data(&file, &line, &data, &flags)) != 0) {
+    if (is_first) {
+      is_first = false;
+    } else {
+      serr << " ";
+    }
+
+    char buf[256];
+    ERR_error_string_n(l, buf, sizeof(buf));
+    serr << buf << ":" << file << ":" << line;
+    if (flags & ERR_TXT_STRING) {
+      serr << ":" << data;
+    }
+  }
+  return serr.str();
+}
+
+string GetSSLErrorDescription(int error_code) {
+  switch (error_code) {
+    case SSL_ERROR_NONE: return "";
+    case SSL_ERROR_ZERO_RETURN: return "SSL_ERROR_ZERO_RETURN";
+    case SSL_ERROR_WANT_READ: return "SSL_ERROR_WANT_READ";
+    case SSL_ERROR_WANT_WRITE: return "SSL_ERROR_WANT_WRITE";
+    case SSL_ERROR_WANT_CONNECT: return "SSL_ERROR_WANT_CONNECT";
+    case SSL_ERROR_WANT_ACCEPT: return "SSL_ERROR_WANT_ACCEPT";
+    case SSL_ERROR_WANT_X509_LOOKUP: return "SSL_ERROR_WANT_X509_LOOKUP";
+    case SSL_ERROR_SYSCALL: {
+      string queued_error = GetOpenSSLErrors();
+      if (!queued_error.empty()) {
+        return queued_error;
+      }
+      return kudu::ErrnoToString(errno);
+    };
+    default: return GetOpenSSLErrors();
+  }
+}
+
+const string& DataFormatToString(DataFormat fmt) {
+  static const string kStrFormatUnknown = "UNKNOWN";
+  static const string kStrFormatDer = "DER";
+  static const string kStrFormatPem = "PEM";
+  switch (fmt) {
+    case DataFormat::DER:
+      return kStrFormatDer;
+    case DataFormat::PEM:
+      return kStrFormatPem;
+    default:
+      return kStrFormatUnknown;
+  }
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/openssl_util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/openssl_util.h b/be/src/kudu/security/openssl_util.h
new file mode 100644
index 0000000..db85a85
--- /dev/null
+++ b/be/src/kudu/security/openssl_util.h
@@ -0,0 +1,190 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <functional>
+#include <memory>
+#include <string>
+
+#include <openssl/err.h>
+#include <openssl/pem.h>
+#include <openssl/ssl.h>
+#include <openssl/x509.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/status.h"
+
+// Forward declarations for the OpenSSL typedefs.
+typedef struct X509_req_st X509_REQ;
+typedef struct bio_st BIO;
+typedef struct evp_pkey_st EVP_PKEY;
+typedef struct ssl_ctx_st SSL_CTX;
+typedef struct ssl_st SSL;
+typedef struct x509_st X509;
+
+#define OPENSSL_CHECK_OK(call) \
+  CHECK_GT((call), 0)
+
+#define OPENSSL_RET_NOT_OK(call, msg) \
+  if ((call) <= 0) { \
+    return Status::RuntimeError((msg), GetOpenSSLErrors()); \
+  }
+
+#define OPENSSL_RET_IF_NULL(call, msg) \
+  if ((call) == nullptr) { \
+    return Status::RuntimeError((msg), GetOpenSSLErrors()); \
+  }
+
+// Scoped helper which DCHECKs that on both scope entry and exit, there are no
+// pending OpenSSL errors for the current thread.
+//
+// This allows us to avoid calling ERR_clear_error() defensively before every
+// OpenSSL call, but rather call it only when we get an error code indicating
+// there may be some pending error.
+//
+// Example usage:
+//
+//    void MyFunc() {
+//      SCOPED_OPENSSL_NO_PENDING_ERRORS;
+//      ... use OpenSSL APIs ...
+//    }
+#define SCOPED_OPENSSL_NO_PENDING_ERRORS \
+  kudu::security::internal::ScopedCheckNoPendingSSLErrors _no_ssl_errors(__PRETTY_FUNCTION__)
+
+namespace kudu {
+namespace security {
+
+// Disable initialization of OpenSSL. Must be called before
+// any call to InitializeOpenSSL().
+Status DisableOpenSSLInitialization() WARN_UNUSED_RESULT;
+
+// Initializes static state required by the OpenSSL library.
+// This is a no-op if DisableOpenSSLInitialization() has been called.
+//
+// Safe to call multiple times.
+void InitializeOpenSSL();
+
+// Fetches errors from the OpenSSL error error queue, and stringifies them.
+//
+// The error queue will be empty after this method returns.
+//
+// See man(3) ERR_get_err for more discussion.
+std::string GetOpenSSLErrors();
+
+// Returns a string representation of the provided error code, which must be
+// from a prior call to the SSL_get_error function.
+//
+// If necessary, the OpenSSL error queue may be inspected and emptied as part of
+// this call, and/or 'errno' may be inspected. As a result, this method should
+// only be used directly after the error occurs, and from the same thread.
+//
+// See man(3) SSL_get_error for more discussion.
+std::string GetSSLErrorDescription(int error_code);
+
+// A generic wrapper for OpenSSL structures.
+template <typename T>
+using c_unique_ptr = std::unique_ptr<T, std::function<void(T*)>>;
+
+// For each SSL type, the Traits class provides the important OpenSSL
+// API functions.
+template<typename SSL_TYPE>
+struct SslTypeTraits {};
+
+template<> struct SslTypeTraits<X509> {
+  static constexpr auto free = &X509_free;
+  static constexpr auto read_pem = &PEM_read_bio_X509;
+  static constexpr auto read_der = &d2i_X509_bio;
+  static constexpr auto write_pem = &PEM_write_bio_X509;
+  static constexpr auto write_der = &i2d_X509_bio;
+};
+template<> struct SslTypeTraits<X509_EXTENSION> {
+  static constexpr auto free = &X509_EXTENSION_free;
+};
+template<> struct SslTypeTraits<X509_REQ> {
+  static constexpr auto free = &X509_REQ_free;
+  static constexpr auto read_pem = &PEM_read_bio_X509_REQ;
+  static constexpr auto read_der = &d2i_X509_REQ_bio;
+  static constexpr auto write_pem = &PEM_write_bio_X509_REQ;
+  static constexpr auto write_der = &i2d_X509_REQ_bio;
+};
+template<> struct SslTypeTraits<EVP_PKEY> {
+  static constexpr auto free = &EVP_PKEY_free;
+};
+template<> struct SslTypeTraits<SSL_CTX> {
+  static constexpr auto free = &SSL_CTX_free;
+};
+
+template<typename SSL_TYPE, typename Traits = SslTypeTraits<SSL_TYPE>>
+c_unique_ptr<SSL_TYPE> ssl_make_unique(SSL_TYPE* d) {
+  return {d, Traits::free};
+}
+
+// Acceptable formats for keys, X509 certificates and X509 CSRs.
+enum class DataFormat {
+  DER = 0,    // DER/ASN1 format (binary): for representing object on the wire
+  PEM = 1,    // PEM format (ASCII): for storing on filesystem, printing, etc.
+};
+
+// Data format representation as a string.
+const std::string& DataFormatToString(DataFormat fmt);
+
+// Template wrapper for dynamically allocated entities with custom deleter.
+// Mostly, using it for xxx_st types from the OpenSSL crypto library.
+template<typename Type>
+class RawDataWrapper {
+ public:
+  typedef Type RawDataType;
+
+  RawDataType* GetRawData() const {
+    return data_.get();
+  }
+
+  void AdoptRawData(RawDataType* d) {
+    data_ = ssl_make_unique(d);
+  }
+
+ protected:
+  c_unique_ptr<RawDataType> data_;
+};
+
+
+namespace internal {
+
+// Implementation of SCOPED_OPENSSL_NO_PENDING_ERRORS. Use the macro form
+// instead of directly instantiating the implementation class.
+struct ScopedCheckNoPendingSSLErrors {
+ public:
+  explicit ScopedCheckNoPendingSSLErrors(const char* func)
+      : func_(func) {
+    DCHECK_EQ(ERR_peek_error(), 0)
+        << "Expected no pending OpenSSL errors on " << func_
+        << " entry, but had: " << GetOpenSSLErrors();
+  }
+  ~ScopedCheckNoPendingSSLErrors() {
+    DCHECK_EQ(ERR_peek_error(), 0)
+        << "Expected no pending OpenSSL errors on " << func_
+        << " exit, but had: " << GetOpenSSLErrors();
+  }
+
+ private:
+  const char* const func_;
+};
+
+} // namespace internal
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/openssl_util_bio.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/openssl_util_bio.h b/be/src/kudu/security/openssl_util_bio.h
new file mode 100644
index 0000000..6ab6a55
--- /dev/null
+++ b/be/src/kudu/security/openssl_util_bio.h
@@ -0,0 +1,113 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "kudu/security/openssl_util.h"
+
+#include <string>
+
+#include <glog/logging.h>
+#include <openssl/bio.h>
+#include <openssl/buffer.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace security {
+
+template<> struct SslTypeTraits<BIO> {
+  static constexpr auto free = &BIO_free;
+};
+
+template<typename TYPE, typename Traits = SslTypeTraits<TYPE>>
+Status ToBIO(BIO* bio, DataFormat format, TYPE* obj) {
+  CHECK(bio);
+  CHECK(obj);
+  switch (format) {
+    case DataFormat::DER:
+      OPENSSL_RET_NOT_OK(Traits::write_der(bio, obj),
+          "error exporting data in DER format");
+      break;
+    case DataFormat::PEM:
+      OPENSSL_RET_NOT_OK(Traits::write_pem(bio, obj),
+          "error exporting data in PEM format");
+      break;
+  }
+  OPENSSL_RET_NOT_OK(BIO_flush(bio), "error flushing BIO");
+  return Status::OK();
+}
+
+template<typename TYPE, typename Traits = SslTypeTraits<TYPE>>
+Status FromBIO(BIO* bio, DataFormat format, c_unique_ptr<TYPE>* ret) {
+  CHECK(bio);
+  switch (format) {
+    case DataFormat::DER:
+      *ret = ssl_make_unique(Traits::read_der(bio, nullptr));
+      break;
+    case DataFormat::PEM:
+      *ret = ssl_make_unique(Traits::read_pem(bio, nullptr, nullptr, nullptr));
+      break;
+  }
+  if (PREDICT_FALSE(!*ret)) {
+    return Status::RuntimeError(GetOpenSSLErrors());
+  }
+  return Status::OK();
+}
+
+template<typename Type, typename Traits = SslTypeTraits<Type>>
+Status FromString(const string& data, DataFormat format,
+                  c_unique_ptr<Type>* ret) {
+  const void* mdata = reinterpret_cast<const void*>(data.data());
+  auto bio = ssl_make_unique(BIO_new_mem_buf(
+#if OPENSSL_VERSION_NUMBER < 0x10002000L
+      const_cast<void*>(mdata),
+#else
+      mdata,
+#endif
+      data.size()));
+  RETURN_NOT_OK_PREPEND((FromBIO<Type, Traits>(bio.get(), format, ret)),
+                        "unable to load data from memory");
+  return Status::OK();
+}
+
+template<typename Type, typename Traits = SslTypeTraits<Type>>
+Status ToString(std::string* data, DataFormat format, Type* obj) {
+  CHECK(data);
+  auto bio = ssl_make_unique(BIO_new(BIO_s_mem()));
+  RETURN_NOT_OK_PREPEND((ToBIO<Type, Traits>(bio.get(), format, obj)),
+                        "error serializing data");
+  BUF_MEM* membuf;
+  OPENSSL_CHECK_OK(BIO_get_mem_ptr(bio.get(), &membuf));
+  data->assign(membuf->data, membuf->length);
+  return Status::OK();
+}
+
+template<typename Type, typename Traits = SslTypeTraits<Type>>
+Status FromFile(const string& fpath, DataFormat format,
+                c_unique_ptr<Type>* ret) {
+  auto bio = ssl_make_unique(BIO_new(BIO_s_file()));
+  OPENSSL_RET_NOT_OK(BIO_read_filename(bio.get(), fpath.c_str()),
+      strings::Substitute("could not read data from file '$0'", fpath));
+  RETURN_NOT_OK_PREPEND((FromBIO<Type, Traits>(bio.get(), format, ret)),
+      strings::Substitute("unable to load data from file '$0'", fpath));
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/security-test-util.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/security-test-util.cc b/be/src/kudu/security/security-test-util.cc
new file mode 100644
index 0000000..b2c8020
--- /dev/null
+++ b/be/src/kudu/security/security-test-util.cc
@@ -0,0 +1,94 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/security/security-test-util.h"
+
+#include <glog/logging.h>
+
+#include "kudu/security/ca/cert_management.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+namespace security {
+
+using ca::CaCertRequestGenerator;
+using ca::CertSigner;
+
+Status GenerateSelfSignedCAForTests(PrivateKey* ca_key, Cert* ca_cert) {
+  static const int64_t kRootCaCertExpirationSeconds = 24 * 60 * 60;
+  // Create a key for the self-signed CA.
+  RETURN_NOT_OK(GeneratePrivateKey(512, ca_key));
+
+  CaCertRequestGenerator::Config config = { "test-ca-cn" };
+  RETURN_NOT_OK(CertSigner::SelfSignCA(*ca_key,
+                                       config,
+                                       kRootCaCertExpirationSeconds,
+                                       ca_cert));
+  return Status::OK();
+}
+
+std::ostream& operator<<(std::ostream& o, PkiConfig c) {
+    switch (c) {
+      case PkiConfig::NONE: o << "NONE"; break;
+      case PkiConfig::SELF_SIGNED: o << "SELF_SIGNED"; break;
+      case PkiConfig::TRUSTED: o << "TRUSTED"; break;
+      case PkiConfig::SIGNED: o << "SIGNED"; break;
+      case PkiConfig::EXTERNALLY_SIGNED: o << "EXTERNALLY_SIGNED"; break;
+    }
+    return o;
+}
+
+Status ConfigureTlsContext(PkiConfig config,
+                           const Cert& ca_cert,
+                           const PrivateKey& ca_key,
+                           TlsContext* tls_context) {
+  switch (config) {
+    case PkiConfig::NONE: break;
+    case PkiConfig::SELF_SIGNED:
+      RETURN_NOT_OK(tls_context->GenerateSelfSignedCertAndKey());
+      break;
+    case PkiConfig::TRUSTED:
+      RETURN_NOT_OK(tls_context->AddTrustedCertificate(ca_cert));
+      break;
+    case PkiConfig::SIGNED: {
+      RETURN_NOT_OK(tls_context->AddTrustedCertificate(ca_cert));
+      RETURN_NOT_OK(tls_context->GenerateSelfSignedCertAndKey());
+      Cert cert;
+      RETURN_NOT_OK(CertSigner(&ca_cert, &ca_key).Sign(*tls_context->GetCsrIfNecessary(), &cert));
+      RETURN_NOT_OK(tls_context->AdoptSignedCert(cert));
+      break;
+    };
+    case PkiConfig::EXTERNALLY_SIGNED: {
+      // Write certificate to file.
+      std::string cert_path = JoinPathSegments(GetTestDataDirectory(), "kudu-test-cert.pem");
+      RETURN_NOT_OK(CreateSSLServerCert(cert_path));
+      // Write private key to file.
+      std::string key_path = JoinPathSegments(GetTestDataDirectory(), "kudu-test-key.pem");
+      RETURN_NOT_OK(CreateSSLPrivateKey(key_path));
+      RETURN_NOT_OK(tls_context->LoadCertificateAndKey(cert_path, key_path));
+      RETURN_NOT_OK(tls_context->LoadCertificateAuthority(cert_path));
+    };
+  }
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/security-test-util.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/security-test-util.h b/be/src/kudu/security/security-test-util.h
new file mode 100644
index 0000000..f0b8977
--- /dev/null
+++ b/be/src/kudu/security/security-test-util.h
@@ -0,0 +1,130 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <ostream>
+#include <string>
+
+#include "kudu/util/env.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace security {
+
+class Cert;
+class PrivateKey;
+class PublicKey;
+class TlsContext;
+
+// TODO(todd): consolidate these certs with those in
+// security/test/test_certs.h once we support configuring a password
+// for the RPC cert.
+inline static Status CreateSSLServerCert(const std::string& file_path) {
+  static const char* test_server_cert = R"(
+-----BEGIN CERTIFICATE-----
+MIIEejCCA2KgAwIBAgIJAKMdvDR5PL82MA0GCSqGSIb3DQEBBQUAMIGEMQswCQYD
+VQQGEwJVUzETMBEGA1UECBMKQ2FsaWZvcm5pYTEWMBQGA1UEBxMNU2FuIEZyYW5j
+aXNjbzERMA8GA1UEChMIQ2xvdWRlcmExEjAQBgNVBAMTCWxvY2FsaG9zdDEhMB8G
+CSqGSIb3DQEJARYSaGVucnlAY2xvdWRlcmEuY29tMB4XDTEzMDkyMjAwMjUxOFoX
+DTQxMDIwNzAwMjUxOFowgYQxCzAJBgNVBAYTAlVTMRMwEQYDVQQIEwpDYWxpZm9y
+bmlhMRYwFAYDVQQHEw1TYW4gRnJhbmNpc2NvMREwDwYDVQQKEwhDbG91ZGVyYTES
+MBAGA1UEAxMJbG9jYWxob3N0MSEwHwYJKoZIhvcNAQkBFhJoZW5yeUBjbG91ZGVy
+YS5jb20wggEiMA0GCSqGSIb3DQEBAQUAA4IBDwAwggEKAoIBAQCoUj3pMQ2ELkrz
+zq+koixljVFBAEEqwUWSjA+GEKwfFb/UPRjeO/wrKndp2r83jc6KRt66rvAIl8cr
+b54yTOsJ/ZcARrjTwG3IG8Tely/54ZQyH0ImdJyEbCSoI04zX3ovjlppz3g5xanj
+WmpAh6pzPgBOTfisCLMPD70xQ8F//QWZdNatoly54STkTWoJv/Oll/UpXcBY8JOR
++ytX82eGgG4F8YoQqmbjrrN5JAmqLRiBAkr3WUena6ekqJBalJRzex/Wh8a9XEV7
+9HFVVngBhezsOJgf81hzBzzhULKfxuXl8uaUj3Z9cZg39CDsyz+ULYbsPm8VoMUI
+VCf7MUVTAgMBAAGjgewwgekwHQYDVR0OBBYEFK94kea7jIKQawAIb+0DqsA1rf6n
+MIG5BgNVHSMEgbEwga6AFK94kea7jIKQawAIb+0DqsA1rf6noYGKpIGHMIGEMQsw
+CQYDVQQGEwJVUzETMBEGA1UECBMKQ2FsaWZvcm5pYTEWMBQGA1UEBxMNU2FuIEZy
+YW5jaXNjbzERMA8GA1UEChMIQ2xvdWRlcmExEjAQBgNVBAMTCWxvY2FsaG9zdDEh
+MB8GCSqGSIb3DQEJARYSaGVucnlAY2xvdWRlcmEuY29tggkAox28NHk8vzYwDAYD
+VR0TBAUwAwEB/zANBgkqhkiG9w0BAQUFAAOCAQEAEtkPPncCnN2IFVJvz04K+VsX
+b6w3qwPynQKc67+++JkNb3TYKrh/0UVM1NrEOu3TGplqOrKgAlITuaWNqNOSBu1R
+WJtrz85YkonED5awjjuALVEY82+c7pOXkuv5G5421RINfRn2hNzgw8VFb5CEvxHH
+jER80Vx6UGKr/S649qTQ8AzVzTwWS86VsGI2azAD7D67G/IDGf+0P7FsXonKY+vl
+vKzkfaO1+qEOLtDHV9mwlsxl3Re/MNym4ExWHi9txynCNiRZHqWoZUS+KyYqIR2q
+seCrQwgi1Fer9Ekd5XNjWjigC3VC3SjMqWaxeKbZ2/AuABJMz5xSiRkgwphXEQ==
+-----END CERTIFICATE-----
+  )";
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), test_server_cert, file_path));
+  return Status::OK();
+}
+
+// Writes the test SSL private key into a temporary file.
+inline static Status CreateSSLPrivateKey(const std::string& file_path) {
+  static const char* test_private_key = R"(
+-----BEGIN RSA PRIVATE KEY-----
+MIIEpAIBAAKCAQEAqFI96TENhC5K886vpKIsZY1RQQBBKsFFkowPhhCsHxW/1D0Y
+3jv8Kyp3adq/N43Oikbeuq7wCJfHK2+eMkzrCf2XAEa408BtyBvE3pcv+eGUMh9C
+JnSchGwkqCNOM196L45aac94OcWp41pqQIeqcz4ATk34rAizDw+9MUPBf/0FmXTW
+raJcueEk5E1qCb/zpZf1KV3AWPCTkfsrV/NnhoBuBfGKEKpm466zeSQJqi0YgQJK
+91lHp2unpKiQWpSUc3sf1ofGvVxFe/RxVVZ4AYXs7DiYH/NYcwc84VCyn8bl5fLm
+lI92fXGYN/Qg7Ms/lC2G7D5vFaDFCFQn+zFFUwIDAQABAoIBABNTpiIxbLDhs998
+uvQ3XsumR08kXVcwa/GgvWOSZIEJOUaAYWubDaBTNvTjlhMl6DI+YvKihZMnAkp9
+fXefF1nFUWJJvI0ryi8w6RD54RtbCG4c4raRqysVU7wumZsSenAdc0o09UQE6zXc
+uth/+1VSKCzVjRkLwquXDg0rD3vHfQHWQvQlzwUh3OACA3LfLezVqzrEB02YVRxm
+xwg5veeMg6Aod8vsvsajry9eE0hKeFGonRANerL9lwZxzD2ZjU2fSEJYY3xxKVgi
+o+QVTKaAt9pivOs10YVZDcIDH0xmDpxAkaLb5ZAbnjwhf7WGYgEm8VruAHkJxyXX
+yPf3rpkCgYEA0dp/Xv5KBIkD6JJao8hnhtP5x9U7o/pTzRxaO3WUflvTI6DtC0nk
+cTOwFVs4WljT76T937q2x4stuX0woHzvIaZ6wwZ2vv81ikDY6pE8eLWsH/xFAmkx
+HBfkSijFgJV6EpTqUnFD7QKU89tzWrh/kxaMO1WgFaBhxPPs3K1LDTUCgYEAzVW5
+3yjfVHNgjWTeAbnbToGvUihOidvIvS5cVo5q0Dhfabz0tiXFxAoQUGErUGPC8Nu2
+n/HxTI3b0PbCCwjBsDNqX2kzxTSe5aTGIrBUWbped1bxme8jggXuWYbg8vvLpsYf
+wAJPxnGIxW/v/aftHUhbTIuVfZX2+UnilrwiwWcCgYEAg8paz4eXaH277KVtMwq6
+qZwac/tgNz0Qv/GcYVcYaLq2QNvhwoMnakhxvxfIrkS25PuTTJxwCaVIlAMhNMkB
+TPrGghBfJtgUAb1z/Ow1NAG0FWpS1I7HfsMqZcBxOK2nOmA3QItNg11pujQJn+Ha
+jL9OVj0SCkLs48nk6ToTtjkCgYEAh8YCtNwq6IWuN3CWGCAUMpIwIqxCWof48Zch
+OZ7MZEiSVrG6QmMxpRJefTfzUyHUOj2eQZ7SxqMa0c8IuhEdOeyVjudaczD7TLAq
+z68252oDovfbo8Tr/sL7OzmjryfuHqXtQqKEq5xRKvR8hYavlGhO7otx2uv5thcz
+/CYE+UsCgYAsgewfzbcBqJwsAUs98DK99EU8VqKuaYlU5wNvAVb27O6sVeysSokM
+G1TGIXJPphA3dSAe4Pf/j4ff/eGaS20FAFhs4BPpw0fAeCHpmD0BjIba0lxBS/gY
+dc+JVPKL8Fe4a8fmsI6ndcZQ9qpOdZM5WOD0ldKRc+SsrYKkTmOOJQ==
+-----END RSA PRIVATE KEY-----
+  )";
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), test_private_key, file_path));
+  return Status::OK();
+}
+
+Status GenerateSelfSignedCAForTests(PrivateKey* ca_key, Cert* ca_cert);
+
+// Describes the options for configuring a TlsContext.
+enum class PkiConfig {
+  // The TLS context has no TLS cert and no trusted certs.
+  NONE,
+  // The TLS context has a self-signed TLS cert and no trusted certs.
+  SELF_SIGNED,
+  // The TLS context has no TLS cert and a trusted cert.
+  TRUSTED,
+  // The TLS context has a signed TLS cert and trusts the corresponding signing cert.
+  SIGNED,
+  // The TLS context has a externally signed TLS cert and trusts the corresponding signing cert.
+  EXTERNALLY_SIGNED,
+};
+
+// PkiConfig pretty-printer.
+std::ostream& operator<<(std::ostream& o, PkiConfig c);
+
+Status ConfigureTlsContext(PkiConfig config,
+                           const Cert& ca_cert,
+                           const PrivateKey& ca_key,
+                           TlsContext* tls_context);
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/simple_acl.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/simple_acl.cc b/be/src/kudu/security/simple_acl.cc
new file mode 100644
index 0000000..75b06a2
--- /dev/null
+++ b/be/src/kudu/security/simple_acl.cc
@@ -0,0 +1,87 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/security/simple_acl.h"
+
+#include <ctype.h>
+
+#include <vector>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/util/status.h"
+
+using std::string;
+using std::vector;
+
+namespace kudu {
+namespace security {
+
+SimpleAcl::SimpleAcl() {
+}
+
+SimpleAcl::~SimpleAcl() {
+}
+
+Status SimpleAcl::ParseFlag(const string& flag) {
+  vector<StringPiece> fields = strings::Split(flag, ",", strings::SkipWhitespace());
+  set<string> users;
+  for (const auto& field : fields) {
+    if (field.empty()) continue;
+    // if any field is a wildcard, no need to include the rest.
+    if (flag == "*") {
+      Reset({"*"});
+      return Status::OK();
+    }
+
+
+    // Leave open the use of various special characters at the start of each
+    // username. We reserve some special characters that might be useful in
+    // ACLs:
+    // '!': might be interpreted as "not"
+    // '@': often used to read data from a file
+    // '#': comments
+    // '$': maybe variable expansion?
+    // '%': used by sudoers for groups
+    // '*': only allowed for special wildcard ACL above
+    // '-', '+', '=': useful for allow/deny style ACLs
+    // <quote characters>: in case we want to add quoted strings
+    // whitespace: down right confusing
+    static const char* kReservedStartingCharacters = "!@#$%*-=+'\"";
+    if (strchr(kReservedStartingCharacters, field[0]) ||
+        isspace(field[0])) {
+      return Status::NotSupported("invalid username", field.ToString());
+    }
+
+    users.insert(field.ToString());
+  }
+
+  Reset(std::move(users));
+  return Status::OK();
+}
+
+void SimpleAcl::Reset(set<string> users) {
+  users_ = std::move(users);
+}
+
+bool SimpleAcl::UserAllowed(const string& username) {
+  return ContainsKey(users_, "*") || ContainsKey(users_, username);
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/simple_acl.h
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/simple_acl.h b/be/src/kudu/security/simple_acl.h
new file mode 100644
index 0000000..96ca598
--- /dev/null
+++ b/be/src/kudu/security/simple_acl.h
@@ -0,0 +1,60 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+#pragma once
+
+#include "kudu/gutil/macros.h"
+
+#include <set>
+#include <string>
+
+namespace kudu {
+class Status;
+
+namespace security {
+
+// Represent a very simple access control list which contains a set of users.
+//
+// This is basically just a wrapper around a set<string> with a bit of parsing logic and
+// support for the '*' wildcard.
+class SimpleAcl {
+ public:
+  SimpleAcl();
+  ~SimpleAcl();
+
+  // Parse a flag value, which should be of the form 'user1,user2,user3' to indicate a
+  // list of users, or '*' to indicate a wildcard. This syntax may be expanded later to
+  // include groups, "allow/deny" style access, etc.
+  //
+  // Thread-unsafe: must be called before the ACL may be consulted, and may not be
+  // called a second time concurrent with reads of the ACL.
+  Status ParseFlag(const std::string& flag);
+
+  // Return true if the given user is allowed by the ACL.
+  //
+  // Thread-safe after initialization.
+  bool UserAllowed(const std::string& username);
+
+  // Reset the ACL to the specific set of usernames.
+  void Reset(std::set<std::string> users);
+
+ private:
+  // The set of users, or a set with the single value '*' for the wildcard.
+  std::set<std::string> users_;
+};
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/test/mini_kdc-test.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/test/mini_kdc-test.cc b/be/src/kudu/security/test/mini_kdc-test.cc
new file mode 100644
index 0000000..01ed84d
--- /dev/null
+++ b/be/src/kudu/security/test/mini_kdc-test.cc
@@ -0,0 +1,149 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <string>
+
+#include <boost/optional.hpp>
+#include <gflags/gflags.h>
+#include <gtest/gtest.h>
+
+#include "kudu/security/init.h"
+#include "kudu/security/test/mini_kdc.h"
+#include "kudu/util/env.h"
+#include "kudu/util/test_util.h"
+
+using std::string;
+
+DECLARE_string(keytab_file);
+DECLARE_string(principal);
+
+namespace kudu {
+
+class MiniKdcTest : public KuduTest {};
+
+TEST_F(MiniKdcTest, TestBasicOperation) {
+  MiniKdcOptions options;
+  MiniKdc kdc(options);
+  ASSERT_OK(kdc.Start());
+  ASSERT_GT(kdc.port(), 0);
+  ASSERT_OK(kdc.CreateUserPrincipal("alice"));
+  ASSERT_OK(kdc.Kinit("alice"));
+
+  ASSERT_OK(kdc.Stop());
+  ASSERT_OK(kdc.Start());
+
+  // Check that alice is kinit'd.
+  string klist;
+  ASSERT_OK(kdc.Klist(&klist));
+  ASSERT_STR_CONTAINS(klist, "alice@KRBTEST.COM");
+
+  ASSERT_OK(kdc.CreateUserPrincipal("bob"));
+  ASSERT_OK(kdc.Kinit("bob"));
+
+  // Check that bob has replaced alice as the kinit'd principal.
+  ASSERT_OK(kdc.Klist(&klist));
+  ASSERT_STR_NOT_CONTAINS(klist, "alice@KRBTEST.COM");
+  ASSERT_STR_CONTAINS(klist, "bob@KRBTEST.COM");
+  ASSERT_STR_CONTAINS(klist, "krbtgt/KRBTEST.COM@KRBTEST.COM");
+
+  // Drop 'bob' credentials. We'll get a RuntimeError because klist
+  // exits with a non-zero exit code if there are no cached credentials.
+  ASSERT_OK(kdc.Kdestroy());
+  ASSERT_TRUE(kdc.Klist(&klist).IsRuntimeError());
+
+  // Test keytab creation.
+  const string kSPN = "kudu/foo.example.com";
+  string kt_path;
+  ASSERT_OK(kdc.CreateServiceKeytab(kSPN, &kt_path));
+  SCOPED_TRACE(kt_path);
+  ASSERT_OK(kdc.KlistKeytab(kt_path, &klist));
+  ASSERT_STR_CONTAINS(klist, "kudu/foo.example.com@KRBTEST.COM");
+
+  // Test programmatic keytab login.
+  kdc.SetKrb5Environment();
+  FLAGS_keytab_file = kt_path;
+  FLAGS_principal = kSPN;
+  ASSERT_OK(security::InitKerberosForServer());
+  ASSERT_EQ("kudu/foo.example.com@KRBTEST.COM", *security::GetLoggedInPrincipalFromKeytab());
+
+  // Test principal canonicalization.
+  string princ = "foo";
+  ASSERT_OK(security::CanonicalizeKrb5Principal(&princ));
+  ASSERT_EQ("foo@KRBTEST.COM", princ);
+
+  // Test auth-to-local mapping for a user from the local realm as well as a remote realm.
+  {
+    string local_user;
+    ASSERT_OK(security::MapPrincipalToLocalName("foo@KRBTEST.COM", &local_user));
+    ASSERT_EQ("foo", local_user);
+
+    ASSERT_OK(security::MapPrincipalToLocalName("foo/host@KRBTEST.COM", &local_user));
+    ASSERT_EQ("foo", local_user);
+
+    // The Heimdal implementation in macOS does not correctly implement auth to
+    // local mapping (see init.cc).
+#ifndef __APPLE__
+    ASSERT_OK(security::MapPrincipalToLocalName("foo@OTHERREALM.COM", &local_user));
+    ASSERT_EQ("other-foo", local_user);
+#endif
+  }
+}
+
+// Regression test to ensure that dropping a stopped MiniKdc doesn't panic.
+TEST_F(MiniKdcTest, TestStopDrop) {
+  MiniKdcOptions options;
+  MiniKdc kdc(options);
+}
+
+TEST_F(MiniKdcTest, TestOperationsWhenKdcNotRunning) {
+  MiniKdcOptions options;
+  MiniKdc kdc(options);
+  ASSERT_OK(kdc.Start());
+  ASSERT_OK(kdc.Stop());
+
+  // MiniKdc::CreateUserPrincipal() works directly with the local files,
+  // so it should work fine even if KDC is shut down.
+  ASSERT_OK(kdc.CreateUserPrincipal("alice"));
+
+  {
+    // Without running KDC it should not be possible to obtain and cache an
+    // initial ticket-granting ticket for principal.
+    const Status s = kdc.Kinit("alice");
+    ASSERT_TRUE(s.IsRuntimeError()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "process exited with non-zero status");
+  }
+  {
+    // Without running KDC klist should fail.
+    string klist;
+    const Status s = kdc.Klist(&klist);
+    ASSERT_TRUE(s.IsRuntimeError()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "process exited with non-zero status");
+  }
+
+  ASSERT_OK(kdc.Start());
+
+  // Once KDC has started, 'kinit' and 'klist' should work with no issues.
+  ASSERT_OK(kdc.Kinit("alice"));
+  {
+    // Check that alice is kinit'd.
+    string klist;
+    ASSERT_OK(kdc.Klist(&klist));
+    ASSERT_STR_CONTAINS(klist, "alice@KRBTEST.COM");
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/84b8155c/be/src/kudu/security/test/mini_kdc.cc
----------------------------------------------------------------------
diff --git a/be/src/kudu/security/test/mini_kdc.cc b/be/src/kudu/security/test/mini_kdc.cc
new file mode 100644
index 0000000..4b84bf2
--- /dev/null
+++ b/be/src/kudu/security/test/mini_kdc.cc
@@ -0,0 +1,390 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/security/test/mini_kdc.h"
+
+#include <csignal>
+#include <stdlib.h>
+
+#include <limits>
+#include <map>
+#include <memory>
+#include <string>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/env.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/subprocess.h"
+#include "kudu/util/test_util.h"
+
+using std::map;
+using std::string;
+using std::unique_ptr;
+using strings::Substitute;
+
+namespace kudu {
+
+string MiniKdcOptions::ToString() const {
+  return strings::Substitute("{ realm: $0, port: $1, data_root: $2 }", realm, port, data_root);
+}
+
+MiniKdc::MiniKdc()
+    : MiniKdc(MiniKdcOptions()) {
+}
+
+MiniKdc::MiniKdc(const MiniKdcOptions& options)
+    : options_(options) {
+  if (options_.realm.empty()) {
+    options_.realm = "KRBTEST.COM";
+  }
+  if (options_.data_root.empty()) {
+    options_.data_root = JoinPathSegments(GetTestDataDirectory(), "krb5kdc");
+  }
+  if (options_.renew_lifetime.empty()) {
+    options_.renew_lifetime = "7d";
+  }
+  if (options_.ticket_lifetime.empty()) {
+    options_.ticket_lifetime = "24h";
+  }
+}
+
+MiniKdc::~MiniKdc() {
+  if (kdc_process_) {
+    WARN_NOT_OK(Stop(), "Unable to stop MiniKdc");
+  }
+}
+
+map<string, string> MiniKdc::GetEnvVars() const {
+  return {
+    {"KRB5_CONFIG", JoinPathSegments(options_.data_root, "krb5.conf")},
+    {"KRB5_KDC_PROFILE", JoinPathSegments(options_.data_root, "kdc.conf")},
+    {"KRB5CCNAME", JoinPathSegments(options_.data_root, "krb5cc")}
+  };
+}
+
+vector<string> MiniKdc::MakeArgv(const vector<string>& in_argv) {
+  vector<string> real_argv = { "env" };
+  for (const auto& p : GetEnvVars()) {
+    real_argv.push_back(Substitute("$0=$1", p.first, p.second));
+  }
+  for (const string& a : in_argv) {
+    real_argv.push_back(a);
+  }
+  return real_argv;
+}
+
+namespace {
+// Attempts to find the path to the specified Kerberos binary, storing it in 'path'.
+Status GetBinaryPath(const string& binary,
+                     const vector<string>& search,
+                     string* path) {
+  string p;
+
+  // First, check specified locations which are sometimes not on the PATH.
+  // This is necessary to check first so that the system Heimdal kerberos
+  // binaries won't be found first on OS X.
+  for (const auto& location : search) {
+    p = JoinPathSegments(location, binary);
+    if (Env::Default()->FileExists(p)) {
+      *path = p;
+      return Status::OK();
+    }
+  }
+
+  // Next check if the binary is on the PATH.
+  Status s = Subprocess::Call({ "which", binary }, "", &p);
+  if (s.ok()) {
+    StripTrailingNewline(&p);
+    *path = p;
+    return Status::OK();
+  }
+
+  return Status::NotFound("Unable to find binary", binary);
+}
+
+// Attempts to find the path to the specified Kerberos binary, storing it in 'path'.
+Status GetBinaryPath(const string& binary, string* path) {
+  static const vector<string> kCommonLocations = {
+    "/usr/local/opt/krb5/sbin", // Homebrew
+    "/usr/local/opt/krb5/bin", // Homebrew
+    "/opt/local/sbin", // Macports
+    "/opt/local/bin", // Macports
+    "/usr/lib/mit/sbin", // SLES
+    "/usr/sbin", // Linux
+  };
+  return GetBinaryPath(binary, kCommonLocations, path);
+}
+} // namespace
+
+
+Status MiniKdc::Start() {
+  SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, "starting KDC");
+  CHECK(!kdc_process_);
+  VLOG(1) << "Starting Kerberos KDC: " << options_.ToString();
+
+  if (!Env::Default()->FileExists(options_.data_root)) {
+    VLOG(1) << "Creating KDC database and configuration files";
+    RETURN_NOT_OK(Env::Default()->CreateDir(options_.data_root));
+
+    RETURN_NOT_OK(CreateKdcConf());
+    RETURN_NOT_OK(CreateKrb5Conf());
+
+    // Create the KDC database using the kdb5_util tool.
+    string kdb5_util_bin;
+    RETURN_NOT_OK(GetBinaryPath("kdb5_util", &kdb5_util_bin));
+
+    RETURN_NOT_OK(Subprocess::Call(MakeArgv({
+        kdb5_util_bin, "create",
+        "-s", // Stash the master password.
+        "-P", "masterpw", // Set a password.
+        "-W", // Use weak entropy (since we don't need real security).
+    })));
+  }
+
+  // Start the Kerberos KDC.
+  string krb5kdc_bin;
+  RETURN_NOT_OK(GetBinaryPath("krb5kdc", &krb5kdc_bin));
+
+  kdc_process_.reset(new Subprocess(
+      MakeArgv({
+      krb5kdc_bin,
+      "-n", // Do not daemonize.
+  })));
+
+  RETURN_NOT_OK(kdc_process_->Start());
+
+  const bool need_config_update = (options_.port == 0);
+  // Wait for KDC to start listening on its ports and commencing operation.
+  RETURN_NOT_OK(WaitForKdcPorts());
+
+  if (need_config_update) {
+    // If we asked for an ephemeral port, grab the actual ports and
+    // rewrite the configuration so that clients can connect.
+    RETURN_NOT_OK(CreateKrb5Conf());
+    RETURN_NOT_OK(CreateKdcConf());
+  }
+
+  return Status::OK();
+}
+
+Status MiniKdc::Stop() {
+  CHECK(kdc_process_);
+  VLOG(1) << "Stopping KDC";
+  unique_ptr<Subprocess> proc(kdc_process_.release());
+  RETURN_NOT_OK(proc->Kill(SIGKILL));
+  RETURN_NOT_OK(proc->Wait());
+
+  return Status::OK();
+}
+
+// Creates a kdc.conf file according to the provided options.
+Status MiniKdc::CreateKdcConf() const {
+  static const string kFileTemplate = R"(
+[kdcdefaults]
+kdc_ports = $2
+kdc_tcp_ports = ""
+
+[realms]
+$1 = {
+        acl_file = $0/kadm5.acl
+        admin_keytab = $0/kadm5.keytab
+        database_name = $0/principal
+        key_stash_file = $0/.k5.$1
+        max_renewable_life = 7d 0h 0m 0s
+}
+  )";
+  string file_contents = strings::Substitute(kFileTemplate, options_.data_root,
+                                             options_.realm, options_.port);
+  return WriteStringToFile(Env::Default(), file_contents,
+                           JoinPathSegments(options_.data_root, "kdc.conf"));
+}
+
+// Creates a krb5.conf file according to the provided options.
+Status MiniKdc::CreateKrb5Conf() const {
+  static const string kFileTemplate = R"(
+[logging]
+    kdc = FILE:/dev/stderr
+
+[libdefaults]
+    default_realm = $1
+    dns_lookup_kdc = false
+    dns_lookup_realm = false
+    forwardable = true
+    renew_lifetime = $2
+    ticket_lifetime = $3
+
+    # In miniclusters, we start daemons on local loopback IPs that
+    # have no reverse DNS entries. So, disable reverse DNS.
+    rdns = false
+
+    # The server side will start its GSSAPI server using the local FQDN.
+    # However, in tests, we connect to it via a non-matching loopback IP.
+    # This enables us to connect despite that mismatch.
+    ignore_acceptor_hostname = true
+
+[realms]
+    $1 = {
+        kdc = 127.0.0.1:$0
+        # This super-arcane syntax can be found documented in various Hadoop
+        # vendors' security guides and very briefly in the MIT krb5 docs.
+        # Basically, this one says to map anyone coming in as foo@OTHERREALM.COM
+        # and map them to a local user 'other-foo'
+        auth_to_local = RULE:[1:other-$$1@$$0](.*@OTHERREALM.COM$$)s/@.*//
+    }
+  )";
+  string file_contents = strings::Substitute(kFileTemplate, options_.port, options_.realm,
+                                             options_.renew_lifetime, options_.ticket_lifetime);
+  return WriteStringToFile(Env::Default(), file_contents,
+                           JoinPathSegments(options_.data_root, "krb5.conf"));
+}
+
+Status MiniKdc::WaitForKdcPorts() {
+  SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, Substitute("waiting for KDC ports"));
+  // We have to use 'lsof' to figure out which ports the KDC bound to if we
+  // requested ephemeral ones. The KDC doesn't log the bound port or expose it
+  // in any other fashion, and re-implementing lsof involves parsing a lot of
+  // files in /proc/. So, requiring lsof for tests and parsing its output seems
+  // more straight-forward. We call lsof in a loop in case the kdc is slow to
+  // bind to the ports.
+
+  string lsof;
+  RETURN_NOT_OK(GetBinaryPath("lsof", {"/sbin", "/usr/sbin"}, &lsof));
+
+  const vector<string> cmd = {
+    lsof, "-wbnP", "-Ffn",
+    "-p", std::to_string(kdc_process_->pid()),
+    "-a", "-i", "4UDP"};
+
+  string lsof_out;
+  for (int i = 1; ; i++) {
+    lsof_out.clear();
+    Status s = Subprocess::Call(cmd, "", &lsof_out);
+
+    if (s.ok()) {
+      StripTrailingNewline(&lsof_out);
+      break;
+    } else if (i > 10) {
+      return s;
+    }
+
+    SleepFor(MonoDelta::FromMilliseconds(i * i));
+  }
+
+  // The '-Ffn' flag gets lsof to output something like:
+  //   p19730
+  //   f123
+  //   n*:41254
+  // The first line is the pid. We ignore it.
+  // The second line is the file descriptor number. We ignore it.
+  // The third line has the bind address and port.
+  vector<string> lines = strings::Split(lsof_out, "\n");
+  int32_t port = -1;
+  if (lines.size() != 3 ||
+      lines[2].substr(0, 3) != "n*:" ||
+      !safe_strto32(lines[2].substr(3), &port) ||
+      port <= 0) {
+    return Status::RuntimeError("unexpected lsof output", lsof_out);
+  }
+  CHECK(port > 0 && port < std::numeric_limits<uint16_t>::max())
+      << "parsed invalid port: " << port;
+  VLOG(1) << "Determined bound KDC port: " << port;
+  if (options_.port == 0) {
+    options_.port = port;
+  } else {
+    // Sanity check: if KDC's port is already established, it's supposed to be
+    // written into the configuration files, so the process must bind to the
+    // already established port.
+    CHECK(options_.port == port);
+  }
+  return Status::OK();
+}
+
+Status MiniKdc::CreateUserPrincipal(const string& username) {
+  SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, Substitute("creating user principal $0", username));
+  string kadmin;
+  RETURN_NOT_OK(GetBinaryPath("kadmin.local", &kadmin));
+  RETURN_NOT_OK(Subprocess::Call(MakeArgv({
+          kadmin, "-q", Substitute("add_principal -pw $0 $0", username)})));
+  return Status::OK();
+}
+
+Status MiniKdc::CreateServiceKeytab(const string& spn,
+                                    string* path) {
+  SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, Substitute("creating service keytab for $0", spn));
+  string kt_path = spn;
+  StripString(&kt_path, "/", '_');
+  kt_path = JoinPathSegments(options_.data_root, kt_path) + ".keytab";
+
+  string kadmin;
+  RETURN_NOT_OK(GetBinaryPath("kadmin.local", &kadmin));
+  RETURN_NOT_OK(Subprocess::Call(MakeArgv({
+          kadmin, "-q", Substitute("add_principal -randkey $0", spn)})));
+  RETURN_NOT_OK(Subprocess::Call(MakeArgv({
+          kadmin, "-q", Substitute("ktadd -k $0 $1", kt_path, spn)})));
+  *path = kt_path;
+  return Status::OK();
+}
+
+Status MiniKdc::Kinit(const string& username) {
+  SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, Substitute("kinit for $0", username));
+  string kinit;
+  RETURN_NOT_OK(GetBinaryPath("kinit", &kinit));
+  RETURN_NOT_OK(Subprocess::Call(MakeArgv({ kinit, username }), username));
+  return Status::OK();
+}
+
+Status MiniKdc::Kdestroy() {
+  SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, "kdestroy");
+  string kdestroy;
+  RETURN_NOT_OK(GetBinaryPath("kdestroy", &kdestroy));
+  return Subprocess::Call(MakeArgv({ kdestroy, "-A" }));
+}
+
+Status MiniKdc::Klist(string* output) {
+  string klist;
+  RETURN_NOT_OK(GetBinaryPath("klist", &klist));
+  RETURN_NOT_OK(Subprocess::Call(MakeArgv({ klist, "-A" }), "", output));
+  return Status::OK();
+}
+
+Status MiniKdc::KlistKeytab(const string& keytab_path, string* output) {
+  string klist;
+  RETURN_NOT_OK(GetBinaryPath("klist", &klist));
+  RETURN_NOT_OK(Subprocess::Call(MakeArgv({ klist, "-k", keytab_path }), "", output));
+  return Status::OK();
+}
+
+Status MiniKdc::SetKrb5Environment() const {
+  if (!kdc_process_) {
+    return Status::IllegalState("KDC not started");
+  }
+  for (const auto& p : GetEnvVars()) {
+    CHECK_ERR(setenv(p.first.c_str(), p.second.c_str(), 1 /*overwrite*/));
+  }
+
+  return Status::OK();
+}
+
+} // namespace kudu