You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by gr...@apache.org on 2019/06/07 21:41:18 UTC

[kudu] 02/03: Support SPNEGO for web server

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

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

commit 1f291b77ef0868ac888a850678adc2d7cce65529
Author: Todd Lipcon <to...@apache.org>
AuthorDate: Tue May 14 22:09:36 2019 -0700

    Support SPNEGO for web server
    
    SPNEGO is a protocol for securing HTTP requests with Kerberos by passing
    negotiation through HTTP headers. It's supported by most major browsers
    and also by most of the Java-based Hadoop components. Notably, it's also
    the typical way in which Apache Knox authenticates itself to Hadoop
    components in the "trusted proxy" mode, allowing them to be secured
    behind Knox's SSO and other policies.
    
    This patch implements the SPNEGO protocol by driving GSSAPI, and
    integrates it into the webserver when configured by a new
    --webserver_require_spnego flag.
    
    The new test verifies this end-to-end using curl's SPNEGO authentication
    support.
    
    Along the way I had to cross-port a small change to the Base64 functions
    in gutil to avoid a UBSAN error. I found the fix in abseil-cpp's copy of
    the same file.
    
    Change-Id: I9449ac610aa7d11bbf320d9178a6d73684ff15f7
    Reviewed-on: http://gerrit.cloudera.org:8080/13341
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/gutil/strings/escaping.cc   |   8 +-
 src/kudu/rpc/client_negotiation.cc   |  35 +------
 src/kudu/security/CMakeLists.txt     |   2 +
 src/kudu/security/gssapi.cc          | 164 +++++++++++++++++++++++++++++
 src/kudu/security/gssapi.h           |  59 +++++++++++
 src/kudu/security/test/mini_kdc.cc   |   9 ++
 src/kudu/security/test/mini_kdc.h    |   4 +
 src/kudu/server/CMakeLists.txt       |   1 +
 src/kudu/server/webserver-test.cc    | 153 ++++++++++++++++++++++++++-
 src/kudu/server/webserver.cc         | 194 ++++++++++++++++++++++++++++++-----
 src/kudu/server/webserver_options.cc |   8 +-
 src/kudu/server/webserver_options.h  |   9 +-
 src/kudu/util/curl_util.cc           |  14 +++
 src/kudu/util/curl_util.h            |  14 +++
 src/kudu/util/test_macros.h          |   7 ++
 thirdparty/build-definitions.sh      |   7 +-
 16 files changed, 623 insertions(+), 65 deletions(-)

diff --git a/src/kudu/gutil/strings/escaping.cc b/src/kudu/gutil/strings/escaping.cc
index b21f981..e8ff33f 100644
--- a/src/kudu/gutil/strings/escaping.cc
+++ b/src/kudu/gutil/strings/escaping.cc
@@ -898,10 +898,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/src/kudu/rpc/client_negotiation.cc b/src/kudu/rpc/client_negotiation.cc
index e003a2c..d74bdbe 100644
--- a/src/kudu/rpc/client_negotiation.cc
+++ b/src/kudu/rpc/client_negotiation.cc
@@ -44,6 +44,7 @@
 #include "kudu/rpc/sasl_helper.h"
 #include "kudu/rpc/serialization.h"
 #include "kudu/security/cert.h"
+#include "kudu/security/gssapi.h"
 #include "kudu/security/tls_context.h"
 #include "kudu/security/tls_handshake.h"
 #include "kudu/util/faststring.h"
@@ -789,36 +790,6 @@ int ClientNegotiation::SecretCb(sasl_conn_t* conn, int id, sasl_secret_t** psecr
   return SASL_OK;
 }
 
-namespace {
-// Retrieve the GSSAPI error description for an error code and type.
-string gss_error_description(OM_uint32 code, int type) {
-  string description;
-  OM_uint32 message_context = 0;
-
-  do {
-    if (!description.empty()) {
-      description.append(": ");
-    }
-    OM_uint32 minor = 0;
-    gss_buffer_desc buf;
-    gss_display_status(&minor, code, type, GSS_C_NULL_OID, &message_context, &buf);
-    description.append(static_cast<const char*>(buf.value), buf.length);
-    gss_release_buffer(&minor, &buf);
-  } while (message_context != 0);
-
-  return description;
-}
-
-// Transforms a GSSAPI major and minor error code into a Kudu Status.
-Status check_gss_error(OM_uint32 major, OM_uint32 minor) {
-    if (GSS_ERROR(major)) {
-      return Status::NotAuthorized(gss_error_description(major, GSS_C_GSS_CODE),
-                                   gss_error_description(minor, GSS_C_MECH_CODE));
-    }
-    return Status::OK();
-}
-} // anonymous namespace
-
 Status ClientNegotiation::CheckGSSAPI() {
   OM_uint32 major, minor;
   gss_cred_id_t cred = GSS_C_NO_CREDENTIAL;
@@ -834,7 +805,7 @@ Status ClientNegotiation::CheckGSSAPI() {
                            &cred,
                            nullptr,
                            nullptr);
-  Status s = check_gss_error(major, minor);
+  Status s = gssapi::MajorMinorToStatus(major, minor);
 
   // Inspect the Kerberos credential to determine if it is expired. The lifetime
   // returned from gss_acquire_cred in the RHEL 6 version of krb5 is always 0,
@@ -843,7 +814,7 @@ Status ClientNegotiation::CheckGSSAPI() {
   OM_uint32 lifetime;
   if (s.ok()) {
     major = gss_inquire_cred(&minor, cred, nullptr, &lifetime, nullptr, nullptr);
-    s = check_gss_error(major, minor);
+    s = gssapi::MajorMinorToStatus(major, minor);
   }
 
   // Release the credential even if gss_inquire_cred fails.
diff --git a/src/kudu/security/CMakeLists.txt b/src/kudu/security/CMakeLists.txt
index e0aca70..7abaabb 100644
--- a/src/kudu/security/CMakeLists.txt
+++ b/src/kudu/security/CMakeLists.txt
@@ -68,6 +68,7 @@ set(SECURITY_SRCS
   cert.cc
   crypto.cc
   kerberos_util.cc
+  gssapi.cc
   init.cc
   openssl_util.cc
   ${PORTED_X509_CHECK_HOST_CC}
@@ -86,6 +87,7 @@ set(SECURITY_LIBS
   kudu_util
   token_proto
 
+  gssapi_krb5
   krb5
   openssl_crypto
   openssl_ssl)
diff --git a/src/kudu/security/gssapi.cc b/src/kudu/security/gssapi.cc
new file mode 100644
index 0000000..6797ec3
--- /dev/null
+++ b/src/kudu/security/gssapi.cc
@@ -0,0 +1,164 @@
+// 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/gssapi.h"
+
+#include <cstring>
+#include <string>
+#include <utility>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/escaping.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/status.h"
+
+using std::string;
+
+namespace kudu {
+namespace gssapi {
+
+namespace {
+
+string ErrorToString(OM_uint32 code, OM_uint32 type) {
+  string description;
+  OM_uint32 message_context = 0;
+
+  do {
+    if (!description.empty()) {
+      description.append(": ");
+    }
+    OM_uint32 minor = 0;
+    gss_buffer_desc buf;
+    gss_display_status(&minor, code, type, GSS_C_NULL_OID, &message_context, &buf);
+    const char* err = static_cast<const char*>(buf.value);
+    // The error message buffer returned has an explicit length, but in some cases
+    // this length appears to include the null terminator character, and in others
+    // it doesn't. So, we trim off any null terminators if necessary.
+    int err_len = strnlen(err, buf.length);
+    description.append(err, err_len);
+
+    // NOTE: call gss_release_buffer explicitly here instead of ReleaseBufferOrWarn
+    // to avoid the potential for infinite recursion back into ErrorToString.
+    gss_release_buffer(&minor, &buf);
+  } while (message_context != 0);
+
+  return description;
+}
+
+// Wrap a call to F(...), which is typically one of the gss_* functions from
+// the gssapi library. These functions all return a major status code and
+// also provide a minor status code as their first out-param.
+template <class F, class... Args>
+Status WrapGssCall(F func, Args&&... args) {
+  OM_uint32 minor = 0;
+  OM_uint32 major = func(&minor, std::forward<Args>(args)...);
+  return MajorMinorToStatus(major, minor);
+}
+
+void ReleaseNameOrWarn(gss_name_t name) {
+  if (name == GSS_C_NO_NAME) return;
+  WARN_NOT_OK(WrapGssCall(gss_release_name, &name), "Unable to release GSS name");
+}
+
+void ReleaseBufferOrWarn(gss_buffer_t buf) {
+  if (buf == GSS_C_NO_BUFFER) return;
+  WARN_NOT_OK(WrapGssCall(gss_release_buffer, buf), "Unable to release GSS buffer");
+}
+
+void ReleaseContextOrWarn(gss_ctx_id_t ctx) {
+  if (ctx == GSS_C_NO_CONTEXT) return;
+  WARN_NOT_OK(WrapGssCall(gss_delete_sec_context, &ctx, GSS_C_NO_BUFFER),
+              "Unable to release GSS context");
+}
+
+} // anonymous namespace
+
+
+Status MajorMinorToStatus(OM_uint32 major, OM_uint32 minor) {
+  if (GSS_ERROR(major)) {
+    string maj_str = ErrorToString(major, GSS_C_GSS_CODE);
+    string min_str = minor != 0 ? ErrorToString(minor, GSS_C_MECH_CODE) : "";
+    return Status::NotAuthorized(maj_str, min_str);
+  }
+  if (major == GSS_S_CONTINUE_NEEDED) {
+    return Status::Incomplete("");
+  }
+
+  return Status::OK();
+}
+
+
+Status SpnegoStep(const string& in_token_b64,
+                  string* out_token_b64,
+                  bool* complete,
+                  string* authenticated_principal) {
+  string token;
+  if (!strings::Base64Unescape(in_token_b64, &token)) {
+    return Status::InvalidArgument("invalid base64 encoding for token");
+  }
+
+  // Workaround MIT krb5 bug [1] fixed in krb5 1.16 and 1.15.3:
+  //
+  // gssint_get_mech_type_oid() was missing some length verification that could
+  // cause reads past the end of the input token. So, we extend the actual
+  // allocation of the input token an extra 256 bytes of padding.
+  //
+  // Without this fix, our ASAN builds fail with an out-of-bounds read.
+  //
+  // [1] http://krbdev.mit.edu/rt/Ticket/History.html?id=8620
+  size_t real_token_size = token.size();
+  token.resize(real_token_size + 256);
+
+  gss_buffer_desc input_token {real_token_size, const_cast<char*>(token.data())};
+
+  gss_ctx_id_t ctx = GSS_C_NO_CONTEXT;
+  gss_name_t client_name = GSS_C_NO_NAME;
+  SCOPED_CLEANUP({ ReleaseNameOrWarn(client_name); });
+
+  gss_buffer_desc out_token {0, nullptr};
+  SCOPED_CLEANUP({ ReleaseBufferOrWarn(&out_token); });
+  Status s = WrapGssCall(gss_accept_sec_context, &ctx, GSS_C_NO_CREDENTIAL,
+                         &input_token, GSS_C_NO_CHANNEL_BINDINGS,
+                         &client_name, /*mech_type=*/ nullptr,
+                         &out_token, /*ret_flags=*/nullptr,
+                         /*time_rec=*/nullptr, /*delegated_cred_handle=*/nullptr);
+  SCOPED_CLEANUP({ ReleaseContextOrWarn(ctx); });
+  if (!s.ok() && !s.IsIncomplete()) {
+    return s;
+  }
+  *complete = s.ok();
+
+  if (*complete) {
+    gss_buffer_desc name_buf {0, nullptr};
+    SCOPED_CLEANUP({ ReleaseBufferOrWarn(&name_buf); });
+    RETURN_NOT_OK_PREPEND(WrapGssCall(gss_display_name, client_name, &name_buf, nullptr),
+                          "Unable to extract authenticated principal name");
+    authenticated_principal->assign(reinterpret_cast<char*>(name_buf.value),
+                                    name_buf.length);
+  }
+
+  string out_token_str(reinterpret_cast<char*>(out_token.value),
+                       out_token.length);
+  strings::Base64Escape(out_token_str, out_token_b64);
+
+  return Status::OK();
+}
+
+
+} // namespace gssapi
+} // namespace kudu
diff --git a/src/kudu/security/gssapi.h b/src/kudu/security/gssapi.h
new file mode 100644
index 0000000..036b3d4
--- /dev/null
+++ b/src/kudu/security/gssapi.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 <string>
+
+#include <gssapi/gssapi.h>
+
+namespace kudu {
+
+class Status;
+
+namespace gssapi {
+
+// Convert the given major/minor GSSAPI error codes into a Status.
+Status MajorMinorToStatus(OM_uint32 major, OM_uint32 minor);
+
+// Run a step of SPNEGO authentication.
+//
+// 'in_token_b64' is the base64-encoded token provided by the client, which may be empty
+// if the client did not provide any such token (e.g. if the HTTP 'Authorization' header
+// was not present).
+
+// 'out_token_b64' is the base64-encoded output token to send back to the client
+// during this round of negotiation.
+//
+// If any error occurs (eg an invalid token is provided), a bad Status is returned.
+//
+// An OK status indicates that the negotiation is proceeding successfully, or has
+// completed, whereas a non-OK status indicates an error or an unsuccessful
+// authentication (in which case the out-parameters will not be modified).
+//
+// In the case of an OK status, '*complete' indicates whether any further rounds are
+// required. On completion of negotiation, 'authenticated_principal' will be set to the
+// full principal name of the remote user.
+//
+// NOTE: per the SPNEGO protocol, the final "complete" negotiation stage may
+// include a token.
+Status SpnegoStep(const std::string& in_token_b64,
+                  std::string* out_token_b64,
+                  bool* complete,
+                  std::string* authenticated_principal);
+
+} // namespace gssapi
+} // namespace kudu
diff --git a/src/kudu/security/test/mini_kdc.cc b/src/kudu/security/test/mini_kdc.cc
index f61b3ef..1662770 100644
--- a/src/kudu/security/test/mini_kdc.cc
+++ b/src/kudu/security/test/mini_kdc.cc
@@ -276,6 +276,15 @@ Status MiniKdc::CreateServiceKeytab(const string& spn,
   return Status::OK();
 }
 
+Status MiniKdc::RandomizePrincipalKey(const string& spn) {
+  SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, Substitute("randomizing key for $0", spn));
+  string kadmin;
+  RETURN_NOT_OK(GetBinaryPath("kadmin.local", &kadmin));
+  RETURN_NOT_OK(Subprocess::Call(MakeArgv({
+          kadmin, "-q", Substitute("change_password -randkey $0", spn)})));
+  return Status::OK();
+}
+
 Status MiniKdc::CreateKeytabForExistingPrincipal(const string& spn) {
   SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, Substitute("creating keytab for $0", spn));
   string kt_path = spn;
diff --git a/src/kudu/security/test/mini_kdc.h b/src/kudu/security/test/mini_kdc.h
index 95e7848..94990a2 100644
--- a/src/kudu/security/test/mini_kdc.h
+++ b/src/kudu/security/test/mini_kdc.h
@@ -92,6 +92,10 @@ class MiniKdc {
   // will be reset and a new keytab will be generated.
   Status CreateServiceKeytab(const std::string& spn, std::string* path);
 
+  // Randomize the key for the given SPN. This invalidates any previously-produced
+  // keytabs.
+  Status RandomizePrincipalKey(const std::string& spn);
+
   // Creates a keytab for an existing principal.
   // 'spn' is the desired service principal name (e.g. "kudu/foo.example.com").
   Status CreateKeytabForExistingPrincipal(const std::string& spn);
diff --git a/src/kudu/server/CMakeLists.txt b/src/kudu/server/CMakeLists.txt
index aa0ce91..ce1d5dd 100644
--- a/src/kudu/server/CMakeLists.txt
+++ b/src/kudu/server/CMakeLists.txt
@@ -80,6 +80,7 @@ endif()
 
 SET_KUDU_TEST_LINK_LIBS(
   kudu_curl_util
+  mini_kdc
   server_process
   security_test_util)
 ADD_KUDU_TEST(rpc_server-test)
diff --git a/src/kudu/server/webserver-test.cc b/src/kudu/server/webserver-test.cc
index aa4da52..8266c1c 100644
--- a/src/kudu/server/webserver-test.cc
+++ b/src/kudu/server/webserver-test.cc
@@ -17,6 +17,8 @@
 
 #include "kudu/server/webserver.h"
 
+#include <cstdlib>
+#include <functional>
 #include <iosfwd>
 #include <memory>
 #include <string>
@@ -30,8 +32,10 @@
 #include "kudu/gutil/integral_types.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/escaping.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
+#include "kudu/security/test/mini_kdc.h"
 #include "kudu/security/test/test_certs.h"
 #include "kudu/security/test/test_pass.h"
 #include "kudu/server/default_path_handlers.h"
@@ -73,6 +77,7 @@ void SetHTPasswdOptions(WebserverOptions* opts) {
   CHECK_OK(security::CreateTestHTPasswd(GetTestDataDirectory(),
                                         &opts->password_file));
 }
+
 } // anonymous namespace
 
 class WebserverTest : public KuduTest {
@@ -90,6 +95,7 @@ class WebserverTest : public KuduTest {
     opts.doc_root = static_dir_;
     if (use_ssl()) SetSslOptions(&opts);
     if (use_htpasswd()) SetHTPasswdOptions(&opts);
+    MaybeSetupSpnego(&opts);
     server_.reset(new Webserver(opts));
 
     AddDefaultPathHandlers(server_.get());
@@ -98,10 +104,13 @@ class WebserverTest : public KuduTest {
     vector<Sockaddr> addrs;
     ASSERT_OK(server_->GetBoundAddresses(&addrs));
     ASSERT_EQ(addrs.size(), 1);
-    addr_ = addrs[0];
+    ASSERT_TRUE(addrs[0].IsWildcard());
+    ASSERT_OK(addr_.ParseString("127.0.0.1", addrs[0].port()));
   }
 
  protected:
+  virtual void MaybeSetupSpnego(WebserverOptions* /*opts*/) {}
+
   // Overridden by subclasses.
   virtual bool use_ssl() const { return false; }
   virtual bool use_htpasswd() const { return false; }
@@ -138,6 +147,142 @@ TEST_F(PasswdWebserverTest, TestPasswdPresent) {
   ASSERT_OK(curl_.FetchURL(auth_url, &buf_));
 }
 
+
+class SpnegoWebserverTest : public WebserverTest {
+ protected:
+  void MaybeSetupSpnego(WebserverOptions* opts) override {
+    kdc_.reset(new MiniKdc(MiniKdcOptions{}));
+    ASSERT_OK(kdc_->Start());
+    kdc_->SetKrb5Environment();
+    string kt_path;
+    ASSERT_OK(kdc_->CreateServiceKeytab("HTTP/127.0.0.1", &kt_path));
+    CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1));
+    ASSERT_OK(kdc_->CreateUserPrincipal("alice"));
+
+    opts->require_spnego = true;
+    opts->spnego_post_authn_callback = [&](const string& spn) {
+      last_authenticated_spn_ = spn;
+    };
+  }
+
+  Status DoSpnegoCurl() {
+    EasyCurl c;
+    c.set_use_spnego(true);
+    if (VLOG_IS_ON(1)) {
+      c.set_verbose(true);
+    }
+    return c.FetchURL(strings::Substitute("http://$0/", addr_.ToString()), &buf_);
+  }
+
+  unique_ptr<MiniKdc> kdc_;
+  const char* const kNotAuthn = "<none>";
+  string last_authenticated_spn_ = kNotAuthn;
+
+  // A SPNEGO token manually captured from a client exchange during some prior test run.
+  // This is used as a basis for some fuzz tests.
+  const char* kWellFormedTokenBase64 =
+      "YIICVwYGKwYBBQUCoIICSzCCAkegDTALBgkqhkiG9xIBAgKiggI0BIICMGCCAiwGCSqGSIb3EgECAgEA"
+      "boICGzCCAhegAwIBBaEDAgEOogcDBQAgAAAAo4IBP2GCATswggE3oAMCAQWhDRsLS1JCVEVTVC5DT02i"
+      "HDAaoAMCAQOhEzARGwRIVFRQGwkxMjcuMC4wLjGjggEBMIH+oAMCARGhAwIBA6KB8QSB7pQrIA2cH2l4"
+      "yfHpwhKz2HKYNxoxOw1j++ODByOfN3O/j9/Pp9PwJzQ7hjo5p5nK2OD+2S5YVuS92Ax/LiX8WaYxt9LC"
+      "Hew8TkssFOiDffhag1taEcMG5KksPVxZejs+4NYiLj8dCwow3kShl/fpaLYXFFUgChaM7mVEDfMEIdos"
+      "WB56k/KMJas7kuAkqDy8sEdPpgzbV7tPmkIFecXPKugZFTttkMREe19LcGO2KnOFflLj7s5F4euWzhrG"
+      "v3oZXxDh0G6iyTouSEH+oh/LG97I0umcHrcEit6CjcjVewNhIUaP/Vn2Cu6X0FsF45qkgb4wgbugAwIB"
+      "EaKBswSBsLrv38pBLMZo74lMEWHyOrbwrBG0kHfLHVSnxJJYikOwjAoNUm0/NUJc801TtbQZX/e6nRjS"
+      "4spS2eU1xnPLcVBbtnonkG7xWSDv/Sl/k73oy7rObVWGQAtYkCJdcfWj1mxeojtrOPcKa9ivBiAuKcKl"
+      "EdT2XD6lk161ygu306e7eH8pcuHv+bl9zP42rj85S0c3q0KXRXvsegAFUFk34+AC3fbmKLddEBUoYxms"
+      "f+uj";
+
+};
+
+TEST_F(SpnegoWebserverTest, TestAuthenticated) {
+  ASSERT_OK(kdc_->Kinit("alice"));
+  ASSERT_OK(DoSpnegoCurl());
+  EXPECT_EQ("alice@KRBTEST.COM", last_authenticated_spn_);
+  EXPECT_STR_CONTAINS(buf_.ToString(), "Kudu");
+}
+
+TEST_F(SpnegoWebserverTest, TestUnauthenticatedBadKeytab) {
+  ASSERT_OK(kdc_->Kinit("alice"));
+  // Randomize the server's key in the KDC so that the key in the keytab doesn't match the
+  // one for which the client will get a ticket. This is just an easy way to provoke an
+  // error and make sure that our error handling works.
+  ASSERT_OK(kdc_->RandomizePrincipalKey("HTTP/127.0.0.1"));
+
+  Status s = DoSpnegoCurl();
+  EXPECT_EQ(s.ToString(), "Remote error: HTTP 401");
+  EXPECT_EQ(kNotAuthn, last_authenticated_spn_);
+  EXPECT_STR_CONTAINS(buf_.ToString(), "GSS failure");
+}
+
+TEST_F(SpnegoWebserverTest, TestUnauthenticatedNoClientAuth) {
+  Status curl_status = DoSpnegoCurl();
+  EXPECT_EQ("Remote error: HTTP 401", curl_status.ToString());
+  EXPECT_EQ("Must authenticate with SPNEGO.", buf_.ToString());
+  EXPECT_EQ(kNotAuthn, last_authenticated_spn_);
+}
+
+// Test some malformed authorization headers.
+TEST_F(SpnegoWebserverTest, TestInvalidHeaders) {
+  const string& url = strings::Substitute("http://$0/", addr_.ToString());
+  EasyCurl c;
+  EXPECT_EQ(c.FetchURL(url, &buf_, { "Authorization: blahblah" }).ToString(),
+            "Remote error: HTTP 500");
+  EXPECT_STR_CONTAINS(buf_.ToString(), "bad Negotiate header");
+  EXPECT_EQ(c.FetchURL(url, &buf_, { "Authorization: Negotiate aaa" }).ToString(),
+            "Remote error: HTTP 401");
+  EXPECT_STR_CONTAINS(buf_.ToString(), "Invalid token was supplied");
+}
+
+// Test all single-bit-flips of a well-formed token, to make sure we don't
+// crash.
+//
+// NOTE: the original token is *well-formed* but not *valid* -- i.e. even if unmodified,
+// it would not produce a successful authentication result, since it is a saved constant
+// from some previous run of SPNEGO on a different KDC. This test is primarily concerned
+// with defending against remote buffer overflows during token parsing, etc.
+TEST_F(SpnegoWebserverTest, TestBitFlippedTokens) {
+  const string& url = strings::Substitute("http://$0/", addr_.ToString());
+  EasyCurl c;
+  string token;
+  CHECK(strings::Base64Unescape(kWellFormedTokenBase64, &token));
+
+  for (int i = 0; i < token.size(); i++) {
+    SCOPED_TRACE(i);
+    for (int bit = 0; bit < 8; bit++) {
+      SCOPED_TRACE(bit);
+      token[i] ^= 1 << bit;
+      string b64_token;
+      strings::Base64Escape(token, &b64_token);
+      string header = strings::Substitute("Authorization: Negotiate $0", b64_token);
+      Status s = c.FetchURL(url, &buf_, { header });
+      EXPECT_TRUE(s.IsRemoteError()) << s.ToString();
+      token[i] ^= 1 << bit;
+    }
+  }
+}
+
+// Test all truncations of a well-formed token, to make sure we don't
+// crash.
+//
+// NOTE: see above regarding "well-formed" vs "valid".
+TEST_F(SpnegoWebserverTest, TestTruncatedTokens) {
+  const string& url = strings::Substitute("http://$0/", addr_.ToString());
+  EasyCurl c;
+  string token;
+  CHECK(strings::Base64Unescape(kWellFormedTokenBase64, &token));
+
+  do {
+    token.resize(token.size() - 1);
+    SCOPED_TRACE(token.size());
+    string b64_token;
+    strings::Base64Escape(token, &b64_token);
+    string header = strings::Substitute("Authorization: Negotiate $0", b64_token);
+    Status s = c.FetchURL(url, &buf_, { header });
+    EXPECT_TRUE(s.IsRemoteError()) << s.ToString();
+  } while (!token.empty());
+}
+
 TEST_F(WebserverTest, TestIndexPage) {
   curl_.set_return_headers(true);
   ASSERT_OK(curl_.FetchURL(strings::Substitute("http://$0/", addr_.ToString()),
@@ -465,4 +610,10 @@ TEST_F(WebserverNegativeTests, BadAdvertisedAddressesZeroPort) {
     });
 }
 
+TEST_F(WebserverNegativeTests, SpnegoWithoutKeytab) {
+  ExpectFailedStartup([](WebserverOptions* opts) {
+      opts->require_spnego = true;
+    });
+}
+
 } // namespace kudu
diff --git a/src/kudu/server/webserver.cc b/src/kudu/server/webserver.cc
index 304d3e8..7e92a3e 100644
--- a/src/kudu/server/webserver.cc
+++ b/src/kudu/server/webserver.cc
@@ -33,11 +33,13 @@
 #include <vector>
 
 #include <boost/algorithm/string.hpp> // IWYU pragma: keep
+#include <boost/optional.hpp>
 #include <gflags/gflags.h>
 #include <glog/logging.h>
 #include <mustache.h>
 #include <squeasel.h>
 
+#include "kudu/gutil/endian.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/stl_util.h"
@@ -47,6 +49,7 @@
 #include "kudu/gutil/strings/stringpiece.h"
 #include "kudu/gutil/strings/strip.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/gssapi.h"
 #include "kudu/security/openssl_util.h"
 #include "kudu/util/easy_json.h"
 #include "kudu/util/env.h"
@@ -84,33 +87,85 @@ DEFINE_string(webserver_x_frame_options, "DENY",
 TAG_FLAG(webserver_x_frame_options, advanced);
 
 
+namespace kudu {
+
 namespace {
-  // Last error message from the webserver.
-  string kWebserverLastErrMsg;
-
-  string HttpStatusCodeToString(kudu::HttpStatusCode code) {
-    switch (code) {
-      case kudu::HttpStatusCode::Ok:
-        return "200 OK";
-      case kudu::HttpStatusCode::BadRequest:
-        return "400 Bad Request";
-      case kudu::HttpStatusCode::NotFound:
-        return "404 Not Found";
-      case kudu::HttpStatusCode::LengthRequired:
-        return "411 Length Required";
-      case kudu::HttpStatusCode::RequestEntityTooLarge:
-        return "413 Request Entity Too Large";
-      case kudu::HttpStatusCode::InternalServerError:
-        return "500 Internal Server Error";
-      case kudu::HttpStatusCode::ServiceUnavailable:
-        return "503 Service Unavailable";
-    }
-    LOG(FATAL) << "Unexpected HTTP response code";
+
+// Last error message from the webserver.
+// TODO(todd) global strings are somewhat messy and lint is complaining
+// about this. Clean this up.
+string kWebserverLastErrMsg; // NOLINT(runtime/string)
+
+string HttpStatusCodeToString(kudu::HttpStatusCode code) {
+  switch (code) {
+    case kudu::HttpStatusCode::Ok:
+      return "200 OK";
+    case kudu::HttpStatusCode::BadRequest:
+      return "400 Bad Request";
+    case kudu::HttpStatusCode::NotFound:
+      return "404 Not Found";
+    case kudu::HttpStatusCode::LengthRequired:
+      return "411 Length Required";
+    case kudu::HttpStatusCode::RequestEntityTooLarge:
+      return "413 Request Entity Too Large";
+    case kudu::HttpStatusCode::InternalServerError:
+      return "500 Internal Server Error";
+    case kudu::HttpStatusCode::ServiceUnavailable:
+      return "503 Service Unavailable";
   }
+  LOG(FATAL) << "Unexpected HTTP response code";
+}
 
-}  // anonymous namespace
+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());
+}
 
-namespace kudu {
+// Return the address of the remote user from the squeasel request info.
+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 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).
+//
+Status RunSpnegoStep(const char* authz_header, string* resp_header,
+                     string* authn_user) {
+  VLOG(2) << "Handling Authorization header "
+          << (authz_header ? KUDU_REDACT(authz_header) : "<null>");
+
+  string neg_token;
+  if (authz_header && !TryStripPrefixString(authz_header, "Negotiate ", &neg_token)) {
+    return Status::InvalidArgument("bad Negotiate header");
+  }
+
+  string resp_token_b64;
+  bool is_complete;
+  RETURN_NOT_OK(gssapi::SpnegoStep(neg_token, &resp_token_b64, &is_complete, authn_user));
+
+  VLOG(2) << "SPNEGO step complete, response token: " << KUDU_REDACT(resp_token_b64);
+
+  if (!resp_token_b64.empty()) {
+    *resp_header = Substitute("WWW-Authenticate: Negotiate $0", resp_token_b64);
+  }
+  return is_complete ? Status::OK() : Status::Incomplete("authn incomplete");
+}
+
+}  // anonymous namespace
 
 Webserver::Webserver(const WebserverOptions& opts)
   : opts_(opts),
@@ -227,6 +282,17 @@ Status Webserver::Start() {
     options.push_back(opts_.password_file);
   }
 
+  if (opts_.require_spnego) {
+    // We assume that security::InitKerberosForServer() has already been called, which
+    // ensures that the keytab path has been propagated into this environment variable
+    // where the GSSAPI calls will pick it up.
+    const char* kt_file = getenv("KRB5_KTNAME");
+    if (!kt_file || !Env::Default()->FileExists(kt_file)) {
+      return Status::InvalidArgument("Unable to configure web server for SPNEGO authentication: "
+                                     "must configure a keytab file for the server");
+    }
+  }
+
   options.emplace_back("listening_ports");
   string listening_str;
   RETURN_NOT_OK(BuildListenSpec(&listening_str));
@@ -380,6 +446,88 @@ int Webserver::BeginRequestCallbackStatic(struct sq_connection* connection) {
 
 int Webserver::BeginRequestCallback(struct sq_connection* connection,
                                     struct sq_request_info* request_info) {
+  if (opts_.require_spnego) {
+    const char* authz_header = sq_get_header(connection, "Authorization");
+    string resp_header, authn_princ;
+    Status s = RunSpnegoStep(authz_header, &resp_header, &authn_princ);
+    if (s.IsIncomplete()) {
+      SendPlainResponse(connection, "401 Authentication Required",
+                         "Must authenticate with SPNEGO.",
+                         { resp_header });
+      return 1;
+    }
+    if (s.ok() && authn_princ.empty()) {
+      s = 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 1;
+    }
+
+    if (opts_.spnego_post_authn_callback) {
+      opts_.spnego_post_authn_callback(authn_princ);
+    }
+
+    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();
+    }
+  }
+
   PathHandler* handler;
   {
     shared_lock<RWMutex> l(lock_);
diff --git a/src/kudu/server/webserver_options.cc b/src/kudu/server/webserver_options.cc
index 65bdc56..dc0108a 100644
--- a/src/kudu/server/webserver_options.cc
+++ b/src/kudu/server/webserver_options.cc
@@ -109,6 +109,11 @@ DEFINE_string(webserver_tls_min_protocol, kudu::security::SecurityDefaults::kDef
               "connections. May be one of 'TLSv1', 'TLSv1.1', or 'TLSv1.2'.");
 TAG_FLAG(webserver_tls_min_protocol, advanced);
 
+DEFINE_bool(webserver_require_spnego, false,
+            "Require connections to the web server to authenticate via Kerberos "
+            "using SPNEGO.");
+TAG_FLAG(webserver_require_spnego, evolving);
+
 namespace kudu {
 
 static bool ValidateTlsFlags() {
@@ -151,7 +156,8 @@ WebserverOptions::WebserverOptions()
     password_file(FLAGS_webserver_password_file),
     tls_ciphers(FLAGS_webserver_tls_ciphers),
     tls_min_protocol(FLAGS_webserver_tls_min_protocol),
-    num_worker_threads(FLAGS_webserver_num_worker_threads) {
+    num_worker_threads(FLAGS_webserver_num_worker_threads),
+    require_spnego(FLAGS_webserver_require_spnego) {
 }
 
 } // namespace kudu
diff --git a/src/kudu/server/webserver_options.h b/src/kudu/server/webserver_options.h
index b0d2df0..0058e82 100644
--- a/src/kudu/server/webserver_options.h
+++ b/src/kudu/server/webserver_options.h
@@ -17,8 +17,9 @@
 #ifndef KUDU_SERVER_WEBSERVER_OPTIONS_H
 #define KUDU_SERVER_WEBSERVER_OPTIONS_H
 
+#include <cstdint>
+#include <functional>
 #include <string>
-#include <stdint.h>
 
 namespace kudu {
 
@@ -41,6 +42,12 @@ struct WebserverOptions {
   std::string tls_ciphers;
   std::string tls_min_protocol;
   uint32_t num_worker_threads;
+
+  bool require_spnego;
+
+  // When a user has authenticated via SPNEGO, the SPN of that user will be passed to this
+  // callback. This is currently used only for testing.
+  std::function<void(const std::string&)> spnego_post_authn_callback;
 };
 
 } // namespace kudu
diff --git a/src/kudu/util/curl_util.cc b/src/kudu/util/curl_util.cc
index c29f781..21f02df 100644
--- a/src/kudu/util/curl_util.cc
+++ b/src/kudu/util/curl_util.cc
@@ -91,6 +91,20 @@ Status EasyCurl::DoRequest(const std::string& url,
         curl_, CURLOPT_SSL_VERIFYPEER, 0)));
   }
 
+  if (use_spnego_) {
+    RETURN_NOT_OK(TranslateError(curl_easy_setopt(
+        curl_, CURLOPT_HTTPAUTH, CURLAUTH_NEGOTIATE)));
+    // It's necessary to pass an empty user/password to trigger the authentication
+    // code paths in curl, even though SPNEGO doesn't use them.
+    RETURN_NOT_OK(TranslateError(curl_easy_setopt(
+        curl_, CURLOPT_USERPWD, ":")));
+  }
+
+  if (verbose_) {
+    RETURN_NOT_OK(TranslateError(curl_easy_setopt(
+        curl_, CURLOPT_VERBOSE, 1)));
+  }
+
   // Add headers if specified.
   struct curl_slist* curl_headers = nullptr;
   auto clean_up_curl_slist = MakeScopedCleanup([&]() {
diff --git a/src/kudu/util/curl_util.h b/src/kudu/util/curl_util.h
index cccd2db..6b4ea9e 100644
--- a/src/kudu/util/curl_util.h
+++ b/src/kudu/util/curl_util.h
@@ -67,6 +67,16 @@ class EasyCurl {
     timeout_ = t;
   }
 
+  void set_use_spnego(bool use_spnego) {
+    use_spnego_ = use_spnego;
+  }
+
+  // Enable verbose mode for curl. This dumps debugging output to stderr, so
+  // is only really useful in the context of tests.
+  void set_verbose(bool v) {
+    verbose_ = v;
+  }
+
  private:
   // Do a request. If 'post_data' is non-NULL, does a POST.
   // Otherwise, does a GET.
@@ -82,6 +92,10 @@ class EasyCurl {
   // Whether to return the HTTP headers with the response.
   bool return_headers_ = false;
 
+  bool use_spnego_ = false;
+
+  bool verbose_ = false;
+
   MonoDelta timeout_;
 
   DISALLOW_COPY_AND_ASSIGN(EasyCurl);
diff --git a/src/kudu/util/test_macros.h b/src/kudu/util/test_macros.h
index 63cae5a..c11bb23 100644
--- a/src/kudu/util/test_macros.h
+++ b/src/kudu/util/test_macros.h
@@ -63,6 +63,13 @@
 #define ASSERT_STR_NOT_CONTAINS(str, substr) \
   ASSERT_THAT(str, testing::Not(testing::HasSubstr(substr)))
 
+#define EXPECT_STR_CONTAINS(str, substr) \
+  EXPECT_THAT(str, testing::HasSubstr(substr))
+
+#define EXPECT_STR_NOT_CONTAINS(str, substr) \
+  EXPECT_THAT(str, testing::Not(testing::HasSubstr(substr)))
+
+
 // Substring regular expressions in extended regex (POSIX) syntax.
 #define ASSERT_STR_MATCHES(str, pattern) \
   ASSERT_THAT(str, testing::ContainsRegex(pattern))
diff --git a/thirdparty/build-definitions.sh b/thirdparty/build-definitions.sh
index 406603b..3aefe40 100644
--- a/thirdparty/build-definitions.sh
+++ b/thirdparty/build-definitions.sh
@@ -650,8 +650,8 @@ build_mustache() {
 }
 
 build_curl() {
-  # Configure for a very minimal install - basically only HTTP(S), since we only
-  # use this for testing our own HTTP/HTTPS endpoints at this point in time.
+  # Configure for a fairly minimal install - we only use this for testing
+  # so we just need HTTP/HTTPS with GSSAPI support (for SPNEGO testing).
   CURL_BDIR=$TP_BUILD_DIR/$CURL_NAME$MODE_SUFFIX
   mkdir -p $CURL_BDIR
   pushd $CURL_BDIR
@@ -679,7 +679,8 @@ build_curl() {
     --disable-telnet \
     --disable-tftp \
     --without-librtmp \
-    --without-libssh2
+    --without-libssh2 \
+    --with-gssapi
   make -j$PARALLEL $EXTRA_MAKEFLAGS install
   popd
 }