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

kudu git commit: KUDU-1845: Kerberos client keytab should be periodically renewed

Repository: kudu
Updated Branches:
  refs/heads/master a23341acd -> 8067304f6


KUDU-1845: Kerberos client keytab should be periodically renewed

With the current kerberos support, if a ticket expires, all
communication on a secure cluster would stop as we do not attempt
to renew the ticket.

This patch adds a renewal thread which periodically wakes up and
either renews the Ticket Granting Ticket or acquires a new one if
the renewal window is closed. The renew interval is controlled by
a newly introduced flag called 'kerberos_reinit_interval'.

A reader-writer lock is used to avoid a race at the time of ticket
reinitializtaion, which can occur between the time the credential
cache is reinitialized and the time the new credentials are placed
in the cache. The reader lock is taken before any call to the
SASL library (if kerberos is enabled) and the writer lock is
taken before reinitializing the cache and placing the new creds
in it.

Testing: Added 2 tests, one for ticket renewal and one for ticket
reacquisition. I haven't added tests to confirm if disabling the
renew thread makes these tests fail as that would require quite
some plumbing which I think is unnecessary. Instead, I manually
tested that disabling the renew thread causes the tests to fail.

Change-Id: Ic4c072c1210216369e60eac88be4a20d9b166b2d
Reviewed-on: http://gerrit.cloudera.org:8080/5820
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/8067304f
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/8067304f
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/8067304f

Branch: refs/heads/master
Commit: 8067304f673492742262b57f0fb23c1334a97274
Parents: a23341a
Author: Sailesh Mukil <sa...@apache.org>
Authored: Fri Jan 27 17:21:26 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Thu Feb 16 07:38:00 2017 +0000

----------------------------------------------------------------------
 .../external_mini_cluster-test.cc               |  44 +++
 .../integration-tests/external_mini_cluster.cc  |   3 +-
 .../integration-tests/external_mini_cluster.h   |   6 +-
 src/kudu/rpc/sasl_common.cc                     |   8 +
 src/kudu/security/init.cc                       | 273 ++++++++++++++++---
 src/kudu/security/init.h                        |   8 +
 src/kudu/security/test/mini_kdc.cc              |  13 +-
 src/kudu/security/test/mini_kdc.h               |   6 +
 8 files changed, 318 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/8067304f/src/kudu/integration-tests/external_mini_cluster-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster-test.cc b/src/kudu/integration-tests/external_mini_cluster-test.cc
index 012808a..5714591 100644
--- a/src/kudu/integration-tests/external_mini_cluster-test.cc
+++ b/src/kudu/integration-tests/external_mini_cluster-test.cc
@@ -51,6 +51,50 @@ INSTANTIATE_TEST_CASE_P(KerberosOnAndOff,
                         ExternalMiniClusterTest,
                         ::testing::Values(WITHOUT_KERBEROS, WITH_KERBEROS));
 
+void SmokeTestKerberizedCluster(const ExternalMiniClusterOptions& opts) {
+  ASSERT_TRUE(opts.enable_kerberos);
+
+  ExternalMiniCluster cluster(opts);
+  ASSERT_OK(cluster.Start());
+
+  // Sleep long enough to ensure that the tserver's ticket would have expired
+  // if not for the renewal thread doing its thing.
+  SleepFor(MonoDelta::FromSeconds(10));
+
+  // Re-kinit for the client, since the client's ticket would have expired as well.
+  ASSERT_OK(cluster.kdc()->Kinit("testuser"));
+
+  // Restart the master, and make sure the tserver is still able to reconnect and
+  // authenticate.
+  cluster.master(0)->Shutdown();
+  ASSERT_OK(cluster.master(0)->Restart());
+  // Ensure that all of the tablet servers can register with the masters.
+  ASSERT_OK(cluster.WaitForTabletServerCount(opts.num_tablet_servers, MonoDelta::FromSeconds(30)));
+  cluster.Shutdown();
+}
+
+TEST_F(ExternalMiniClusterTest, TestKerberosRenewal) {
+  ExternalMiniClusterOptions opts;
+  opts.enable_kerberos = true;
+  // Set the kerberos ticket lifetime as 5 seconds to force ticket renewal every 5 seconds.
+  opts.mini_kdc_options.ticket_lifetime = "5s";
+  opts.num_tablet_servers = 1;
+
+  SmokeTestKerberizedCluster(opts);
+}
+
+TEST_F(ExternalMiniClusterTest, TestKerberosReacquire) {
+  ExternalMiniClusterOptions opts;
+  opts.enable_kerberos = true;
+  // Set the kerberos ticket lifetime and the renew lifetime as 5 seconds each, to force the
+  // processes to acquire a new ticket instead of being able to renew the existing one.
+  opts.mini_kdc_options.ticket_lifetime = "5s";
+  opts.mini_kdc_options.renew_lifetime = "5s";
+  opts.num_tablet_servers = 1;
+
+  SmokeTestKerberizedCluster(opts);
+}
+
 TEST_P(ExternalMiniClusterTest, TestBasicOperation) {
   ExternalMiniClusterOptions opts;
   opts.enable_kerberos = GetParam() == WITH_KERBEROS;

http://git-wip-us.apache.org/repos/asf/kudu/blob/8067304f/src/kudu/integration-tests/external_mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster.cc b/src/kudu/integration-tests/external_mini_cluster.cc
index 57f2874..a3a3f1c 100644
--- a/src/kudu/integration-tests/external_mini_cluster.cc
+++ b/src/kudu/integration-tests/external_mini_cluster.cc
@@ -33,7 +33,6 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
 #include "kudu/master/master.proxy.h"
-#include "kudu/security/test/mini_kdc.h"
 #include "kudu/server/server_base.pb.h"
 #include "kudu/server/server_base.proxy.h"
 #include "kudu/tserver/tserver_service.proxy.h"
@@ -147,7 +146,7 @@ Status ExternalMiniCluster::Start() {
   }
 
   if (opts_.enable_kerberos) {
-    kdc_.reset(new MiniKdc());
+    kdc_.reset(new MiniKdc(opts_.mini_kdc_options));
     RETURN_NOT_OK(kdc_->Start());
     RETURN_NOT_OK_PREPEND(kdc_->CreateUserPrincipal("testuser"),
                           "could not create client principal");

http://git-wip-us.apache.org/repos/asf/kudu/blob/8067304f/src/kudu/integration-tests/external_mini_cluster.h
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster.h b/src/kudu/integration-tests/external_mini_cluster.h
index 880fd0a..cc5022b 100644
--- a/src/kudu/integration-tests/external_mini_cluster.h
+++ b/src/kudu/integration-tests/external_mini_cluster.h
@@ -29,6 +29,7 @@
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/integration-tests/mini_cluster_base.h"
+#include "kudu/security/test/mini_kdc.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/status.h"
@@ -41,7 +42,6 @@ class ExternalTabletServer;
 class HostPort;
 class MetricPrototype;
 class MetricEntityPrototype;
-class MiniKdc;
 class NodeInstancePB;
 class Sockaddr;
 class Subprocess;
@@ -129,6 +129,10 @@ struct ExternalMiniClusterOptions {
   // master.
   std::vector<uint16_t> master_rpc_ports;
 
+  // Options to configure the MiniKdc before starting it up.
+  // Only used when 'enable_kerberos' is 'true'.
+  MiniKdcOptions mini_kdc_options;
+
   // If true, set up a KDC as part of this MiniCluster, generate keytabs for
   // the servers, and require Kerberos authentication from clients.
   //

http://git-wip-us.apache.org/repos/asf/kudu/blob/8067304f/src/kudu/rpc/sasl_common.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_common.cc b/src/kudu/rpc/sasl_common.cc
index 6b60839..a0ae7e1 100644
--- a/src/kudu/rpc/sasl_common.cc
+++ b/src/kudu/rpc/sasl_common.cc
@@ -37,9 +37,13 @@
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/mutex.h"
 #include "kudu/util/net/sockaddr.h"
+#include "kudu/util/rw_mutex.h"
+#include "kudu/security/init.h"
 
 using std::set;
 
+DECLARE_bool(server_require_kerberos);
+
 namespace kudu {
 namespace rpc {
 
@@ -312,7 +316,11 @@ Status WrapSaslCall(sasl_conn_t* conn, const std::function<int()>& call) {
   // is set globally rather than on a per-connection basis.
   string err;
   g_auth_failure_capture = &err;
+
+  // Take the 'kerberos_reinit_lock' here to avoid a possible race with ticket renewal.
+  if (FLAGS_server_require_kerberos) kudu::security::KerberosReinitLock()->ReadLock();
   int rc = call();
+  if (FLAGS_server_require_kerberos) kudu::security::KerberosReinitLock()->ReadUnlock();
   g_auth_failure_capture = nullptr;
 
   switch (rc) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/8067304f/src/kudu/security/init.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/init.cc b/src/kudu/security/init.cc
index 5d0ded2..b5e2e41 100644
--- a/src/kudu/security/init.cc
+++ b/src/kudu/security/init.cc
@@ -17,14 +17,18 @@
 
 #include "kudu/security/init.h"
 
+#include <algorithm>
 #include <krb5/krb5.h>
+#include <random>
 #include <string>
 
 #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, "", "Path to the Kerberos Keytab for this server");
 TAG_FLAG(keytab, experimental);
@@ -33,81 +37,269 @@ DEFINE_string(kerberos_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(kerberos_principal, experimental);
+
 // TODO(todd): this currently only affects the keytab login which is used
 // for client credentials, but doesn't affect the SASL server code path.
 // We probably need to plumb the same configuration into the RPC code.
 
+using std::mt19937;
+using std::random_device;
 using std::string;
+using std::uniform_int_distribution;
+using std::uniform_real_distribution;
 
 namespace kudu {
 namespace security {
 
 namespace {
 
+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);
+
+  krb5_context* krb5_ctx() { return &krb5_ctx_; }
+
+ private:
+  krb5_principal principal_;
+  krb5_keytab keytab_;
+  krb5_ccache ccache_;
+  krb5_get_init_creds_opt* opts_;
+  krb5_context krb5_ctx_;
+
+  // This is the time that the current TGT in use expires.
+  int32_t ticket_end_timestamp_;
+};
+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;
+
 Status Krb5CallToStatus(krb5_context ctx, krb5_error_code code) {
   if (code == 0) return Status::OK();
   return Status::RuntimeError(krb5_get_error_message(ctx, code));
 }
 #define KRB5_RETURN_NOT_OK_PREPEND(call, prepend) \
-  RETURN_NOT_OK_PREPEND(Krb5CallToStatus(ctx, (call)), (prepend))
+  RETURN_NOT_OK_PREPEND(Krb5CallToStatus(*g_kinit_ctx->krb5_ctx(), (call)), (prepend))
 
-// 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) {
-  krb5_context ctx;
-  if (krb5_init_context(&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));
+}
+
+// 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(krb5_ctx_, ccache_, &cursor),
+                             "Failed to peek into ccache");
+  auto cleanup_cursor = MakeScopedCleanup([&]() {
+      krb5_cc_end_seq_get(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(krb5_ctx_, ccache_, &cursor, &creds))) {
+    auto cleanup_creds = MakeScopedCleanup([&]() {
+        krb5_free_cred_contents(krb5_ctx_, &creds); });
+    if (krb5_is_config_principal(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(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(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(krb5_ctx_, ccache_, principal_),
+                                   "Reacquire error: could not init ccache");
+
+        KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_store_cred(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(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(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(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) {
+  if (krb5_init_context(&krb5_ctx_) != 0) {
     return Status::RuntimeError("could not initialize krb5 library");
   }
-  auto cleanup_ctx = MakeScopedCleanup([&]() { krb5_free_context(ctx); });
 
   // Parse the principal
-  krb5_principal client_principal;
-  KRB5_RETURN_NOT_OK_PREPEND(krb5_parse_name(ctx, principal.c_str(), &client_principal),
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_parse_name(krb5_ctx_, principal.c_str(), &principal_),
                              "could not parse principal");
-  auto cleanup_client_principal = MakeScopedCleanup([&]() {
-      krb5_free_principal(ctx, client_principal); });
 
-  krb5_keytab keytab;
-  KRB5_RETURN_NOT_OK_PREPEND(krb5_kt_resolve(ctx, keytab_path.c_str(), &keytab),
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_kt_resolve(krb5_ctx_, keytab_path.c_str(), &keytab_),
                              "unable to resolve keytab");
-  auto cleanup_keytab = MakeScopedCleanup([&]() { krb5_kt_close(ctx, keytab); });
 
-  krb5_ccache ccache;
-  KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_default(ctx, &ccache),
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_default(krb5_ctx_, &ccache_),
                              "unable to get default credentials cache");
-  auto cleanup_ccache = MakeScopedCleanup([&]() { krb5_cc_close(ctx, ccache); });
-
 
-  krb5_get_init_creds_opt* opt;
-  KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_opt_alloc(ctx, &opt),
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_opt_alloc(krb5_ctx_, &opts_),
                              "unable to allocate get_init_creds_opt struct");
-  auto cleanup_opt = MakeScopedCleanup([&]() { krb5_get_init_creds_opt_free(ctx, opt); });
 
 #ifndef __APPLE__
-  KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_opt_set_out_ccache(ctx, opt, ccache),
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_opt_set_out_ccache(krb5_ctx_, opts_, ccache_),
                              "unable to set init_creds options");
 #endif
 
   krb5_creds creds;
-  KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_keytab(ctx, &creds, client_principal,
-                                                        keytab, 0 /* valid from now */,
-                                                        nullptr /* TKT service name */,
-                                                        opt),
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_keytab(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(ctx, &creds); });
+  auto cleanup_creds = MakeScopedCleanup([&]() {
+      krb5_free_cred_contents(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(ctx, ccache, client_principal),
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_initialize(krb5_ctx_, ccache_, principal_),
                              "could not init ccache");
 
-  KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_store_cred(ctx, ccache, &creds),
+  KRB5_RETURN_NOT_OK_PREPEND(krb5_cc_store_cred(krb5_ctx_, ccache_, &creds),
                              "could not store creds in cache");
 #endif
   return Status::OK();
@@ -127,6 +319,10 @@ Status GetLoginPrincipal(string* principal) {
 
 } // anonymous namespace
 
+RWMutex* KerberosReinitLock() {
+  return g_kerberos_reinit_lock;
+}
+
 Status InitKerberosForServer() {
   if (FLAGS_keytab.empty()) return Status::OK();
 
@@ -137,12 +333,15 @@ Status InitKerberosForServer() {
   setenv("KRB5CCNAME", "MEMORY:kudu", 1);
   setenv("KRB5_KTNAME", FLAGS_keytab.c_str(), 1);
 
+  g_kinit_ctx = new KinitContext();
   string principal;
   RETURN_NOT_OK(GetLoginPrincipal(&principal));
-  RETURN_NOT_OK_PREPEND(Kinit(FLAGS_keytab, principal), "unable to kinit");
+  RETURN_NOT_OK_PREPEND(g_kinit_ctx->Kinit(FLAGS_keytab, principal), "unable to kinit");
 
-  // TODO(todd) we likely need to start a "renewal thread" here, since the credentials
-  // can expire.
+  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();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/8067304f/src/kudu/security/init.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/init.h b/src/kudu/security/init.h
index 2497fcc..a86e119 100644
--- a/src/kudu/security/init.h
+++ b/src/kudu/security/init.h
@@ -19,11 +19,19 @@
 #include "kudu/util/status.h"
 
 namespace kudu {
+
+class RWMutex;
+
 namespace security {
 
 // Initializes Kerberos for a server. In particular, this processes
 // the '--keytab' 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();
+
 } // namespace security
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/8067304f/src/kudu/security/test/mini_kdc.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/test/mini_kdc.cc b/src/kudu/security/test/mini_kdc.cc
index e365dee..09a2088 100644
--- a/src/kudu/security/test/mini_kdc.cc
+++ b/src/kudu/security/test/mini_kdc.cc
@@ -61,6 +61,12 @@ MiniKdc::MiniKdc(const MiniKdcOptions& options)
   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() {
@@ -219,8 +225,8 @@ Status MiniKdc::CreateKrb5Conf() const {
     dns_lookup_kdc = false
     dns_lookup_realm = false
     forwardable = true
-    renew_lifetime = 7d
-    ticket_lifetime = 24h
+    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.
@@ -236,7 +242,8 @@ Status MiniKdc::CreateKrb5Conf() const {
         kdc = 127.0.0.1:$0
     }
   )";
-  string file_contents = strings::Substitute(kFileTemplate, options_.port, options_.realm);
+  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"));
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/8067304f/src/kudu/security/test/mini_kdc.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/test/mini_kdc.h b/src/kudu/security/test/mini_kdc.h
index bb5dede..f80aac1 100644
--- a/src/kudu/security/test/mini_kdc.h
+++ b/src/kudu/security/test/mini_kdc.h
@@ -46,6 +46,12 @@ struct MiniKdcOptions {
   // Default: 0 (ephemeral port).
   uint16_t port = 0;
 
+  // The default lifetime for initial ticket requests.
+  std::string ticket_lifetime;
+
+  // The default renewable lifetime for initial ticket requests.
+  std::string renew_lifetime;
+
   // Returns a string representation of the options suitable for debug printing.
   std::string ToString() const;
 };