You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2020/03/11 16:40:07 UTC

[kudu] 01/02: KUDU-3050: recover from corrupt kerberos ccache

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

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

commit d74ad32df7e41f6c9b03edb8b7be27706b507c2c
Author: Tim Armstrong <ta...@cloudera.com>
AuthorDate: Thu Feb 6 18:05:35 2020 -0800

    KUDU-3050: recover from corrupt kerberos ccache
    
    This handles two failure modes:
    * krb5_cc_start_seq_get() can fail if the kerberos credential cache gets
      corrupted on disk, e.g. is truncated.
    * the renewal can fail to find a credential in the credential cache,
      either if it is missing or the renewal thread hits an error while
      reading through credentials.
    
    Also add some additional logging and limit the max backoff time
    to make it easier to debug other kinds of renewal errors.
    
    The test triggers a pre-existing memory leak bug in some older
    Kerberos libraries. Added a suppression for leak sanitizer
    to ClientNegotiation::CheckGSSAPI() to suppress it.
    
    Test:
    Add a test that exercises the recovery logic after truncating
    the credential cache. The test failed before this change.
    
    Change-Id: I2d6e06c3ea65708896a6bf0134cc84838b3f1b58
    Reviewed-on: http://gerrit.cloudera.org:8080/15394
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/integration-tests/security-itest.cc |  39 +++++++++
 src/kudu/rpc/client_negotiation.cc           |   7 ++
 src/kudu/security/init.cc                    | 117 ++++++++++++++-------------
 src/kudu/security/kinit_context.h            |  85 +++++++++++++++++++
 src/kudu/security/test/mini_kdc.cc           |  11 ++-
 src/kudu/security/test/mini_kdc.h            |   3 +
 6 files changed, 200 insertions(+), 62 deletions(-)

diff --git a/src/kudu/integration-tests/security-itest.cc b/src/kudu/integration-tests/security-itest.cc
index 1003461..905adee 100644
--- a/src/kudu/integration-tests/security-itest.cc
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -18,6 +18,8 @@
 #include <sys/stat.h>
 
 #include <cstdio>
+#include <cstdlib>
+#include <initializer_list>
 #include <memory>
 #include <ostream>
 #include <string>
@@ -46,6 +48,7 @@
 #include "kudu/mini-cluster/external_mini_cluster.h"
 #include "kudu/rpc/messenger.h"
 #include "kudu/rpc/rpc_controller.h"
+#include "kudu/security/kinit_context.h"
 #include "kudu/security/test/mini_kdc.h"
 #include "kudu/security/token.pb.h"
 #include "kudu/server/server_base.pb.h"
@@ -397,6 +400,42 @@ TEST_F(SecurityITest, TestWorldReadablePrivateKey) {
       credentials_name));
 }
 
+// Test that our Kinit implementation can handle corrupted credential caches.
+TEST_F(SecurityITest, TestCorruptKerberosCC) {
+  ASSERT_OK(StartCluster());
+  string admin_keytab = cluster_->kdc()->GetKeytabPathForPrincipal("test-admin");
+  ASSERT_OK(cluster_->kdc()->CreateKeytabForExistingPrincipal("test-admin"));
+
+  security::KinitContext kinit_ctx;
+  ASSERT_OK(kinit_ctx.Kinit(admin_keytab, "test-admin"));
+
+  // Truncate at different lengths to exercise different failure modes, e.g. failed to
+  // read header, some credentials missing.
+  for (int trunc_len : {10, 75, 500}) {
+    // Truncate the credential cache so that it no longer contains a valid ticket for
+    // "test-admin".
+    const char* cc_path = getenv("KRB5CCNAME");
+    SCOPED_TRACE(Substitute("Truncating ccache at '$0' to $1", cc_path, trunc_len));
+    {
+      RWFileOptions opts;
+      opts.mode = Env::MUST_EXIST;
+      unique_ptr<RWFile> cc_file;
+      ASSERT_OK(env_->NewRWFile(opts, cc_path, &cc_file));
+      ASSERT_OK(cc_file->Truncate(trunc_len));
+    }
+
+    // With corrupt cache, we shouldn't be able to open connection.
+    Status s = TrySetFlagOnTS();
+    EXPECT_FALSE(s.ok());
+    ASSERT_STR_CONTAINS(s.ToString(), "server requires authentication, but client does "
+        "not have Kerberos credentials available");
+
+    // Renewal should fix the corrupted credential cache and allow secure connections.
+    ASSERT_OK(kinit_ctx.DoRenewal());
+    ASSERT_OK(TrySetFlagOnTS());
+  }
+}
+
 Status AssignIPToClient(bool external) {
   // If the test does not require an external IP address
   // assign loopback IP to FLAGS_local_ip_for_outbound_sockets.
diff --git a/src/kudu/rpc/client_negotiation.cc b/src/kudu/rpc/client_negotiation.cc
index b43f55d..4129193 100644
--- a/src/kudu/rpc/client_negotiation.cc
+++ b/src/kudu/rpc/client_negotiation.cc
@@ -47,6 +47,7 @@
 #include "kudu/security/gssapi.h"
 #include "kudu/security/tls_context.h"
 #include "kudu/security/tls_handshake.h"
+#include "kudu/util/debug/leakcheck_disabler.h"
 #include "kudu/util/faststring.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/net/socket.h"
@@ -793,6 +794,12 @@ int ClientNegotiation::SecretCb(sasl_conn_t* conn, int id, sasl_secret_t** psecr
 }
 
 Status ClientNegotiation::CheckGSSAPI() {
+  // Disable leak checking in this function to work around memory leak in libgssapi_krb5
+  // when opening a corrupt credential cache fails:
+  // https://krbdev.mit.edu/rt/Ticket/Display.html?id=8437.
+  // Fixed in MIT Kerberos 1.13.7, 1.14.4 and 1.15.
+  debug::ScopedLeakCheckDisabler disable_leak_checks;
+
   OM_uint32 major, minor;
   gss_cred_id_t cred = GSS_C_NO_CREDENTIAL;
 
diff --git a/src/kudu/security/init.cc b/src/kudu/security/init.cc
index d0b2226..e740fcb 100644
--- a/src/kudu/security/init.cc
+++ b/src/kudu/security/init.cc
@@ -30,6 +30,7 @@
 #include <random>
 #include <string>
 #include <type_traits>
+#include <utility>
 
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
@@ -40,6 +41,7 @@
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
+#include "kudu/security/kinit_context.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/net_util.h"
@@ -82,67 +84,19 @@ using strings::Substitute;
 namespace kudu {
 namespace security {
 
-namespace {
+// Global instance of the context used by the kinit/reacquire thread.
+KinitContext* g_kinit_ctx;
 
-class KinitContext;
+namespace {
 
 // Global context for usage of the Krb5 library.
 krb5_context g_krb5_ctx;
 
-// Global instance of the context used by the kinit/reacquire thread.
-KinitContext* g_kinit_ctx;
-
 // This lock is used to avoid a race while reacquiring the kerberos ticket.
 // The race can occur between the time we reinitialize the cache and the
 // time when we actually store the new credentials 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();
 
@@ -171,11 +125,11 @@ void InitKrb5Ctx() {
       char* unused_realm;
       CHECK_EQ(krb5_get_default_realm(g_krb5_ctx, &unused_realm), 0);
       krb5_free_default_realm(g_krb5_ctx, unused_realm);
+
+      g_kerberos_reinit_lock = new RWMutex(RWMutex::Priority::PREFER_WRITING);
     });
 }
 
-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));
@@ -202,7 +156,13 @@ 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)));
+    int64_t renew_interval_s = g_kinit_ctx->GetNextRenewInterval(failure_retries);
+    if (failure_retries > 0) {
+      // Log in the abnormal case where something failed.
+      LOG(INFO) << Substitute("Renew thread sleeping after $0 failures for $1s",
+          failure_retries, renew_interval_s);
+    }
+    SleepFor(MonoDelta::FromSeconds(renew_interval_s));
 
     Status s = g_kinit_ctx->DoRenewal();
     WARN_NOT_OK(s, "Kerberos reacquire error: ");
@@ -213,6 +173,17 @@ void RenewThread() {
     }
   }
 }
+} // anonymous namespace
+
+KinitContext::KinitContext() {}
+
+KinitContext::~KinitContext() {
+  // Free memory associated with these objects.
+  if (principal_ != nullptr) krb5_free_principal(g_krb5_ctx, principal_);
+  if (keytab_ != nullptr) krb5_kt_close(g_krb5_ctx, keytab_);
+  if (ccache_ != nullptr) krb5_cc_close(g_krb5_ctx, ccache_);
+  if (opts_ != nullptr) krb5_get_init_creds_opt_free(g_krb5_ctx, opts_);
+}
 
 int32_t KinitContext::GetNextRenewInterval(uint32_t num_retries) {
   int32_t time_remaining = ticket_end_timestamp_ - time(nullptr);
@@ -242,7 +213,8 @@ int32_t KinitContext::GetNextRenewInterval(uint32_t num_retries) {
 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);
+  // Don't back off more than 30 minutes.
+  int32_t base_time = std::min(next_interval * (1 << num_retries), 1800);
   random_device rd;
   mt19937 generator(rd());
   uniform_real_distribution<> dist(0.5, 1.5);
@@ -250,11 +222,31 @@ int32_t KinitContext::GetBackedOffRenewInterval(int32_t time_remaining, uint32_t
 }
 
 Status KinitContext::DoRenewal() {
+  bool found_in_cache;
+  RETURN_NOT_OK(DoRenewalInternal(&found_in_cache));
+  if (!found_in_cache) {
+    // Cache appears to be corrupted somehow - redo the kinit and retry.
+    RETURN_NOT_OK_PREPEND(KinitInternal(), "redoing kinit after error");
+    RETURN_NOT_OK(DoRenewalInternal(&found_in_cache));
+    if (!found_in_cache) {
+      return Status::RuntimeError("Could not find TGT in cache after kinit");
+    }
+  }
+  return Status::OK();
+}
 
+Status KinitContext::DoRenewalInternal(bool* found_in_cache) {
+  *found_in_cache = false;
   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");
+  Status status =
+      Krb5CallToStatus(g_krb5_ctx, krb5_cc_start_seq_get(g_krb5_ctx, ccache_, &cursor));
+  if (!status.ok()) {
+    LOG(WARNING) << Substitute("Error while opening credential cache '$0' of type $1: $2",
+        krb5_cc_get_name(g_krb5_ctx, ccache_), krb5_cc_get_type(g_krb5_ctx, ccache_),
+        status.ToString());
+    return Status::OK();
+  }
   SCOPED_CLEANUP({
       krb5_cc_end_seq_get(g_krb5_ctx, ccache_, &cursor); });
 
@@ -276,6 +268,7 @@ Status KinitContext::DoRenewal() {
         data_eq(creds.server->realm, principal_->realm) == 0) {
       continue;
     }
+    *found_in_cache = true;
 
     krb5_creds new_creds;
     memset(&new_creds, 0, sizeof(krb5_creds));
@@ -304,6 +297,11 @@ Status KinitContext::DoRenewal() {
     ticket_end_timestamp_ = new_creds.times.endtime;
     break;
   }
+  if (!*found_in_cache) {
+    LOG(WARNING) << Substitute(
+        "Could not find kerberos principal in credential cache '$0' of type $1",
+        krb5_cc_get_name(g_krb5_ctx, ccache_), krb5_cc_get_type(g_krb5_ctx, ccache_));
+  }
   return Status::OK();
 }
 
@@ -322,7 +320,10 @@ Status KinitContext::Kinit(const string& keytab_path, const string& principal) {
 
   KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_opt_alloc(g_krb5_ctx, &opts_),
                              "unable to allocate get_init_creds_opt struct");
+  return KinitInternal();
+}
 
+Status KinitContext::KinitInternal() {
 #if defined(HAVE_KRB5_GET_INIT_CREDS_OPT_SET_OUT_CCACHE)
   KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_opt_set_out_ccache(g_krb5_ctx, opts_, ccache_),
                              "unable to set init_creds options");
@@ -362,6 +363,7 @@ Status KinitContext::Kinit(const string& keytab_path, const string& principal) {
   return Status::OK();
 }
 
+namespace {
 // 'in_principal' is the user specified principal to use with Kerberos. It may have a token
 // in the string of the form '_HOST', which if present, needs to be replaced with the FQDN of the
 // current host.
@@ -491,7 +493,6 @@ Status InitKerberosForServer(const std::string& raw_principal, const std::string
   RETURN_NOT_OK_PREPEND(g_kinit_ctx->Kinit(
       keytab_file, configured_principal), "unable to kinit");
 
-  g_kerberos_reinit_lock = new RWMutex(RWMutex::Priority::PREFER_WRITING);
   scoped_refptr<Thread> reacquire_thread;
   // Start the reacquire thread.
   RETURN_NOT_OK(Thread::Create("kerberos", "reacquire thread", &RenewThread, &reacquire_thread));
diff --git a/src/kudu/security/kinit_context.h b/src/kudu/security/kinit_context.h
new file mode 100644
index 0000000..a6c2b41
--- /dev/null
+++ b/src/kudu/security/kinit_context.h
@@ -0,0 +1,85 @@
+// 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 <cstdint>
+#include <string>
+
+#include <krb5/krb5.h>
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace security {
+
+class KinitContext {
+ public:
+  KinitContext();
+  ~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 std::string& keytab_path, const std::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.
+  static int32_t GetBackedOffRenewInterval(int32_t time_remaining, uint32_t num_retries);
+
+  const std::string& principal_str() const { return principal_str_; }
+  const std::string& username_str() const { return username_str_; }
+
+ private:
+  Status KinitInternal();
+
+  // Helper for DoRenewal() that tries to do a renewal. On success, returns OK and sets
+  // *found_in_cache = true. If there is an error doing the renewal itself, returns an
+  // error. If the TGT to be renewed was not found in the cache, return OK and set
+  // *found_in_cache = false.
+  Status DoRenewalInternal(bool* found_in_cache);
+
+  krb5_principal principal_ = nullptr;
+  krb5_keytab keytab_ = nullptr;
+  krb5_ccache ccache_ = nullptr;
+  krb5_get_init_creds_opt* opts_ = nullptr;
+
+  // The stringified principal and username that we are logged in as.
+  std::string principal_str_, username_str_;
+
+  // This is the time that the current TGT in use expires.
+  int32_t ticket_end_timestamp_;
+};
+
+} // namespace security
+} // namespace kudu
diff --git a/src/kudu/security/test/mini_kdc.cc b/src/kudu/security/test/mini_kdc.cc
index 1662770..e3d9604 100644
--- a/src/kudu/security/test/mini_kdc.cc
+++ b/src/kudu/security/test/mini_kdc.cc
@@ -287,10 +287,7 @@ Status MiniKdc::RandomizePrincipalKey(const string& spn) {
 
 Status MiniKdc::CreateKeytabForExistingPrincipal(const string& spn) {
   SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, Substitute("creating keytab for $0", spn));
-  string kt_path = spn;
-  StripString(&kt_path, "/", '_');
-  kt_path = JoinPathSegments(options_.data_root, kt_path) + ".keytab";
-
+  string kt_path = GetKeytabPathForPrincipal(spn);
   string kadmin;
   RETURN_NOT_OK(GetBinaryPath("kadmin.local", &kadmin));
   RETURN_NOT_OK(Subprocess::Call(MakeArgv({
@@ -298,6 +295,12 @@ Status MiniKdc::CreateKeytabForExistingPrincipal(const string& spn) {
   return Status::OK();
 }
 
+string MiniKdc::GetKeytabPathForPrincipal(const string& spn) const {
+  string kt_path = spn;
+  StripString(&kt_path, "/", '_');
+  return JoinPathSegments(options_.data_root, kt_path) + ".keytab";
+}
+
 Status MiniKdc::Kinit(const string& username) {
   SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, Substitute("kinit for $0", username));
   string kinit;
diff --git a/src/kudu/security/test/mini_kdc.h b/src/kudu/security/test/mini_kdc.h
index 94990a2..10dc669 100644
--- a/src/kudu/security/test/mini_kdc.h
+++ b/src/kudu/security/test/mini_kdc.h
@@ -100,6 +100,9 @@ class MiniKdc {
   // 'spn' is the desired service principal name (e.g. "kudu/foo.example.com").
   Status CreateKeytabForExistingPrincipal(const std::string& spn);
 
+  // Returns the path where CreateKeytabForExistingPrincipal creates the keytab.
+  std::string GetKeytabPathForPrincipal(const std::string& spn) const;
+
   // Kinit a user to the mini KDC.
   Status Kinit(const std::string& username) WARN_UNUSED_RESULT;