You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by sa...@apache.org on 2018/04/23 17:39:24 UTC

[6/9] impala git commit: IMPALA-5893: Remove old kinit code for Impala 3

IMPALA-5893: Remove old kinit code for Impala 3

We've gone through a couple of releases with Kudu's kinit as the
default way to use kerberos and we've not come across any major issues.

Since we're going to have a major release soon, it's time to get rid of
the old Kinit code that's largely unused for a while now.

Testing: Made sure that our current kerberos tests continue to work
without the old code.

Cherry-picks: not for 2.x

Change-Id: Ic78de10f3fb9ec36537de7a090916e4be123234b
Reviewed-on: http://gerrit.cloudera.org:8080/9941
Reviewed-by: Sailesh Mukil <sa...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: 4dc3d340d866ef9a7ef1f654cbfff402e0bc69cc
Parents: 7134d81
Author: Sailesh Mukil <sa...@cloudera.com>
Authored: Thu Apr 5 18:19:17 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Fri Apr 20 21:40:13 2018 +0000

----------------------------------------------------------------------
 be/src/common/global-flags.cc         |  2 +
 be/src/rpc/auth-provider.h            | 21 +-------
 be/src/rpc/authentication.cc          | 82 ++----------------------------
 be/src/rpc/rpc-mgr-kerberized-test.cc |  6 +--
 be/src/rpc/thrift-server-test.cc      |  5 +-
 be/src/testutil/mini-kdc-wrapper.h    |  5 +-
 6 files changed, 11 insertions(+), 110 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/4dc3d340/be/src/common/global-flags.cc
----------------------------------------------------------------------
diff --git a/be/src/common/global-flags.cc b/be/src/common/global-flags.cc
index a15e91b..ea88b28 100644
--- a/be/src/common/global-flags.cc
+++ b/be/src/common/global-flags.cc
@@ -226,6 +226,7 @@ REMOVED_FLAG(enable_partitioned_aggregation);
 REMOVED_FLAG(enable_partitioned_hash_join);
 REMOVED_FLAG(enable_phj_probe_side_filtering);
 REMOVED_FLAG(enable_rm);
+REMOVED_FLAG(kerberos_reinit_interval);
 REMOVED_FLAG(llama_addresses);
 REMOVED_FLAG(llama_callback_port);
 REMOVED_FLAG(llama_host);
@@ -246,4 +247,5 @@ REMOVED_FLAG(rpc_cnxn_retry_interval_ms);
 REMOVED_FLAG(staging_cgroup);
 REMOVED_FLAG(suppress_unknown_disk_id_warnings);
 REMOVED_FLAG(use_statestore);
+REMOVED_FLAG(use_kudu_kinit);
 REMOVED_FLAG(disable_admission_control);

http://git-wip-us.apache.org/repos/asf/impala/blob/4dc3d340/be/src/rpc/auth-provider.h
----------------------------------------------------------------------
diff --git a/be/src/rpc/auth-provider.h b/be/src/rpc/auth-provider.h
index 3e5517f..f3180d5 100644
--- a/be/src/rpc/auth-provider.h
+++ b/be/src/rpc/auth-provider.h
@@ -24,14 +24,11 @@
 
 #include "common/status.h"
 #include "util/promise.h"
-#include "util/thread.h"
 
 namespace sasl { class TSasl; }
 
 namespace impala {
 
-class Thread;
-
 /// An AuthProvider creates Thrift transports that are set up to authenticate themselves
 /// using a protocol such as Kerberos or PLAIN/SASL. Both server and client transports are
 /// provided by this class, using slightly different mechanisms (servers use a factory,
@@ -70,9 +67,8 @@ class SaslAuthProvider : public AuthProvider {
   SaslAuthProvider(bool is_internal) : has_ldap_(false), is_internal_(is_internal),
       needs_kinit_(false) {}
 
-  /// Performs initialization of external state.  If we're using kerberos and
-  /// need to kinit, start that thread.  If we're using ldap, set up appropriate
-  /// certificate usage.
+  /// Performs initialization of external state. Kinit if configured to use kerberos.
+  /// If we're using ldap, set up appropriate certificate usage.
   virtual Status Start();
 
   /// Wrap the client transport with a new TSaslClientTransport.  This is only for
@@ -143,19 +139,6 @@ class SaslAuthProvider : public AuthProvider {
   /// function as a client.
   bool needs_kinit_;
 
-  /// Runs "RunKinit" below if needs_kinit_ is true and FLAGS_use_kudu_kinit is false
-  /// and FLAGS_use_krpc is false. Once started, this thread lives as long as the process
-  /// does and periodically forks impalad and execs the 'kinit' process.
-  std::unique_ptr<Thread> kinit_thread_;
-
-  /// Periodically (roughly once every FLAGS_kerberos_reinit_interval minutes) calls kinit
-  /// to get a ticket granting ticket from the kerberos server for principal_, which is
-  /// kept in the kerberos cache associated with this process. This ensures that we have
-  /// valid kerberos credentials when operating as a client. Once the first attempt to
-  /// obtain a ticket has completed, first_kinit is Set() with the status of the operation.
-  /// Additionally, if the first attempt fails, this method will return.
-  void RunKinit(Promise<Status>* first_kinit);
-
   /// One-time kerberos-specific environment variable setup.  Called by InitKerberos().
   Status InitKerberosEnv() WARN_UNUSED_RESULT;
 };

http://git-wip-us.apache.org/repos/asf/impala/blob/4dc3d340/be/src/rpc/authentication.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/authentication.cc b/be/src/rpc/authentication.cc
index 4c3df50..3eb77bb 100644
--- a/be/src/rpc/authentication.cc
+++ b/be/src/rpc/authentication.cc
@@ -20,7 +20,6 @@
 #include <stdio.h>
 #include <signal.h>
 #include <boost/algorithm/string.hpp>
-#include <boost/thread/thread.hpp>
 #include <boost/scoped_ptr.hpp>
 #include <boost/random/mersenne_twister.hpp>
 #include <boost/random/uniform_int.hpp>
@@ -48,7 +47,6 @@
 #include "util/network-util.h"
 #include "util/os-util.h"
 #include "util/promise.h"
-#include "util/thread.h"
 #include "util/time.h"
 
 #include <sys/types.h>    // for stat system call
@@ -74,11 +72,6 @@ DECLARE_string(be_principal);
 DECLARE_string(krb5_conf);
 DECLARE_string(krb5_debug_file);
 
-// TODO: Remove this flag in a compatibility-breaking release. (IMPALA-5893)
-DEFINE_int32(kerberos_reinit_interval, 60,
-    "Interval, in minutes, between kerberos ticket renewals. "
-    "Only used when FLAGS_use_krpc is false");
-
 DEFINE_string(sasl_path, "", "Colon separated list of paths to look for SASL "
     "security library plugins.");
 DEFINE_bool(enable_ldap_auth, false,
@@ -108,13 +101,6 @@ DEFINE_string(internal_principals_whitelist, "hdfs", "(Advanced) Comma-separated
     "'hdfs' which is the system user that in certain deployments must access "
     "catalog server APIs.");
 
-// TODO: Remove this flag and the old kerberos code once we remove 'use_krpc' flag.
-// (IMPALA-5893)
-DEFINE_bool(use_kudu_kinit, true, "If true, Impala will programatically perform kinit "
-    "by calling into the libkrb5 library using the provided APIs. If false, it will fork "
-    "off a kinit process. If use_krpc=true, this flag is treated as true regardless of "
-    "what it's set to.");
-
 namespace impala {
 
 // Sasl callbacks.  Why are these here?  Well, Sasl isn't that bright, and
@@ -496,51 +482,6 @@ static int SaslGetPath(void* context, const char** path) {
   return SASL_OK;
 }
 
-// When operating as a Kerberos client (internal connections only), we need to
-// 'kinit' as the principal.  A thread is created and calls this function for
-// that purpose, and to periodically renew the ticket as well.
-//
-// first_kinit: Used to communicate success/failure of the initial kinit call to
-//              the parent thread
-// Return: Only if the first call to 'kinit' fails
-void SaslAuthProvider::RunKinit(Promise<Status>* first_kinit) {
-
-  // Pass the path to the key file and the principal.
-  const string kinit_cmd = Substitute("kinit -k -t $0 $1 2>&1",
-      keytab_file_, principal_);
-
-  bool first_time = true;
-  std::random_device rd;
-  mt19937 generator(rd());
-  uniform_int<> dist(0, 300);
-
-  while (true) {
-    LOG(INFO) << "Registering " << principal_ << ", keytab file " << keytab_file_;
-    string kinit_output;
-    bool success = RunShellProcess(kinit_cmd, &kinit_output);
-
-    if (!success) {
-      const string& err_msg = Substitute(
-          "Failed to obtain Kerberos ticket for principal: $0. $1", principal_,
-          kinit_output);
-      if (first_time) {
-        first_kinit->Set(Status(err_msg));
-        return;
-      } else {
-        LOG(ERROR) << err_msg;
-      }
-    } else if (first_time) {
-      first_time = false;
-      first_kinit->Set(Status::OK());
-    }
-
-    // Sleep for the renewal interval, minus a random time between 0-5 minutes to help
-    // avoid a storm at the KDC. Additionally, never sleep less than a minute to
-    // reduce KDC stress due to frequent renewals.
-    SleepForMs(1000 * max((60 * FLAGS_kerberos_reinit_interval) - dist(generator), 60));
-  }
-}
-
 namespace {
 
 // SASL requires mutexes for thread safety, but doesn't implement
@@ -842,25 +783,10 @@ Status SaslAuthProvider::Start() {
   if (needs_kinit_) {
     DCHECK(is_internal_);
     DCHECK(!principal_.empty());
-    if (FLAGS_use_kudu_kinit || FLAGS_use_krpc) {
-      // With KRPC enabled, we always rely on the Kudu library to carry out the Kerberos
-      // authentication during connection negotiation.
-      if (!FLAGS_use_kudu_kinit) {
-        LOG(INFO) << "Ignoring --use_kudu_kinit=false as KRPC and Kerberos are enabled";
-      }
-      // Starts a thread that periodically does a 'kinit'. The thread lives as long as the
-      // process does.
-      KUDU_RETURN_IF_ERROR(kudu::security::InitKerberosForServer(principal_, keytab_file_,
-          KRB5CCNAME_PATH, false), "Could not init kerberos");
-    } else {
-      Promise<Status> first_kinit;
-      stringstream thread_name;
-      thread_name << "kinit-" << principal_;
-      RETURN_IF_ERROR(Thread::Create("authentication", thread_name.str(),
-          &SaslAuthProvider::RunKinit, this, &first_kinit, &kinit_thread_));
-      LOG(INFO) << "Waiting for Kerberos ticket for principal: " << principal_;
-      RETURN_IF_ERROR(first_kinit.Get());
-    }
+    // Starts a thread that periodically does a 'kinit'. The thread lives as long as the
+    // process does.
+    KUDU_RETURN_IF_ERROR(kudu::security::InitKerberosForServer(principal_, keytab_file_,
+        KRB5CCNAME_PATH, false), "Could not init kerberos");
     LOG(INFO) << "Kerberos ticket granted to " << principal_;
   }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/4dc3d340/be/src/rpc/rpc-mgr-kerberized-test.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/rpc-mgr-kerberized-test.cc b/be/src/rpc/rpc-mgr-kerberized-test.cc
index 141f359..0407308 100644
--- a/be/src/rpc/rpc-mgr-kerberized-test.cc
+++ b/be/src/rpc/rpc-mgr-kerberized-test.cc
@@ -18,7 +18,6 @@
 #include "rpc/rpc-mgr-test-base.h"
 #include "service/fe-support.h"
 
-DECLARE_bool(use_kudu_kinit);
 DECLARE_bool(use_krpc);
 
 DECLARE_string(be_principal);
@@ -39,9 +38,7 @@ class RpcMgrKerberizedTest :
     public RpcMgrTestBase<testing::TestWithParam<KerberosSwitch> > {
 
   virtual void SetUp() override {
-    KerberosSwitch k = GetParam();
     FLAGS_use_krpc = true;
-    FLAGS_use_kudu_kinit = k == USE_KRPC_KUDU_KERBEROS;
     FLAGS_principal = "dummy-service/host@realm";
     FLAGS_be_principal = strings::Substitute("$0@$1", kdc_principal, kdc_realm);
     ASSERT_OK(InitAuth(CURRENT_EXECUTABLE_PATH));
@@ -56,8 +53,7 @@ class RpcMgrKerberizedTest :
 
 INSTANTIATE_TEST_CASE_P(KerberosOnAndOff,
                         RpcMgrKerberizedTest,
-                        ::testing::Values(USE_KRPC_IMPALA_KERBEROS,
-                                          USE_KRPC_KUDU_KERBEROS));
+                        ::testing::Values(KERBEROS_ON));
 
 TEST_P(RpcMgrKerberizedTest, MultipleServicesTls) {
   // TODO: We're starting a seperate RpcMgr here instead of configuring

http://git-wip-us.apache.org/repos/asf/impala/blob/4dc3d340/be/src/rpc/thrift-server-test.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-server-test.cc b/be/src/rpc/thrift-server-test.cc
index f0a0bc5..03bd295 100644
--- a/be/src/rpc/thrift-server-test.cc
+++ b/be/src/rpc/thrift-server-test.cc
@@ -35,7 +35,6 @@ using namespace strings;
 using namespace apache::thrift;
 using apache::thrift::transport::SSLProtocol;
 
-DECLARE_bool(use_kudu_kinit);
 DECLARE_bool(use_krpc);
 
 DECLARE_string(principal);
@@ -110,7 +109,6 @@ class ThriftKerberizedParamsTest :
       FLAGS_principal.clear();
       FLAGS_be_principal.clear();
     } else {
-      FLAGS_use_kudu_kinit = k == USE_THRIFT_KUDU_KERBEROS;
       FLAGS_principal = "dummy-service/host@realm";
       FLAGS_be_principal = strings::Substitute("$0@$1", kdc_principal, kdc_realm);
     }
@@ -127,8 +125,7 @@ class ThriftKerberizedParamsTest :
 INSTANTIATE_TEST_CASE_P(KerberosOnAndOff,
                         ThriftKerberizedParamsTest,
                         ::testing::Values(KERBEROS_OFF,
-                                          USE_THRIFT_KUDU_KERBEROS,
-                                          USE_THRIFT_IMPALA_KERBEROS));
+                                          KERBEROS_ON));
 
 TEST(ThriftTestBase, Connectivity) {
   int port = GetServerPort();

http://git-wip-us.apache.org/repos/asf/impala/blob/4dc3d340/be/src/testutil/mini-kdc-wrapper.h
----------------------------------------------------------------------
diff --git a/be/src/testutil/mini-kdc-wrapper.h b/be/src/testutil/mini-kdc-wrapper.h
index 17c174a..602c15b 100644
--- a/be/src/testutil/mini-kdc-wrapper.h
+++ b/be/src/testutil/mini-kdc-wrapper.h
@@ -29,10 +29,7 @@ namespace impala {
 
 enum KerberosSwitch {
   KERBEROS_OFF,
-  USE_KRPC_KUDU_KERBEROS,    // FLAGS_use_kudu_kinit = true,  FLAGS_use_krpc = true
-  USE_KRPC_IMPALA_KERBEROS,  // FLAGS_use_kudu_kinit = false, FLAGS_use_krpc = true
-  USE_THRIFT_KUDU_KERBEROS,  // FLAGS_use_kudu_kinit = true,  FLAGS_use_krpc = false
-  USE_THRIFT_IMPALA_KERBEROS // FLAGS_use_kudu_kinit = false, FLAGS_use_krpc = false
+  KERBEROS_ON
 };
 
 /// This class allows tests to easily start and stop a KDC and configure Impala's auth