You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2019/07/16 15:26:39 UTC

[impala] 01/06: Support SPNEGO for Impala webserver

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

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

commit 9f0cd9743a9c364d1eb42f29f67494298ed574ae
Author: Todd Lipcon <to...@apache.org>
AuthorDate: Mon Jul 1 16:59:24 2019 -0700

    Support SPNEGO for Impala webserver
    
    This ports over changes from kudu commit
    1f291b77ef0868ac888a850678adc2d7cce65529 which implemented SPNEGO for
    the Kudu webserver.
    
    Unfortunately, thorough testing of this is difficult given that curl
    isn't currently in the toolchain. I was able to manually test this by
    adding a 'sleep(1000)' call into the newly added test case, then setting
    up $KRB5_CONFIG in my shell to point to the temporary KDC's environment,
    and using 'curl -u : --negotiate http://...' to authenticate.
    
    Strangely, using the version of curl on el7 didn't seem to work properly
    (perhaps an el7 curl bug) but using curl on my Ubuntu 18 laptop I was
    able to authenticate with SPNEGO.
    
    Change-Id: Ife2b04310e1571d231bf8ee1bcfd3b7afc2edd8f
    Reviewed-on: http://gerrit.cloudera.org:8080/13774
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/exec/kudu-util.h          |  34 +-------
 be/src/gutil/strings/escaping.cc |   8 +-
 be/src/util/CMakeLists.txt       |   1 +
 be/src/util/kudu-status-util.h   |  59 ++++++++++++++
 be/src/util/webserver-test.cc    |  33 +++++++-
 be/src/util/webserver.cc         | 165 ++++++++++++++++++++++++++++++++++++++-
 be/src/util/webserver.h          |   7 ++
 7 files changed, 266 insertions(+), 41 deletions(-)

diff --git a/be/src/exec/kudu-util.h b/be/src/exec/kudu-util.h
index 37755bd..b6c049f 100644
--- a/be/src/exec/kudu-util.h
+++ b/be/src/exec/kudu-util.h
@@ -21,33 +21,16 @@
 // TODO: Remove when toolchain callbacks.h properly defines ::tm.
 struct tm;
 
-#include <gutil/strings/substitute.h>
 #include <kudu/client/callbacks.h>
 #include <kudu/client/client.h>
 #include <kudu/client/value.h>
 
-#include "common/status.h"
+#include "util/kudu-status-util.h"
 #include "runtime/string-value.h"
 #include "runtime/types.h"
 
 namespace impala {
 
-/// Takes a Kudu status and returns an impala one, if it's not OK.
-/// Evaluates the prepend argument only if the status is not OK.
-#define KUDU_RETURN_IF_ERROR(expr, prepend) \
-  do { \
-    const kudu::Status& _s = (expr); \
-    if (UNLIKELY(!_s.ok())) {                                      \
-      return Status(strings::Substitute("$0: $1", prepend, _s.ToString())); \
-    } \
-  } while (0)
-
-#define KUDU_ASSERT_OK(status)                                     \
-  do {                                                             \
-    const Status& _s = FromKuduStatus(status);                     \
-    ASSERT_TRUE(_s.ok()) << "Error: " << _s.GetDetail();           \
-  } while (0)
-
 class TimestampValue;
 
 /// Returns false when running on an operating system that Kudu doesn't support. If this
@@ -94,21 +77,6 @@ Status CreateKuduValue(const ColumnType& col_type, void* value,
 ColumnType KuduDataTypeToColumnType(kudu::client::KuduColumnSchema::DataType type,
     const kudu::client::KuduColumnTypeAttributes& type_attributes);
 
-/// Utility function for creating an Impala Status object based on a kudu::Status object.
-/// 'k_status' is the kudu::Status object.
-/// 'prepend' is a string to be prepended to details of 'k_status' when creating the
-/// Impala Status object.
-/// Note that we don't translate the kudu::Status error code to Impala error code
-/// so the returned status' type is always of TErrorCode::GENERAL.
-inline Status FromKuduStatus(
-    const kudu::Status& k_status, const std::string prepend = "") {
-  if (LIKELY(k_status.ok())) return Status::OK();
-  const std::string& err_msg = prepend.empty() ? k_status.ToString() :
-      strings::Substitute("$0: $1", prepend, k_status.ToString());
-  VLOG(1) << err_msg;
-  return Status::Expected(err_msg);
-}
-
 /// Converts 'mode' to its equivalent ReadMode, stored in 'out'. Possible values for
 /// 'mode' are 'READ_LATEST' and 'READ_AT_SNAPSHOT'. If 'mode' is invalid, an error is
 /// returned.
diff --git a/be/src/gutil/strings/escaping.cc b/be/src/gutil/strings/escaping.cc
index adb8093..aaae9bf 100644
--- a/be/src/gutil/strings/escaping.cc
+++ b/be/src/gutil/strings/escaping.cc
@@ -895,10 +895,10 @@ int Base64UnescapeInternal(const char *src, int szsrc,
       // szsrc claims the string is).
 
       if (!src[0] || !src[1] || !src[2] ||
-          (temp = ((unbase64[src[0]] << 18) |
-                   (unbase64[src[1]] << 12) |
-                   (unbase64[src[2]] << 6) |
-                   (unbase64[src[3]]))) & 0x80000000) {
+          (temp = ((unsigned(unbase64[src[0]]) << 18) |
+                   (unsigned(unbase64[src[1]]) << 12) |
+                   (unsigned(unbase64[src[2]]) << 6) |
+                   (unsigned(unbase64[src[3]])))) & 0x80000000) {
         // Iff any of those four characters was bad (null, illegal,
         // whitespace, padding), then temp's high bit will be set
         // (because unbase64[] is -1 for all bad characters).
diff --git a/be/src/util/CMakeLists.txt b/be/src/util/CMakeLists.txt
index b12ae2d..605060e 100644
--- a/be/src/util/CMakeLists.txt
+++ b/be/src/util/CMakeLists.txt
@@ -201,4 +201,5 @@ ADD_UNIFIED_BE_LSAN_TEST(time-test "TimeTest.*")
 ADD_UNIFIED_BE_LSAN_TEST(uid-util-test "UidUtil.*")
 # Using standalone webserver-test for now, nonstandard main() passes in a port.
 ADD_BE_LSAN_TEST(webserver-test)
+TARGET_LINK_LIBRARIES(webserver-test mini_kdc)
 ADD_UNIFIED_BE_LSAN_TEST(zip-util-test "ZipUtilTest.*")
diff --git a/be/src/util/kudu-status-util.h b/be/src/util/kudu-status-util.h
new file mode 100644
index 0000000..5b9484b
--- /dev/null
+++ b/be/src/util/kudu-status-util.h
@@ -0,0 +1,59 @@
+// 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 "common/status.h"
+#include "gutil/strings/substitute.h"
+#include "kudu/util/status.h"
+
+/// Takes a Kudu status and returns an impala one, if it's not OK.
+/// Evaluates the prepend argument only if the status is not OK.
+#define KUDU_RETURN_IF_ERROR(expr, prepend)                        \
+  do {                                                             \
+    const kudu::Status& _s = (expr);                               \
+    if (UNLIKELY(!_s.ok())) {                                      \
+      return impala::Status(strings::Substitute(                   \
+          "$0: $1", prepend, _s.ToString()));                      \
+    }                                                              \
+  } while (0)
+
+#define KUDU_ASSERT_OK(status)                                     \
+  do {                                                             \
+    const impala::Status& _s = FromKuduStatus(status);             \
+    ASSERT_TRUE(_s.ok()) << "Error: " << _s.GetDetail();           \
+  } while (0)
+
+
+namespace impala {
+
+/// Utility function for creating an Impala Status object based on a kudu::Status object.
+/// 'k_status' is the kudu::Status object.
+/// 'prepend' is a string to be prepended to details of 'k_status' when creating the
+/// Impala Status object.
+/// Note that we don't translate the kudu::Status error code to Impala error code
+/// so the returned status' type is always of TErrorCode::GENERAL.
+inline Status FromKuduStatus(
+    const kudu::Status& k_status, const std::string prepend = "") {
+  if (LIKELY(k_status.ok())) return Status::OK();
+  const std::string& err_msg = prepend.empty() ? k_status.ToString() :
+      strings::Substitute("$0: $1", prepend, k_status.ToString());
+  VLOG(1) << err_msg;
+  return Status::Expected(err_msg);
+}
+
+} // namespace impala
diff --git a/be/src/util/webserver-test.cc b/be/src/util/webserver-test.cc
index c0c542a..f56bc68 100644
--- a/be/src/util/webserver-test.cc
+++ b/be/src/util/webserver-test.cc
@@ -25,9 +25,14 @@
 #include "common/init.h"
 #include "testutil/gtest-util.h"
 #include "testutil/scoped-flag-setter.h"
-#include "util/webserver.h"
+
 #include "util/default-path-handlers.h"
+#include "util/kudu-status-util.h"
+#include "util/webserver.h"
+
+#include "kudu/security/test/mini_kdc.h"
 
+DECLARE_bool(webserver_require_spnego);
 DECLARE_int32(webserver_port);
 DECLARE_string(webserver_password_file);
 DECLARE_string(webserver_certificate_file);
@@ -320,6 +325,32 @@ TEST(Webserver, SslGoodTlsVersion) {
   }
 }
 
+using kudu::MiniKdc;
+using kudu::MiniKdcOptions;
+
+TEST(Webserver, TestWithSpnego) {
+  MiniKdc kdc(MiniKdcOptions{});
+  KUDU_ASSERT_OK(kdc.Start());
+  kdc.SetKrb5Environment();
+
+  string kt_path;
+  KUDU_ASSERT_OK(kdc.CreateServiceKeytab("HTTP/127.0.0.1", &kt_path));
+  CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1));
+  KUDU_ASSERT_OK(kdc.CreateUserPrincipal("alice"));
+
+  gflags::FlagSaver saver;
+  FLAGS_webserver_require_spnego = true;
+
+  Webserver webserver(FLAGS_webserver_port);
+  ASSERT_OK(webserver.Start());
+
+  // Don't expect HTTP requests to work without Kerberos credentials.
+  stringstream contents;
+  ASSERT_FALSE(HttpGet("localhost", FLAGS_webserver_port, "/", &contents).ok());
+
+  // TODO(todd): import curl into native-toolchain and test this with
+  // authentication.
+}
 
 TEST(Webserver, StartWithPasswordFileTest) {
   stringstream password_file;
diff --git a/be/src/util/webserver.cc b/be/src/util/webserver.cc
index caf45d3..f222bdf 100644
--- a/be/src/util/webserver.cc
+++ b/be/src/util/webserver.cc
@@ -28,12 +28,18 @@
 #include <boost/lexical_cast.hpp>
 #include <boost/mem_fn.hpp>
 #include <boost/thread/locks.hpp>
-#include <gutil/strings/substitute.h>
 #include <rapidjson/document.h>
 #include <rapidjson/prettywriter.h>
 #include <rapidjson/stringbuffer.h>
 
 #include "common/logging.h"
+#include "gutil/endian.h"
+#include "gutil/strings/substitute.h"
+#include "gutil/strings/strip.h"
+#include "kudu/util/env.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/security/gssapi.h"
 #include "rpc/thrift-util.h"
 #include "runtime/exec-env.h"
 #include "service/impala-server.h"
@@ -41,13 +47,13 @@
 #include "util/asan.h"
 #include "util/coding-util.h"
 #include "util/cpu-info.h"
+#include "util/debug-util.h"
 #include "util/disk-info.h"
 #include "util/mem-info.h"
 #include "util/os-info.h"
 #include "util/os-util.h"
-#include "util/process-state-info.h"
-#include "util/debug-util.h"
 #include "util/pretty-printer.h"
+#include "util/process-state-info.h"
 #include "util/stopwatch.h"
 
 #include "common/names.h"
@@ -105,6 +111,10 @@ DEFINE_int32(webserver_max_post_length_bytes, 1024 * 1024,
              "The maximum length of a POST request that will be accepted by "
              "the embedded web server.");
 
+DEFINE_bool(webserver_require_spnego, false,
+            "Require connections to the web server to authenticate via Kerberos "
+            "using SPNEGO.");
+
 DECLARE_bool(is_coordinator);
 DECLARE_string(ssl_minimum_version);
 DECLARE_string(ssl_cipher_list);
@@ -163,6 +173,53 @@ string HttpStatusCodeToString(HttpStatusCode code) {
   LOG(FATAL) << "Unexpected HTTP response code";
   return "";
 }
+
+
+void SendPlainResponse(struct sq_connection* connection,
+                       const string& response_code_line,
+                       const string& content,
+                       const vector<string>& header_lines) {
+  sq_printf(connection, "HTTP/1.1 %s\r\n", response_code_line.c_str());
+  for (const auto& h : header_lines) {
+    sq_printf(connection, "%s\r\n", h.c_str());
+  }
+  sq_printf(connection, "Content-Type: text/plain\r\n");
+  sq_printf(connection, "Content-Length: %zd\r\n\r\n", content.size());
+  sq_printf(connection, "%s", content.c_str());
+}
+
+// Return the address of the remote user from the squeasel request info.
+kudu::Sockaddr GetRemoteAddress(const struct sq_request_info* req) {
+  struct sockaddr_in addr;
+  addr.sin_family = AF_INET;
+  addr.sin_port = NetworkByteOrder::FromHost16(req->remote_port);
+  addr.sin_addr.s_addr = NetworkByteOrder::FromHost32(req->remote_ip);
+  return kudu::Sockaddr(addr);
+}
+
+
+// Performs a step of SPNEGO authorization by parsing the HTTP Authorization header
+// 'authz_header' and running it through GSSAPI. If authentication fails or the header
+// is invalid, a bad Status will be returned (and the other out-parameters left
+// untouched).
+kudu::Status RunSpnegoStep(const char* authz_header, string* resp_header,
+                     string* authn_user) {
+  string neg_token;
+  if (authz_header && !TryStripPrefixString(authz_header, "Negotiate ", &neg_token)) {
+    return kudu::Status::InvalidArgument("bad Negotiate header");
+  }
+
+  string resp_token_b64;
+  bool is_complete;
+  RETURN_NOT_OK(kudu::gssapi::SpnegoStep(
+      neg_token, &resp_token_b64, &is_complete, authn_user));
+
+  if (!resp_token_b64.empty()) {
+    *resp_header = Substitute("WWW-Authenticate: Negotiate $0", resp_token_b64);
+  }
+   return is_complete ? kudu::Status::OK() : kudu::Status::Incomplete("authn incomplete");
+}
+
 } // anonymous namespace
 
 // Builds a valid HTTP header given the response code and a content type.
@@ -310,6 +367,19 @@ Status Webserver::Start() {
     options.push_back(FLAGS_webserver_password_file.c_str());
   }
 
+  if (FLAGS_webserver_require_spnego) {
+    // If Kerberos has been configured, security::InitKerberosForServer() will
+    // already have been called, ensuring that the keytab path has been
+    // propagated into this environment variable where the GSSAPI calls will
+    // pick it up. In other words, we aren't expecting users to pass in this
+    // environment variable specifically.
+    const char* kt_file = getenv("KRB5_KTNAME");
+    if (!kt_file || !kudu::Env::Default()->FileExists(kt_file)) {
+      return Status("Unable to configure web server for SPNEGO authentication: "
+                    "must configure a keytab file for the server");
+    }
+  }
+
   options.push_back("listening_ports");
   options.push_back(listening_str.c_str());
 
@@ -399,6 +469,13 @@ sq_callback_result_t Webserver::BeginRequestCallbackStatic(
 
 sq_callback_result_t Webserver::BeginRequestCallback(struct sq_connection* connection,
     struct sq_request_info* request_info) {
+  if (FLAGS_webserver_require_spnego){
+    sq_callback_result_t spnego_result = HandleSpnego(connection, request_info);
+    if (spnego_result != SQ_CONTINUE_HANDLING) {
+      return spnego_result;
+    }
+  }
+
   if (!FLAGS_webserver_doc_root.empty() && FLAGS_enable_webserver_doc_root) {
     if (strncmp(DOC_FOLDER, request_info->uri, DOC_FOLDER_LEN) == 0) {
       VLOG(2) << "HTTP File access: " << request_info->uri;
@@ -501,6 +578,88 @@ sq_callback_result_t Webserver::BeginRequestCallback(struct sq_connection* conne
   return SQ_HANDLED_OK;
 }
 
+sq_callback_result_t Webserver::HandleSpnego(
+    struct sq_connection* connection,
+    struct sq_request_info* request_info) {
+  const char* authz_header = sq_get_header(connection, "Authorization");
+  string resp_header, authn_princ;
+  kudu::Status s = RunSpnegoStep(authz_header, &resp_header, &authn_princ);
+  if (s.IsIncomplete()) {
+    SendPlainResponse(connection, "401 Authentication Required",
+                      "Must authenticate with SPNEGO.",
+                      { resp_header });
+    return SQ_HANDLED_OK;
+  }
+  if (s.ok() && authn_princ.empty()) {
+    s = kudu::Status::RuntimeError("SPNEGO indicated complete, but got empty principal");
+    // Crash in debug builds, but fall through to treating as an error 500 in
+    // release.
+    LOG(DFATAL) << "Got no authenticated principal for SPNEGO-authenticated "
+                << " connection from "
+                << GetRemoteAddress(request_info).ToString()
+                << ": " << s.ToString();
+  }
+  if (!s.ok()) {
+    LOG(WARNING) << "Failed to authenticate request from "
+                 << GetRemoteAddress(request_info).ToString()
+                 << " via SPNEGO: " << s.ToString();
+    const char* http_status = s.IsNotAuthorized() ? "401 Authentication Required" :
+        "500 Internal Server Error";
+
+    SendPlainResponse(connection, http_status, s.ToString(), {});
+    return SQ_HANDLED_OK;
+  }
+
+
+  request_info->remote_user = strdup(authn_princ.c_str());
+
+  // NOTE: According to the SPNEGO RFC (https://tools.ietf.org/html/rfc4559) it
+  // is possible that a non-empty token will be returned along with the HTTP 200
+  // response:
+  //
+  //     A status code 200 status response can also carry a "WWW-Authenticate"
+  //     response header containing the final leg of an authentication.  In
+  //     this case, the gssapi-data will be present.  Before using the
+  //     contents of the response, the gssapi-data should be processed by
+  //     gss_init_security_context to determine the state of the security
+  //     context.  If this function indicates success, the response can be
+  //     used by the application.  Otherwise, an appropriate action, based on
+  //     the authentication status, should be taken.
+  //
+  //     For example, the authentication could have failed on the final leg if
+  //     mutual authentication was requested and the server was not able to
+  //     prove its identity.  In this case, the returned results are suspect.
+  //     It is not always possible to mutually authenticate the server before
+  //     the HTTP operation.  POST methods are in this category.
+  //
+  // In fact, from inspecting the MIT krb5 source code, it appears that this
+  // only happens when the client requests mutual authentication by passing
+  // 'GSS_C_MUTUAL_FLAG' when establishing its side of the protocol. In practice,
+  // this seems to be widely unimplemented:
+  //
+  // - curl has some source code to support GSS_C_MUTUAL_FLAG, but in order to
+  //   enable it, you have to modify a FALSE constant to TRUE and recompile curl.
+  //   In fact, it was broken for all of 2015 without anyone noticing (see curl
+  //   commit 73f1096335d468b5be7c3cc99045479c3314f433)
+  //
+  // - Chrome doesn't support mutual auth at all -- see DelegationTypeToFlag(...)
+  //   in src/net/http/http_auth_gssapi_posix.cc.
+  //
+  // In practice, users depend on TLS to authenticate the server, and SPNEGO
+  // is used to authenticate the client.
+  //
+  // Given this, and because actually sending back the token on an OK response
+  // would require significant code restructuring (eg buffering the header until
+  // after the response handler has run) we just ignore any response token, but
+  // log a periodic warning just in case it turns out we're wrong about the above.
+  if (!resp_header.empty()) {
+    KLOG_EVERY_N_SECS(WARNING, 5) << "ignoring SPNEGO token on HTTP 200 response "
+                                  << "for user " << authn_princ << " at host "
+                                  << GetRemoteAddress(request_info).ToString();
+  }
+  return SQ_CONTINUE_HANDLING;
+}
+
 void Webserver::RenderUrlWithTemplate(const WebRequest& req,
     const UrlHandler& url_handler, stringstream* output, ContentType* content_type) {
   Document document;
diff --git a/be/src/util/webserver.h b/be/src/util/webserver.h
index 21b85be..7d36894 100644
--- a/be/src/util/webserver.h
+++ b/be/src/util/webserver.h
@@ -163,6 +163,13 @@ class Webserver {
   sq_callback_result_t BeginRequestCallback(struct sq_connection* connection,
       struct sq_request_info* request_info);
 
+  // Handle SPNEGO authentication for this request. Returns SQ_CONTINUE_HANDLING
+  // if authentication was successful, otherwise responds to the request and
+  // returns SQ_HANDLED_OK.
+  sq_callback_result_t HandleSpnego(
+      struct sq_connection* connection,
+      struct sq_request_info* request_info);
+
   /// Renders URLs through the Mustache templating library.
   /// - Default ContentType is HTML.
   /// - Argument 'raw' renders the page with PLAIN ContentType.