You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by da...@apache.org on 2017/02/03 19:13:07 UTC

kudu git commit: [security] add channel binding to krpc

Repository: kudu
Updated Branches:
  refs/heads/master 848180654 -> 6fec9dd52


[security] add channel binding to krpc

Channel binding prevents a MITM attack when using unauthenticated TLS
with Kerberos. The channel binding codepath is exercised by the existing
TLS + GSSAPI negotiation test, but I'm punting on testing that it
protects against a MITM for now.

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


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

Branch: refs/heads/master
Commit: 6fec9dd52ecfd0248a46c5c5d6987e2ac2e0a4cf
Parents: 8481806
Author: Dan Burkert <da...@apache.org>
Authored: Fri Jan 27 14:15:12 2017 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Fri Feb 3 19:12:54 2017 +0000

----------------------------------------------------------------------
 docs/design-docs/rpc.md                 | 53 ++++++++++++++++++
 src/kudu/rpc/client_negotiation.cc      | 71 ++++++++++++++++++++++++
 src/kudu/rpc/client_negotiation.h       |  7 +++
 src/kudu/rpc/rpc_header.proto           | 20 ++++---
 src/kudu/rpc/sasl_common.cc             | 20 ++++++-
 src/kudu/rpc/sasl_common.h              | 11 ++--
 src/kudu/rpc/server_negotiation.cc      | 59 +++++++++++++++++---
 src/kudu/rpc/server_negotiation.h       |  8 ++-
 src/kudu/security/ca/cert_management.cc |  9 ----
 src/kudu/security/cert.cc               | 80 ++++++++++++++++++++++------
 src/kudu/security/cert.h                | 12 +++--
 src/kudu/security/openssl_util.cc       |  2 -
 src/kudu/security/openssl_util.h        | 29 ++++++++--
 src/kudu/security/tls_handshake.cc      |  5 --
 src/kudu/security/tls_socket.cc         | 26 +++++++++
 src/kudu/security/tls_socket.h          | 20 +++++--
 src/kudu/util/status.cc                 |  2 +-
 17 files changed, 369 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/docs/design-docs/rpc.md
----------------------------------------------------------------------
diff --git a/docs/design-docs/rpc.md b/docs/design-docs/rpc.md
index 8b4fdeb..6c2c598 100644
--- a/docs/design-docs/rpc.md
+++ b/docs/design-docs/rpc.md
@@ -487,6 +487,59 @@ Client                                                                    Server
    |                                                +---------------------+ |
 ```
 
+### Authentication
+
+When security is enabled, the negotiation process is responsible for mutually
+authenticating the client and server to each other. There are three distinct
+methods of mutual authentication, one of which will be chosen during
+negotiation. Which method is chosen for a particular connection depends on the
+configuration of the client and server. All authentication methods require the
+connection to be protected with TLS encryption.
+
+| server authentication of client | client authentication of server | notes |
+|---|---|---|
+| Kerberos | Kerberos | Kerberos provides strong mutual authentication, channel binding ties the Kerberos authentication to the TLS channel |
+| Certificate | Certificate | client and server authenticate via certs signed by mutually trusted CA |
+| Token | Certificate | client authenticates by passing a token signed by a key which is trusted by the server |
+
+#### Kerberos Authentication
+
+Kerberos authentication requires the client and server to be configured with
+Kerberos principal credentials. Typically the client must have an active TGT
+acquired from the `kinit` command, and the server must be configured with a
+keytab. Kerberos authentication is handled through the SASL `GSSAPI` mechanism.
+When using Kerberos authentication TLS certificates are not verified.
+
+When the SASL `GSSAPI` negotiation is complete, the server returns a special
+channel binding token to the client as part of the `SASL_SUCCESS` message. The
+channel binding token contains a hash of the server's certificate, wrapped in a
+SASL integrity protected envelope. The client must check the channel binding
+token against the certificate presented by the server during the TLS handshake
+before continuing to use the connection. See RFC 5056 for more information on
+channel binding and why it is necessary, and RFC 5929 for a description of the
+specific 'tls-server-end-point' channel binding type used.
+
+#### Certificate Authentication
+
+When the client and server are configured with certificates signed by a mutually
+trusted certificate authority (CA), certificate authentication can be used
+to authenticate the client and server.
+
+TODO(dan): explain how the two sides decide on certificate authentication.
+           Probably via a special SASL mechanism (`KUDU_CERTIFICATE`) which
+           short-circuits any actual SASL messages.
+
+#### Token Authentication
+
+Token authentication requires the server to be configured with a certificate,
+and the client to be configured with a token. The server's certificate must be
+signed by a CA trusted by the client, and the client's token must be signed by a
+token-signing-key which is trusted by the server.
+
+TODO(dan): explain how the two sides decide on token authentication.
+           Probably via a special SASL mechanism (`KUDU_TOKEN`) which sends a
+           single round of SASL messages containing the token and a reply ack.
+
 ## Connection Context
 
 Once the SASL negotiation is complete, before the first request, the client

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/rpc/client_negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/client_negotiation.cc b/src/kudu/rpc/client_negotiation.cc
index 6796d60..3d32e14 100644
--- a/src/kudu/rpc/client_negotiation.cc
+++ b/src/kudu/rpc/client_negotiation.cc
@@ -19,6 +19,7 @@
 
 #include <string.h>
 
+#include <algorithm>
 #include <map>
 #include <memory>
 #include <set>
@@ -27,6 +28,7 @@
 #include <glog/logging.h>
 #include <sasl/sasl.h>
 
+#include "kudu/gutil/casts.h"
 #include "kudu/gutil/endian.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/stl_util.h"
@@ -39,8 +41,10 @@
 #include "kudu/rpc/sasl_common.h"
 #include "kudu/rpc/sasl_helper.h"
 #include "kudu/rpc/serialization.h"
+#include "kudu/security/cert.h"
 #include "kudu/security/tls_context.h"
 #include "kudu/security/tls_handshake.h"
+#include "kudu/security/tls_socket.h"
 #include "kudu/util/faststring.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/net/socket.h"
@@ -100,6 +104,7 @@ ClientNegotiation::ClientNegotiation(unique_ptr<Socket> socket)
     : socket_(std::move(socket)),
       helper_(SaslHelper::CLIENT),
       tls_context_(nullptr),
+      tls_negotiated_(false),
       negotiated_mech_(SaslMechanism::INVALID),
       deadline_(MonoTime::Max()) {
   callbacks_.push_back(SaslBuildCallback(SASL_CB_GETOPT,
@@ -191,6 +196,7 @@ Status ClientNegotiation::Negotiate() {
       s = HandleTlsHandshake(response);
     }
     RETURN_NOT_OK(s);
+    tls_negotiated_ = true;
   }
 
   // Step 4: SASL negotiation.
@@ -207,6 +213,7 @@ Status ClientNegotiation::Negotiate() {
         break;
       // SASL_SUCCESS: Server has accepted our authentication request. Negotiation successful.
       case NegotiatePB::SASL_SUCCESS:
+        RETURN_NOT_OK(HandleSaslSuccess(response));
         cont = false;
         break;
       default:
@@ -451,6 +458,13 @@ Status ClientNegotiation::SendSaslInitiate() {
   // Check that the SASL library is using the mechanism that we picked.
   DCHECK_EQ(SaslMechanism::value_of(negotiated_mech), negotiated_mech_);
 
+  // If we are speaking TLS and the negotiated mechanism is GSSAPI (Kerberos),
+  // configure SASL to use integrity protection so that the channel bindings
+  // can be verified.
+  if (tls_negotiated_ && negotiated_mech_ == SaslMechanism::GSSAPI) {
+    RETURN_NOT_OK(EnableIntegrityProtection(sasl_conn_.get()));
+  }
+
   NegotiatePB msg;
   msg.set_step(NegotiatePB::SASL_INITIATE);
   msg.mutable_token()->assign(init_msg, init_msg_len);
@@ -481,6 +495,42 @@ Status ClientNegotiation::HandleSaslChallenge(const NegotiatePB& response) {
   return SendSaslResponse(out, out_len);
 }
 
+Status ClientNegotiation::HandleSaslSuccess(const NegotiatePB& response) {
+  TRACE("Received SASL_SUCCESS response from server");
+
+  if (tls_negotiated_ && negotiated_mech_ == SaslMechanism::Type::GSSAPI) {
+    // Check the channel bindings provided by the server against the expected
+    // channel bindings.
+    security::TlsSocket* tls_socket = down_cast<security::TlsSocket*>(socket_.get());
+    security::Cert cert;
+    RETURN_NOT_OK(tls_socket->GetRemoteCert(&cert));
+
+    string expected_channel_bindings;
+    RETURN_NOT_OK_PREPEND(cert.GetServerEndPointChannelBindings(&expected_channel_bindings),
+                          "failed to generate expected channel bindings");
+
+    if (!response.has_channel_bindings()) {
+      return Status::NotAuthorized("no channel bindings provided by server");
+    }
+
+    string recieved_channel_bindings;
+    RETURN_NOT_OK_PREPEND(SaslDecode(response.channel_bindings(), &recieved_channel_bindings),
+                          "failed to decode channel bindings");
+
+    if (expected_channel_bindings != recieved_channel_bindings) {
+      Sockaddr addr;
+      ignore_result(socket_->GetPeerAddress(&addr));
+
+      LOG(WARNING) << "Recieved unexpected channel bindings from server "
+                   << addr.ToString()
+                   << ", this could indicate an active network man-in-the-middle";
+      return Status::NotAuthorized("channel bindings do not match");
+    }
+  }
+
+  return Status::OK();
+}
+
 Status ClientNegotiation::DoSaslStep(const string& in, const char** out, unsigned* out_len) {
   TRACE("Calling sasl_client_step()");
 
@@ -489,6 +539,27 @@ Status ClientNegotiation::DoSaslStep(const string& in, const char** out, unsigne
   });
 }
 
+Status ClientNegotiation::SaslDecode(const string& encoded, string* plaintext) {
+  size_t offset = 0;
+  const char* out;
+  unsigned out_len;
+
+  // The SASL library can only decode up to a maximum amount at a time, so we
+  // have to call decode multiple times if our input is larger than this max.
+  while (offset < encoded.size()) {
+    size_t len = std::min(kSaslMaxOutBufLen, encoded.size() - offset);
+
+    RETURN_NOT_OK(WrapSaslCall(sasl_conn_.get(), [&]() {
+        return sasl_decode(sasl_conn_.get(), encoded.data() + offset, len, &out, &out_len);
+    }));
+
+    plaintext->append(out, out_len);
+    offset += len;
+  }
+
+  return Status::OK();
+}
+
 Status ClientNegotiation::SendConnectionContext() {
   TRACE("Sending connection context");
   RequestHeader header;

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/rpc/client_negotiation.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/client_negotiation.h b/src/kudu/rpc/client_negotiation.h
index 5563ff6..0e14ee9 100644
--- a/src/kudu/rpc/client_negotiation.h
+++ b/src/kudu/rpc/client_negotiation.h
@@ -164,6 +164,9 @@ class ClientNegotiation {
   // Handle case when server sends SASL_CHALLENGE response.
   Status HandleSaslChallenge(const NegotiatePB& response) WARN_UNUSED_RESULT;
 
+  // Handle case when server sends SASL_SUCCESS response.
+  Status HandleSaslSuccess(const NegotiatePB& response) WARN_UNUSED_RESULT;
+
   // Perform a client-side step of the SASL negotiation.
   // Input is what came from the server. Output is what we will send back to the server.
   // Returns:
@@ -172,6 +175,9 @@ class ClientNegotiation {
   // otherwise returns an appropriate error status.
   Status DoSaslStep(const std::string& in, const char** out, unsigned* out_len) WARN_UNUSED_RESULT;
 
+  // Decode the provided SASL-encoded data and append it to 'plaintext'.
+  Status SaslDecode(const std::string& encoded, std::string* plaintext) WARN_UNUSED_RESULT;
+
   Status SendConnectionContext() WARN_UNUSED_RESULT;
 
   // The socket to the remote server.
@@ -185,6 +191,7 @@ class ClientNegotiation {
   // TLS state.
   const security::TlsContext* tls_context_;
   security::TlsHandshake tls_handshake_;
+  bool tls_negotiated_;
 
   // Authentication state.
   std::string plain_auth_user_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/rpc/rpc_header.proto
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc_header.proto b/src/kudu/rpc/rpc_header.proto
index 3683a23..5bc628d 100644
--- a/src/kudu/rpc/rpc_header.proto
+++ b/src/kudu/rpc/rpc_header.proto
@@ -37,13 +37,11 @@ message UserInformationPB {
   required string real_user = 2;
 }
 
-/**
- * The connection context is sent as part of the connection establishment.
- * It establishes the context for ALL RPC calls within the connection.
- * This is sent on connection setup after the connection preamble is sent
- * and SASL has been negotiated.
- * No response is sent from the server to the client.
- */
+// The connection context is sent as part of the connection establishment.
+// It establishes the context for ALL RPC calls within the connection.
+// This is sent on connection setup after the connection preamble is sent
+// and SASL has been negotiated.
+// No response is sent from the server to the client.
 message ConnectionContextPB {
   // UserInfo beyond what is determined as part of security handshake
   // at connection time (kerberos, tokens etc).
@@ -119,6 +117,14 @@ message NegotiatePB {
   // During the TLS_HANDSHAKE step, contains the TLS handshake message.
   optional bytes tls_handshake = 5 [(REDACT) = true];
 
+  // The tls-server-end-point channel bindings as specified in RFC 5929.  Sent
+  // from the server to the client during the SASL_SUCCESS step when the
+  // Kerberos (GSSAPI) SASL mechanism is used with TLS, in order to bind the
+  // Kerberos authenticated channel to the TLS channel. The value is integrity
+  // protected through SASL. The client is responsible for validating that the
+  // value matches the expected value.
+  optional bytes channel_bindings = 6 [(REDACT) = true];
+
   // During the NEGOTIATE step, contains the supported SASL mechanisms.
   // During the SASL_INITIATE step, contains the single chosen SASL mechanism.
   repeated SaslAuth auths      = 4;

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/rpc/sasl_common.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_common.cc b/src/kudu/rpc/sasl_common.cc
index 8027721..6b60839 100644
--- a/src/kudu/rpc/sasl_common.cc
+++ b/src/kudu/rpc/sasl_common.cc
@@ -17,10 +17,13 @@
 
 #include "kudu/rpc/sasl_common.h"
 
-#include <boost/algorithm/string/predicate.hpp>
+#include <string.h>
+
+#include <limits>
 #include <mutex>
 #include <string>
 
+#include <boost/algorithm/string/predicate.hpp>
 #include <gflags/gflags.h>
 #include <glog/logging.h>
 #include <regex.h>
@@ -42,6 +45,7 @@ namespace rpc {
 
 const char* const kSaslMechPlain = "PLAIN";
 const char* const kSaslMechGSSAPI = "GSSAPI";
+extern const size_t kSaslMaxOutBufLen = 1024;
 
 // See WrapSaslCall().
 static __thread string* g_auth_failure_capture = nullptr;
@@ -318,6 +322,7 @@ Status WrapSaslCall(sasl_conn_t* conn, const std::function<int()>& call) {
       return Status::Incomplete("");
     case SASL_FAIL:      // Generic failure (encompasses missing krb5 credentials).
     case SASL_BADAUTH:   // Authentication failure.
+    case SASL_BADMAC:    // Decode failure.
     case SASL_NOAUTHZ:   // Authorization failure.
     case SASL_NOUSER:    // User not found.
     case SASL_WRONGMECH: // Server doesn't support requested mechanism.
@@ -366,6 +371,19 @@ sasl_callback_t SaslBuildCallback(int id, int (*proc)(void), void* context) {
   return callback;
 }
 
+Status EnableIntegrityProtection(sasl_conn_t* sasl_conn) {
+  sasl_security_properties_t sec_props;
+  memset(&sec_props, 0, sizeof(sec_props));
+  sec_props.min_ssf = 1;
+  sec_props.max_ssf = std::numeric_limits<sasl_ssf_t>::max();
+  sec_props.maxbufsize = kSaslMaxOutBufLen;
+
+  RETURN_NOT_OK_PREPEND(WrapSaslCall(sasl_conn, [&] () {
+    return sasl_setprop(sasl_conn, SASL_SEC_PROPS, &sec_props);
+  }), "failed to set SASL security properties");
+  return Status::OK();
+}
+
 SaslMechanism::Type SaslMechanism::value_of(const string& mech) {
   if (boost::iequals(mech, "PLAIN")) {
     return PLAIN;

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/rpc/sasl_common.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_common.h b/src/kudu/rpc/sasl_common.h
index d56f7ec..3d7e07f 100644
--- a/src/kudu/rpc/sasl_common.h
+++ b/src/kudu/rpc/sasl_common.h
@@ -38,6 +38,7 @@ using std::string;
 // Constants
 extern const char* const kSaslMechPlain;
 extern const char* const kSaslMechGSSAPI;
+extern const size_t kSaslMaxOutBufLen;
 
 struct SaslMechanism {
   enum Type {
@@ -63,10 +64,10 @@ struct SaslMechanism {
 //
 // This function is thread safe and uses a static lock.
 // This function should NOT be called during static initialization.
-Status SaslInit();
+Status SaslInit() WARN_UNUSED_RESULT;
 
 // Disable Kudu's initialization of SASL. See equivalent method in client.h.
-Status DisableSaslInitialization();
+Status DisableSaslInitialization() WARN_UNUSED_RESULT;
 
 // Wrap a call into the SASL library. 'call' should be a lambda which
 // returns a SASL error code.
@@ -79,7 +80,7 @@ Status DisableSaslInitialization();
 //
 // The Status message is beautified to be more user-friendly compared
 // to the underlying sasl_errdetails() call.
-Status WrapSaslCall(sasl_conn_t* conn, const std::function<int()>& call);
+Status WrapSaslCall(sasl_conn_t* conn, const std::function<int()>& call) WARN_UNUSED_RESULT;
 
 // Return <ip>;<port> string formatted for SASL library use.
 string SaslIpPortString(const Sockaddr& addr);
@@ -93,6 +94,10 @@ std::set<SaslMechanism::Type> SaslListAvailableMechs();
 // context: An object to pass to the callback as the context pointer, or NULL.
 sasl_callback_t SaslBuildCallback(int id, int (*proc)(void), void* context);
 
+// Require integrity protection on the SASL connection. Should be called before
+// initiating the SASL negotiation.
+Status EnableIntegrityProtection(sasl_conn_t* sasl_conn) WARN_UNUSED_RESULT;
+
 // Deleter for sasl_conn_t instances, for use with gscoped_ptr after calling sasl_*_new()
 struct SaslDeleter {
   inline void operator()(sasl_conn_t* conn) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/rpc/server_negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/server_negotiation.cc b/src/kudu/rpc/server_negotiation.cc
index b1f2893..6f5c118 100644
--- a/src/kudu/rpc/server_negotiation.cc
+++ b/src/kudu/rpc/server_negotiation.cc
@@ -17,6 +17,7 @@
 
 #include "kudu/rpc/server_negotiation.h"
 
+#include <algorithm>
 #include <limits>
 #include <memory>
 #include <set>
@@ -26,14 +27,17 @@
 #include <google/protobuf/message_lite.h>
 #include <sasl/sasl.h>
 
+#include "kudu/gutil/casts.h"
 #include "kudu/gutil/endian.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/rpc/blocking_ops.h"
 #include "kudu/rpc/constants.h"
 #include "kudu/rpc/serialization.h"
+#include "kudu/security/cert.h"
 #include "kudu/security/tls_context.h"
 #include "kudu/security/tls_handshake.h"
+#include "kudu/security/tls_socket.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/net/socket.h"
 #include "kudu/util/scoped_cleanup.h"
@@ -67,6 +71,7 @@ ServerNegotiation::ServerNegotiation(unique_ptr<Socket> socket)
     : socket_(std::move(socket)),
       helper_(SaslHelper::SERVER),
       tls_context_(nullptr),
+      tls_negotiated_(false),
       negotiated_mech_(SaslMechanism::INVALID),
       deadline_(MonoTime::Max()) {
   callbacks_.push_back(SaslBuildCallback(SASL_CB_GETOPT,
@@ -148,6 +153,7 @@ Status ServerNegotiation::Negotiate() {
       if (s.ok()) break;
       if (!s.IsIncomplete()) return s;
     }
+    tls_negotiated_ = true;
   }
 
   // Step 4: SASL negotiation.
@@ -426,6 +432,15 @@ Status ServerNegotiation::HandleSaslInitiate(const NegotiatePB& request) {
   const NegotiatePB::SaslAuth& auth = request.auths(0);
   TRACE("Client requested to use mechanism: $0", auth.mechanism());
 
+  negotiated_mech_ = SaslMechanism::value_of(auth.mechanism());
+
+  // If we are speaking TLS and the negotiated mechanism is GSSAPI (Kerberos),
+  // configure SASL to use integrity protection so that the channel bindings
+  // can be verified.
+  if (tls_negotiated_ && negotiated_mech_ == SaslMechanism::GSSAPI) {
+    RETURN_NOT_OK(EnableIntegrityProtection(sasl_conn_.get()));
+  }
+
   const char* server_out = nullptr;
   uint32_t server_out_len = 0;
   TRACE("Calling sasl_server_start()");
@@ -445,17 +460,37 @@ Status ServerNegotiation::HandleSaslInitiate(const NegotiatePB& request) {
     return s;
   }
 
-  negotiated_mech_ = SaslMechanism::value_of(auth.mechanism());
-
   // We have a valid mechanism match
   if (s.ok()) {
-    RETURN_NOT_OK(SendSaslSuccess(server_out, server_out_len));
+    DCHECK(server_out_len == 0);
+    RETURN_NOT_OK(SendSaslSuccess());
   } else { // s.IsIncomplete() (equivalent to SASL_CONTINUE)
     RETURN_NOT_OK(SendSaslChallenge(server_out, server_out_len));
   }
   return s;
 }
 
+Status ServerNegotiation::SaslEncode(const std::string& plaintext, std::string* encoded) {
+  size_t offset = 0;
+  const char* out;
+  unsigned out_len;
+
+  // The SASL library can only encode up to a maximum amount at a time, so we
+  // have to call encode multiple times if our input is larger than this max.
+  while (offset < plaintext.size()) {
+    size_t len = std::min(kSaslMaxOutBufLen, plaintext.size() - offset);
+
+    RETURN_NOT_OK(WrapSaslCall(sasl_conn_.get(), [&]() {
+        return sasl_encode(sasl_conn_.get(), plaintext.data() + offset, len, &out, &out_len);
+    }));
+
+    encoded->append(out, out_len);
+    offset += len;
+  }
+
+  return Status::OK();
+}
+
 Status ServerNegotiation::HandleSaslResponse(const NegotiatePB& request) {
   if (PREDICT_FALSE(request.step() != NegotiatePB::SASL_RESPONSE)) {
     Status s =  Status::NotAuthorized("expected SASL_RESPONSE step",
@@ -484,7 +519,8 @@ Status ServerNegotiation::HandleSaslResponse(const NegotiatePB& request) {
     });
 
   if (s.ok()) {
-    return SendSaslSuccess(server_out, server_out_len);
+    DCHECK(server_out_len == 0);
+    return SendSaslSuccess();
   }
   if (s.IsIncomplete()) {
     return SendSaslChallenge(server_out, server_out_len);
@@ -501,12 +537,21 @@ Status ServerNegotiation::SendSaslChallenge(const char* challenge, unsigned clen
   return Status::Incomplete("");
 }
 
-Status ServerNegotiation::SendSaslSuccess(const char* token, unsigned tlen) {
+Status ServerNegotiation::SendSaslSuccess() {
   NegotiatePB response;
   response.set_step(NegotiatePB::SASL_SUCCESS);
-  if (PREDICT_FALSE(tlen > 0)) {
-    response.mutable_token()->assign(token, tlen);
+
+  if (tls_negotiated_ && negotiated_mech_ == SaslMechanism::Type::GSSAPI) {
+    // Send the channel bindings to the client.
+    security::TlsSocket* tls_socket = down_cast<security::TlsSocket*>(socket_.get());
+    security::Cert cert;
+    RETURN_NOT_OK(tls_socket->GetLocalCert(&cert));
+
+    string plaintext_channel_bindings;
+    RETURN_NOT_OK(cert.GetServerEndPointChannelBindings(&plaintext_channel_bindings));
+    RETURN_NOT_OK(SaslEncode(plaintext_channel_bindings, response.mutable_channel_bindings()));
   }
+
   RETURN_NOT_OK(SendNegotiatePB(response));
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/rpc/server_negotiation.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/server_negotiation.h b/src/kudu/rpc/server_negotiation.h
index dca3fa0..88c24d9 100644
--- a/src/kudu/rpc/server_negotiation.h
+++ b/src/kudu/rpc/server_negotiation.h
@@ -172,8 +172,11 @@ class ServerNegotiation {
   // Send a SASL_CHALLENGE response to the client with a challenge token.
   Status SendSaslChallenge(const char* challenge, unsigned clen) WARN_UNUSED_RESULT;
 
-  // Send a SASL_SUCCESS response to the client with an token (typically empty).
-  Status SendSaslSuccess(const char* token, unsigned tlen) WARN_UNUSED_RESULT;
+  // Send a SASL_SUCCESS response to the client.
+  Status SendSaslSuccess() WARN_UNUSED_RESULT;
+
+  // Encode the provided data and append it to 'encoded'.
+  Status SaslEncode(const std::string& plaintext, std::string* encoded) WARN_UNUSED_RESULT;
 
   // Receive and validate the ConnectionContextPB.
   Status RecvConnectionContext(faststring* recv_buf) WARN_UNUSED_RESULT;
@@ -189,6 +192,7 @@ class ServerNegotiation {
   // TLS state.
   const security::TlsContext* tls_context_;
   security::TlsHandshake tls_handshake_;
+  bool tls_negotiated_;
 
   // The set of features supported by the client. Filled in during negotiation.
   std::set<RpcFeatureFlag> client_features_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/security/ca/cert_management.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/ca/cert_management.cc b/src/kudu/security/ca/cert_management.cc
index a871b75..9f0dde2 100644
--- a/src/kudu/security/ca/cert_management.cc
+++ b/src/kudu/security/ca/cert_management.cc
@@ -56,15 +56,6 @@ template<> struct SslTypeTraits<ASN1_INTEGER> {
 template<> struct SslTypeTraits<BIGNUM> {
   static constexpr auto free = &BN_free;
 };
-template<> struct SslTypeTraits<X509> {
-  static constexpr auto free = &X509_free;
-};
-template<> struct SslTypeTraits<X509_REQ> {
-  static constexpr auto free = &X509_REQ_free;
-};
-template<> struct SslTypeTraits<X509_EXTENSION> {
-  static constexpr auto free = &X509_EXTENSION_free;
-};
 template<> struct SslTypeTraits<EVP_PKEY> {
   static constexpr auto free = &EVP_PKEY_free;
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/security/cert.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/cert.cc b/src/kudu/security/cert.cc
index f04577f..fa373da 100644
--- a/src/kudu/security/cert.cc
+++ b/src/kudu/security/cert.cc
@@ -19,6 +19,7 @@
 
 #include <string>
 
+#include <openssl/evp.h>
 #include <openssl/pem.h>
 #include <openssl/x509.h>
 
@@ -31,21 +32,6 @@ using std::string;
 namespace kudu {
 namespace security {
 
-template<> struct SslTypeTraits<X509> {
-  static constexpr auto free = &X509_free;
-  static constexpr auto read_pem = &PEM_read_bio_X509;
-  static constexpr auto read_der = &d2i_X509_bio;
-  static constexpr auto write_pem = &PEM_write_bio_X509;
-  static constexpr auto write_der = &i2d_X509_bio;
-};
-template<> struct SslTypeTraits<X509_REQ> {
-  static constexpr auto free = &X509_REQ_free;
-  static constexpr auto read_pem = &PEM_read_bio_X509_REQ;
-  static constexpr auto read_der = &d2i_X509_REQ_bio;
-  static constexpr auto write_pem = &PEM_write_bio_X509_REQ;
-  static constexpr auto write_der = &i2d_X509_REQ_bio;
-};
-
 string X509NameToString(X509_NAME* name) {
   CHECK(name);
   auto bio = ssl_make_unique(BIO_new(BIO_s_mem()));
@@ -76,6 +62,70 @@ string Cert::IssuerName() const {
   return X509NameToString(X509_get_issuer_name(data_.get()));
 }
 
+Status Cert::GetServerEndPointChannelBindings(string* channel_bindings) const {
+  // Find the signature type of the certificate. This corresponds to the digest
+  // (hash) algorithm, and the public key type which signed the cert.
+
+#if OPENSSL_VERSION_NUMBER >= 0x10002000L
+  int signature_nid = X509_get_signature_nid(data_.get());
+#else
+  // Older version of OpenSSL appear not to have a public way to get the
+  // signature digest method from a certificate. Instead, we reach into the
+  // 'private' internals.
+  int signature_nid = OBJ_obj2nid(data_->sig_alg->algorithm);
+#endif
+
+  // Retrieve the digest algorithm type.
+  int digest_nid;
+  OBJ_find_sigid_algs(signature_nid, &digest_nid, nullptr /* public_key_nid */);
+
+  // RFC 5929: if the certificate's signatureAlgorithm uses no hash functions or
+  // uses multiple hash functions, then this channel binding type's channel
+  // bindings are undefined at this time (updates to is channel binding type may
+  // occur to address this issue if it ever arises).
+  //
+  // TODO(dan): can the multiple hash function scenario actually happen? What
+  // does OBJ_find_sigid_algs do in that scenario?
+  if (digest_nid == NID_undef) {
+    return Status::NotSupported("server certificate has no signature digest (hash) algorithm");
+  }
+
+  // RFC 5929: if the certificate's signatureAlgorithm uses a single hash
+  // function, and that hash function is either MD5 [RFC1321] or SHA-1
+  // [RFC3174], then use SHA-256 [FIPS-180-3];
+  if (digest_nid == NID_md5 || digest_nid == NID_sha1) {
+    digest_nid = NID_sha256;
+  }
+
+  const EVP_MD* md = EVP_get_digestbynid(digest_nid);
+  OPENSSL_RET_IF_NULL(md, "digest for nid not found");
+
+  // Create a digest BIO. All data written to the BIO will be sent through the
+  // digest (hash) function. The digest BIO requires a null BIO to writethrough to.
+  auto null_bio = ssl_make_unique(BIO_new(BIO_s_null()));
+  auto md_bio = ssl_make_unique(BIO_new(BIO_f_md()));
+  OPENSSL_RET_NOT_OK(BIO_set_md(md_bio.get(), md), "failed to set digest for BIO");
+  BIO_push(md_bio.get(), null_bio.get());
+
+  // Write the cert to the digest BIO.
+  RETURN_NOT_OK(ToBIO(md_bio.get(), DataFormat::DER, data_.get()));
+
+  // Read the digest from the BIO and append it to 'channel_bindings'.
+  char buf[EVP_MAX_MD_SIZE];
+  int digest_len = BIO_gets(md_bio.get(), buf, sizeof(buf));
+  OPENSSL_RET_NOT_OK(digest_len, "failed to get cert digest from BIO");
+  channel_bindings->assign(buf, digest_len);
+  return Status::OK();
+}
+
+void Cert::AdoptAndAddRefRawData(X509* data) {
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
+  CHECK_GT(CRYPTO_add(&data->references, 1, CRYPTO_LOCK_X509), 1) << "X509 use-after-free detected";
+#else
+  OPENSSL_CHECK_OK(X509_up_ref(data)) << "X509 use-after-free detected: " << GetOpenSSLErrors();
+#endif
+  AdoptRawData(data);
+}
 
 Status CertSignRequest::FromString(const std::string& data, DataFormat format) {
   return ::kudu::security::FromString(data, format, &data_);

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/security/cert.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/cert.h b/src/kudu/security/cert.h
index c21ae0b..e43127f 100644
--- a/src/kudu/security/cert.h
+++ b/src/kudu/security/cert.h
@@ -21,11 +21,6 @@
 
 #include "kudu/security/openssl_util.h"
 
-// Forward declarations for the OpenSSL typedefs.
-typedef struct x509_st X509;
-typedef struct X509_req_st X509_REQ;
-typedef struct X509_name_st X509_NAME;
-
 namespace kudu {
 
 class Status;
@@ -43,6 +38,13 @@ class Cert : public RawDataWrapper<X509> {
 
   std::string SubjectName() const;
   std::string IssuerName() const;
+
+  // Returns the 'tls-server-end-point' channel bindings for the certificate as
+  // specified in RFC 5929.
+  Status GetServerEndPointChannelBindings(std::string* channel_bindings) const;
+
+  // Adopts the provided X509 certificate, and increments the reference count.
+  void AdoptAndAddRefRawData(X509* data);
 };
 
 class CertSignRequest : public RawDataWrapper<X509_REQ> {

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/security/openssl_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/openssl_util.cc b/src/kudu/security/openssl_util.cc
index 9f552f8..ea4a259 100644
--- a/src/kudu/security/openssl_util.cc
+++ b/src/kudu/security/openssl_util.cc
@@ -28,7 +28,6 @@
 #include <openssl/rand.h>
 #include <openssl/ssl.h>
 
-#include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/debug/leakcheck_disabler.h"
 #include "kudu/util/errno.h"
 #include "kudu/util/mutex.h"
@@ -37,7 +36,6 @@
 
 using std::ostringstream;
 using std::string;
-using strings::Substitute;
 
 namespace kudu {
 namespace security {

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/security/openssl_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/openssl_util.h b/src/kudu/security/openssl_util.h
index 1691727..35ff8d6 100644
--- a/src/kudu/security/openssl_util.h
+++ b/src/kudu/security/openssl_util.h
@@ -21,28 +21,31 @@
 #include <memory>
 #include <string>
 
+#include <openssl/pem.h>
+#include <openssl/x509.h>
+
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/status.h"
 
 // Forward declarations for the OpenSSL typedefs.
+typedef struct X509_req_st X509_REQ;
 typedef struct bio_st BIO;
 typedef struct evp_pkey_st EVP_PKEY;
 typedef struct ssl_ctx_st SSL_CTX;
 typedef struct ssl_st SSL;
+typedef struct x509_st X509;
 
 #define OPENSSL_CHECK_OK(call) \
   CHECK_GT((call), 0)
 
 #define OPENSSL_RET_NOT_OK(call, msg) \
   if ((call) <= 0) { \
-    return Status::RuntimeError(::strings::Substitute("$0: $1", \
-        (msg), GetOpenSSLErrors())); \
+    return Status::RuntimeError((msg), GetOpenSSLErrors()); \
   }
 
 #define OPENSSL_RET_IF_NULL(call, msg) \
   if ((call) == nullptr) { \
-    return Status::RuntimeError(::strings::Substitute("$0: $1", \
-        (msg), GetOpenSSLErrors())); \
+    return Status::RuntimeError((msg), GetOpenSSLErrors()); \
   }
 
 namespace kudu {
@@ -80,6 +83,24 @@ using c_unique_ptr = std::unique_ptr<T, std::function<void(T*)>>;
 template<typename SSL_TYPE>
 struct SslTypeTraits {};
 
+template<> struct SslTypeTraits<X509> {
+  static constexpr auto free = &X509_free;
+  static constexpr auto read_pem = &PEM_read_bio_X509;
+  static constexpr auto read_der = &d2i_X509_bio;
+  static constexpr auto write_pem = &PEM_write_bio_X509;
+  static constexpr auto write_der = &i2d_X509_bio;
+};
+template<> struct SslTypeTraits<X509_EXTENSION> {
+  static constexpr auto free = &X509_EXTENSION_free;
+};
+template<> struct SslTypeTraits<X509_REQ> {
+  static constexpr auto free = &X509_REQ_free;
+  static constexpr auto read_pem = &PEM_read_bio_X509_REQ;
+  static constexpr auto read_der = &d2i_X509_REQ_bio;
+  static constexpr auto write_pem = &PEM_write_bio_X509_REQ;
+  static constexpr auto write_der = &i2d_X509_REQ_bio;
+};
+
 template<typename SSL_TYPE, typename Traits = SslTypeTraits<SSL_TYPE>>
 c_unique_ptr<SSL_TYPE> ssl_make_unique(SSL_TYPE* d) {
   return {d, Traits::free};

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/security/tls_handshake.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_handshake.cc b/src/kudu/security/tls_handshake.cc
index 49f0252..522fee6 100644
--- a/src/kudu/security/tls_handshake.cc
+++ b/src/kudu/security/tls_handshake.cc
@@ -40,11 +40,6 @@ using std::unique_ptr;
 namespace kudu {
 namespace security {
 
-template<> struct SslTypeTraits<X509> {
-  static constexpr auto free = &X509_free;
-};
-
-
 void TlsHandshake::SetSSLVerify() {
   CHECK(ssl_);
   CHECK(!has_started_);

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/security/tls_socket.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_socket.cc b/src/kudu/security/tls_socket.cc
index 1eec3dd..27a6af6 100644
--- a/src/kudu/security/tls_socket.cc
+++ b/src/kudu/security/tls_socket.cc
@@ -19,8 +19,10 @@
 
 #include <openssl/err.h>
 #include <openssl/ssl.h>
+#include <openssl/x509.h>
 
 #include "kudu/gutil/basictypes.h"
+#include "kudu/security/cert.h"
 #include "kudu/security/openssl_util.h"
 
 namespace kudu {
@@ -35,6 +37,30 @@ TlsSocket::~TlsSocket() {
   ignore_result(Close());
 }
 
+Status TlsSocket::GetLocalCert(Cert* cert) const {
+  X509* raw_cert = SSL_get_certificate(ssl_.get());
+
+  // This can happen if the cert has not been set (e.g. this is a client->server
+  // socket with no cert), or a non-PKI cipher is being used.
+  OPENSSL_RET_IF_NULL(raw_cert, "TLS socket has no local certificate");
+
+  // For whatever reason, SSL_get_certificate (unlike SSL_get_peer_certificate)
+  // does not increment the X509's reference count.
+  cert->AdoptAndAddRefRawData(raw_cert);
+  return Status::OK();
+}
+
+Status TlsSocket::GetRemoteCert(Cert* cert) const {
+  X509* raw_cert = SSL_get_peer_certificate(ssl_.get());
+
+  // This can happen if the cert has not been set (e.g. this is a server->client
+  // socket with no verification), or a non-PKI cipher is being used.
+  OPENSSL_RET_IF_NULL(raw_cert, "TLS socket has no remote certificate");
+
+  cert->AdoptRawData(raw_cert);
+  return Status::OK();
+}
+
 Status TlsSocket::Write(const uint8_t *buf, int32_t amt, int32_t *nwritten) {
   CHECK(ssl_);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/security/tls_socket.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_socket.h b/src/kudu/security/tls_socket.h
index 70da1e2..1d0b26f 100644
--- a/src/kudu/security/tls_socket.h
+++ b/src/kudu/security/tls_socket.h
@@ -27,18 +27,30 @@ typedef ssl_st SSL;
 namespace kudu {
 namespace security {
 
+class Cert;
+
 class TlsSocket : public Socket {
  public:
 
   ~TlsSocket() override;
 
-  Status Write(const uint8_t *buf, int32_t amt, int32_t *nwritten) override;
+  // Retrieve the local certificate. This will return an error status if there
+  // is no local certificate.
+  Status GetLocalCert(Cert* cert) const WARN_UNUSED_RESULT;
+
+  // Retrieve the remote peer's certificate. This will return an error status if
+  // there is no remote certificate.
+  Status GetRemoteCert(Cert* cert) const WARN_UNUSED_RESULT;
+
+  Status Write(const uint8_t *buf, int32_t amt, int32_t *nwritten) override WARN_UNUSED_RESULT;
 
-  Status Writev(const struct ::iovec *iov, int iov_len, int32_t *nwritten) override;
+  Status Writev(const struct ::iovec *iov,
+                int iov_len,
+                int32_t *nwritten) override WARN_UNUSED_RESULT;
 
-  Status Recv(uint8_t *buf, int32_t amt, int32_t *nread) override;
+  Status Recv(uint8_t *buf, int32_t amt, int32_t *nread) override WARN_UNUSED_RESULT;
 
-  Status Close() override;
+  Status Close() override WARN_UNUSED_RESULT;
 
  private:
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/6fec9dd5/src/kudu/util/status.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/status.cc b/src/kudu/util/status.cc
index d13eeea..9f88da1 100644
--- a/src/kudu/util/status.cc
+++ b/src/kudu/util/status.cc
@@ -22,7 +22,7 @@ const char* Status::CopyState(const char* state) {
 
 Status::Status(Code code, const Slice& msg, const Slice& msg2,
                int16_t posix_code) {
-  assert(code != kOk);
+  DCHECK(code != kOk);
   const uint32_t len1 = msg.size();
   const uint32_t len2 = msg2.size();
   const uint32_t size = len1 + (len2 ? (2 + len2) : 0);